From 37f008afd0380974132395acc530c94ab410f8a1 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Fri, 26 Jun 2015 18:11:31 +0300 Subject: [PATCH 01/24] dbms: Server: Added support for multiple time zones: part 1 (does not add any new functionality, does not change current functionality) [#METR-15618] --- .../DB/Functions/FunctionsConversion.h | 8 +- dbms/include/DB/Functions/FunctionsDateTime.h | 90 ++++++------ dbms/include/DB/IO/WriteHelpers.h | 6 +- dbms/src/Server/OLAPQueryParser.cpp | 2 +- .../Storages/MergeTree/ActiveDataPartSet.cpp | 4 +- .../MergeTree/MergeTreeDataMerger.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- .../Storages/StorageReplicatedMergeTree.cpp | 4 +- libs/libcommon/src/DateLUT.cpp | 92 ++++++------ libs/libcommon/src/DateLUTImpl.cpp | 132 ++++++++++++++++++ libs/libcommon/src/tests/date_lut2.cpp | 2 +- libs/libcommon/src/tests/date_lut3.cpp | 2 +- libs/libcommon/src/tests/date_lut4.cpp | 4 +- libs/libmysqlxx/include/mysqlxx/Date.h | 6 +- libs/libmysqlxx/include/mysqlxx/DateTime.h | 4 +- libs/libmysqlxx/include/mysqlxx/Value.h | 6 +- 16 files changed, 255 insertions(+), 113 deletions(-) create mode 100644 libs/libcommon/src/DateLUTImpl.cpp diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index 3017c7ee0a3..5d4e28809a1 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -69,7 +69,7 @@ struct ConvertImpl static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { typedef DataTypeDate::FieldType FromFieldType; - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { @@ -108,7 +108,7 @@ struct ConvertImpl static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { @@ -149,7 +149,7 @@ struct ConvertImpl32Or64ToDate typedef DataTypeDate::FieldType ToFieldType; template - static To convert(const From & from, const DateLUT & date_lut) + static To convert(const From & from, const DateLUTImpl & date_lut) { return from < 0xFFFF ? from @@ -158,7 +158,7 @@ struct ConvertImpl32Or64ToDate static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - DateLUT & date_lut = DateLUT::instance(); + auto & 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 6c338122250..2e416863f10 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, DateLUT & date_lut) { return date_lut.toYear(t); } - static inline UInt16 execute(UInt16 d, DateLUT & date_lut) { return date_lut.toYear(DayNum_t(d)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toYear(t); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toYear(DayNum_t(d)); } }; struct ToMonthImpl { - 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)); } + static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toMonth(t); } + static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toMonth(DayNum_t(d)); } }; struct ToDayOfMonthImpl { - 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)); } + static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toDayOfMonth(t); } + static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); } }; struct ToDayOfWeekImpl { - 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)); } + static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toDayOfWeek(t); } + static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); } }; struct ToHourImpl { - static inline UInt8 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toHourInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUT & date_lut) + static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toHourInaccurate(t); } + static inline UInt8 execute(UInt16 d, DateLUTImpl & 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, DateLUT & date_lut) { return date_lut.toMinuteInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUT & date_lut) + static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toMinuteInaccurate(t); } + static inline UInt8 execute(UInt16 d, DateLUTImpl & 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, DateLUT & date_lut) { return date_lut.toSecondInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUT & date_lut) + static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toSecondInaccurate(t); } + static inline UInt8 execute(UInt16 d, DateLUTImpl & 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, 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)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); } }; struct ToStartOfMonthImpl { - 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)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); } }; struct ToStartOfQuarterImpl { - 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)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfQuarter(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); } }; struct ToStartOfYearImpl { - 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)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); } }; struct ToTimeImpl { /// При переводе во время, дату будем приравнивать к 1970-01-02. - static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; } - static inline UInt32 execute(UInt16 d, DateLUT & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; } + static inline UInt32 execute(UInt16 d, DateLUTImpl & 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, DateLUT & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUT & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); } + static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) { throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -143,8 +143,8 @@ struct ToStartOfMinuteImpl struct ToStartOfFiveMinuteImpl { - static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfFiveMinuteInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUT & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toStartOfFiveMinuteInaccurate(t); } + static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) { throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -152,8 +152,8 @@ struct ToStartOfFiveMinuteImpl struct ToStartOfHourImpl { - static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toStartOfHourInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUT & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toStartOfHourInaccurate(t); } + static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) { throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -161,33 +161,33 @@ struct ToStartOfHourImpl struct ToRelativeYearNumImpl { - 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)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toYear(t); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toYear(DayNum_t(d)); } }; struct ToRelativeMonthNumImpl { - 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)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeMonthNum(t); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toRelativeMonthNum(DayNum_t(d)); } }; struct ToRelativeWeekNumImpl { - 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)); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeWeekNum(t); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toRelativeWeekNum(DayNum_t(d)); } }; struct ToRelativeDayNumImpl { - 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); } + static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toDayNum(t); } + static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return static_cast(d); } }; struct ToRelativeHourNumImpl { - static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeHourNum(t); } - static inline UInt32 execute(UInt16 d, DateLUT & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeHourNum(t); } + static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) { throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -195,8 +195,8 @@ struct ToRelativeHourNumImpl struct ToRelativeMinuteNumImpl { - static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return date_lut.toRelativeMinuteNum(t); } - static inline UInt32 execute(UInt16 d, DateLUT & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeMinuteNum(t); } + static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) { throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -204,8 +204,8 @@ struct ToRelativeMinuteNumImpl struct ToRelativeSecondNumImpl { - static inline UInt32 execute(UInt32 t, DateLUT & date_lut) { return t; } - static inline UInt32 execute(UInt16 d, DateLUT & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return t; } + static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) { throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -217,7 +217,7 @@ struct DateTimeTransformImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { diff --git a/dbms/include/DB/IO/WriteHelpers.h b/dbms/include/DB/IO/WriteHelpers.h index 8e2b73dcd86..d10916aa8c8 100644 --- a/dbms/include/DB/IO/WriteHelpers.h +++ b/dbms/include/DB/IO/WriteHelpers.h @@ -326,7 +326,7 @@ inline void writeDateText(DayNum_t date, WriteBuffer & buf) return; } - const DateLUT::Values & values = DateLUT::instance().getValues(date); + const auto & values = DateLUT::instance().getValues(date); s[0] += values.year / 1000; s[1] += (values.year / 100) % 10; @@ -368,8 +368,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_deli return; } - DateLUT & date_lut = DateLUT::instance(); - const DateLUT::Values & values = date_lut.getValues(datetime); + auto & date_lut = DateLUT::instance(); + const auto & values = date_lut.getValues(datetime); s[0] += values.year / 1000; s[1] += (values.year / 100) % 10; diff --git a/dbms/src/Server/OLAPQueryParser.cpp b/dbms/src/Server/OLAPQueryParser.cpp index defbcc7ebdd..a577e4d4815 100644 --- a/dbms/src/Server/OLAPQueryParser.cpp +++ b/dbms/src/Server/OLAPQueryParser.cpp @@ -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); - DateLUT & date_lut = DateLUT::instance(); + 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; diff --git a/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp b/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp index b53d12ceb43..90a73932b2c 100644 --- a/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/dbms/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -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) { - DateLUT & date_lut = DateLUT::instance(); + 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; - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); part.left_date = date_lut.YYYYMMDDToDayNum(parse(file_name.substr(matches[1].offset, matches[1].length))); part.right_date = date_lut.YYYYMMDDToDayNum(parse(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 d75573dd62b..f94a3496d6e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -46,7 +46,7 @@ bool MergeTreeDataMerger::selectPartsToMerge(MergeTreeData::DataPartsVector & pa { MergeTreeData::DataParts data_parts = data.getDataParts(); - DateLUT & date_lut = DateLUT::instance(); + auto & 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 cf6d0d32af9..30468cc1228 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); - DateLUT & date_lut = DateLUT::instance(); + 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; - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index b9fb09b4e42..2ca46790a49 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2214,8 +2214,8 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right) { /// Диапазон дат - весь месяц. - DateLUT & lut = DateLUT::instance(); - time_t start_time = DateLUT::instance().YYYYMMDDToDate(parse(month_name + "01")); + auto & lut = DateLUT::instance(); + time_t start_time = lut.YYYYMMDDToDate(parse(month_name + "01")); DayNum_t left_date = lut.toDayNum(start_time); DayNum_t right_date = DayNum_t(static_cast(left_date) + lut.daysInMonth(start_time) - 1); diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index 83402b2ca52..d45c8503815 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -1,57 +1,67 @@ -#include #include #include +#include +#include +#include +std::string DateLUT::default_time_zone; DateLUT::DateLUT() { - size_t i = 0; - time_t start_of_day = DATE_LUT_MIN; + using namespace icu; - do + std::unique_ptr tz(TimeZone::createDefault()); + if (tz == nullptr) + throw Poco::Exception("Failed to query the host time zone."); + + UnicodeString u_out; + tz->getID(u_out); + u_out.toUTF8String(default_time_zone); + + std::unique_ptr time_zone_ids(TimeZone::createEnumeration()); + if (time_zone_ids == nullptr) + throw Poco::Exception("Failed to query the list of time zones."); + + UErrorCode status = U_ZERO_ERROR; + const UnicodeString * zone_id = time_zone_ids->snext(status); + if (zone_id == nullptr) + throw Poco::Exception("No time zone available."); + + while ((zone_id != nullptr) && (status == U_ZERO_ERROR)) { - if (i > DATE_LUT_MAX_DAY_NUM) - throw Poco::Exception("Cannot create DateLUT: i > DATE_LUT_MAX_DAY_NUM."); + std::string zone_id_str; + zone_id->toUTF8String(zone_id_str); + date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(zone_id_str), std::forward_as_tuple(nullptr)); + zone_id = time_zone_ids->snext(status); + } +} - tm time_descr; - localtime_r(&start_of_day, &time_descr); +DateLUTImpl & DateLUT::instance(const std::string & time_zone) +{ + auto & date_lut = Singleton::instance(); + return date_lut.get(time_zone); +} - time_descr.tm_hour = 0; - time_descr.tm_min = 0; - time_descr.tm_sec = 0; - time_descr.tm_isdst = -1; +DateLUTImpl & DateLUT::get(const std::string & time_zone) +{ + auto it = date_lut_impl_list.find(time_zone); + if (it == date_lut_impl_list.end()) + throw Poco::Exception("Invalid time zone " + time_zone); - start_of_day = mktime(&time_descr); + auto & wrapper = it->second; - Values & values = lut[i]; - - values.year = time_descr.tm_year + 1900; - values.month = time_descr.tm_mon + 1; - values.day_of_week = time_descr.tm_wday == 0 ? 7 : time_descr.tm_wday; - values.day_of_month = time_descr.tm_mday; - - values.date = start_of_day; - - /// Переходим на следующий день. - ++time_descr.tm_mday; - - /** Обратите внимание, что в 1981-1984 году в России, - * 1 апреля начиналось в час ночи, а не в полночь. - * Если здесь оставить час равным нулю, то прибавление единицы к дню, привело бы к 23 часам того же дня. - */ - time_descr.tm_hour = 12; - start_of_day = mktime(&time_descr); - - ++i; - } while (start_of_day <= DATE_LUT_MAX); - - /// Заполняем lookup таблицу для годов - memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0])); - for (size_t day = 0; day < i && lut[day].year <= DATE_LUT_MAX_YEAR; ++day) + DateLUTImpl * tmp = wrapper.load(std::memory_order_acquire); + if (tmp == nullptr) { - if (lut[day].month == 1 && lut[day].day_of_month == 1) - years_lut[lut[day].year - DATE_LUT_MIN_YEAR] = day; + std::lock_guard guard(mux); + tmp = wrapper.load(std::memory_order_acquire); + if (tmp == nullptr) + { + tmp = new DateLUTImpl(time_zone); + wrapper.store(tmp, std::memory_order_release); + } } - offset_at_start_of_epoch = 86400 - lut[findIndex(86400)].date; + return *tmp; } + diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp new file mode 100644 index 00000000000..33e4e07b48f --- /dev/null +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -0,0 +1,132 @@ +#include +#include +#include +#include +#include + +namespace details { namespace { + +struct GTimeZoneUnref +{ + void operator()(GTimeZone * tz) const + { + g_time_zone_unref(tz); + } +}; + +using GTimeZonePtr = std::unique_ptr; + +struct GDateTimeUnref +{ + void operator()(GDateTime * dt) const + { + g_date_time_unref(dt); + } +}; + +using GDateTimePtr = std::unique_ptr; + +GTimeZonePtr createGTimeZone(const std::string & description) +{ + GTimeZone * tz = g_time_zone_new(description.c_str()); + if (tz == nullptr) + throw Poco::Exception("Failed to create GTimeZone object."); + + return GTimeZonePtr(tz); +} + +GDateTimePtr createGDateTime(time_t timestamp) +{ + GDateTime * dt= g_date_time_new_from_unix_utc(timestamp); + if (dt == nullptr) + throw Poco::Exception("Failed to create GDateTime object."); + + return GDateTimePtr(dt); +} + +GDateTimePtr createGDateTime(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt) +{ + GDateTime * dt = p_dt.get(); + if (dt == nullptr) + throw Poco::Exception("Null pointer."); + + GDateTime * local_dt = g_date_time_new(p_tz.get(), + g_date_time_get_year(dt), + g_date_time_get_month(dt), + g_date_time_get_day_of_month(dt), + g_date_time_get_hour(dt), + g_date_time_get_minute(dt), + g_date_time_get_second(dt)); + if (local_dt == nullptr) + throw Poco::Exception("Failed to create GDateTime object."); + + return GDateTimePtr(local_dt); +} + +GDateTimePtr toNextDay(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt) +{ + GDateTime * dt = p_dt.get(); + if (dt == nullptr) + throw Poco::Exception("Null pointer."); + + dt = g_date_time_add_days(dt, 1); + if (dt == nullptr) + throw Poco::Exception("Null pointer"); + + dt = g_date_time_new(p_tz.get(), + g_date_time_get_year(dt), + g_date_time_get_month(dt), + g_date_time_get_day_of_month(dt), + 0, 0, 0); + if (dt == nullptr) + throw Poco::Exception("Failed to create GDateTime object."); + + return GDateTimePtr(dt); +} + +}} + +DateLUTImpl::DateLUTImpl(const std::string & time_zone) +{ + details::GTimeZonePtr p_tz = details::createGTimeZone(time_zone); + + size_t i = 0; + time_t start_of_day = DATE_LUT_MIN; + + details::GDateTimePtr p_dt = details::createGDateTime(start_of_day); + + p_dt = details::createGDateTime(p_tz, p_dt); + + do + { + if (i > DATE_LUT_MAX_DAY_NUM) + throw Poco::Exception("Cannot create DateLUTImpl: i > DATE_LUT_MAX_DAY_NUM."); + + GDateTime * dt = p_dt.get(); + + start_of_day = g_date_time_to_unix(dt); + + Values & values = lut[i]; + + values.year = g_date_time_get_year(dt); + values.month = g_date_time_get_month(dt); + values.day_of_week = g_date_time_get_day_of_week(dt); + values.day_of_month = g_date_time_get_day_of_month(dt); + values.date = start_of_day; + + /// Переходим на следующий день. + p_dt = details::toNextDay(p_tz, p_dt); + ++i; + } + while (start_of_day <= DATE_LUT_MAX); + + /// Заполняем lookup таблицу для годов + memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0])); + 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; + } + + offset_at_start_of_epoch = g_time_zone_get_offset(p_tz.get(), g_time_zone_find_interval(p_tz.get(), G_TIME_TYPE_UNIVERSAL, 0)); +} diff --git a/libs/libcommon/src/tests/date_lut2.cpp b/libs/libcommon/src/tests/date_lut2.cpp index d09ed9d9794..6738e66a8d6 100644 --- a/libs/libcommon/src/tests/date_lut2.cpp +++ b/libs/libcommon/src/tests/date_lut2.cpp @@ -33,7 +33,7 @@ static time_t orderedIdentifierToDate(unsigned value) void loop(time_t begin, time_t end, int step) { - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); for (time_t t = begin; t < end; t += step) std::cout << toString(t) diff --git a/libs/libcommon/src/tests/date_lut3.cpp b/libs/libcommon/src/tests/date_lut3.cpp index a968ddaa5d5..185e8ee5f5f 100644 --- a/libs/libcommon/src/tests/date_lut3.cpp +++ b/libs/libcommon/src/tests/date_lut3.cpp @@ -35,7 +35,7 @@ static time_t orderedIdentifierToDate(unsigned value) void loop(time_t begin, time_t end, int step) { - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); for (time_t t = begin; t < end; t += step) { diff --git a/libs/libcommon/src/tests/date_lut4.cpp b/libs/libcommon/src/tests/date_lut4.cpp index 0c6bcc82718..b009aaad4b5 100644 --- a/libs/libcommon/src/tests/date_lut4.cpp +++ b/libs/libcommon/src/tests/date_lut4.cpp @@ -7,12 +7,12 @@ int main(int argc, char ** argv) /** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */ static const time_t TIME = 66130; - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); std::cerr << date_lut.toHourInaccurate(TIME) << std::endl; std::cerr << date_lut.toDayNum(TIME) << std::endl; - const DateLUT::Values * values = reinterpret_cast(&date_lut); + const auto * values = reinterpret_cast(&date_lut); std::cerr << values[0].date << ", " << time_t(values[1].date - values[0].date) << std::endl; diff --git a/libs/libmysqlxx/include/mysqlxx/Date.h b/libs/libmysqlxx/include/mysqlxx/Date.h index d78f74ea266..a23942d2c44 100644 --- a/libs/libmysqlxx/include/mysqlxx/Date.h +++ b/libs/libmysqlxx/include/mysqlxx/Date.h @@ -29,8 +29,8 @@ private: void init(time_t time) { - DateLUT & date_lut = DateLUT::instance(); - const DateLUT::Values & values = date_lut.getValues(time); + auto & date_lut = DateLUT::instance(); + const auto & values = date_lut.getValues(time); m_year = values.year; m_month = values.month; @@ -66,7 +66,7 @@ public: Date(DayNum_t day_num) { - const DateLUT::Values & values = DateLUT::instance().getValues(day_num); + const auto & values = DateLUT::instance().getValues(day_num); m_year = values.year; m_month = values.month; m_day = values.day_of_month; diff --git a/libs/libmysqlxx/include/mysqlxx/DateTime.h b/libs/libmysqlxx/include/mysqlxx/DateTime.h index 1c122b6cc7e..9581dcd8694 100644 --- a/libs/libmysqlxx/include/mysqlxx/DateTime.h +++ b/libs/libmysqlxx/include/mysqlxx/DateTime.h @@ -43,8 +43,8 @@ private: return; } - DateLUT & date_lut = DateLUT::instance(); - const DateLUT::Values & values = date_lut.getValues(time); + auto & date_lut = DateLUT::instance(); + const auto & values = date_lut.getValues(time); m_year = values.year; m_month = values.month; diff --git a/libs/libmysqlxx/include/mysqlxx/Value.h b/libs/libmysqlxx/include/mysqlxx/Value.h index 2d128db7e72..d4cb52b7cb8 100644 --- a/libs/libmysqlxx/include/mysqlxx/Value.h +++ b/libs/libmysqlxx/include/mysqlxx/Value.h @@ -145,7 +145,7 @@ private: time_t getDateTimeImpl() const { - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); if (m_length == 10) { @@ -173,7 +173,7 @@ private: time_t getDateImpl() const { - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); if (m_length == 10 || m_length == 19) { @@ -216,7 +216,7 @@ private: return getDateImpl(); else { - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); return date_lut.toDate(getIntImpl()); } } From 779a90d4381f95f44779a43bbb7db11307417a2b Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Fri, 26 Jun 2015 20:57:49 +0300 Subject: [PATCH 02/24] dbms: Server: Support for multiple time zones: take into account the existence of equivalent time zones [#METR-15618] --- libs/libcommon/src/DateLUT.cpp | 44 ++++++++++++++++++++++++------ libs/libcommon/src/DateLUTImpl.cpp | 7 +++-- 2 files changed, 40 insertions(+), 11 deletions(-) diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index d45c8503815..e992d8969cf 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -1,8 +1,9 @@ #include #include -#include + #include #include +#include std::string DateLUT::default_time_zone; @@ -27,13 +28,35 @@ DateLUT::DateLUT() if (zone_id == nullptr) throw Poco::Exception("No time zone available."); + std::vector time_zones; while ((zone_id != nullptr) && (status == U_ZERO_ERROR)) { - std::string zone_id_str; - zone_id->toUTF8String(zone_id_str); - date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(zone_id_str), std::forward_as_tuple(nullptr)); + time_zones.push_back(*zone_id); zone_id = time_zone_ids->snext(status); } + + for (const auto & time_zone : time_zones) + { + auto count = TimeZone::countEquivalentIDs(time_zone); + + const UnicodeString & u_group_id = TimeZone::getEquivalentID(time_zone, 0); + std::string group_id; + u_group_id.toUTF8String(group_id); + + auto it = time_zone_to_group.find(group_id); + if (it == time_zone_to_group.end()) + { + for (auto i = 1; i < count; ++i) + { + const UnicodeString & u_equivalent_id = TimeZone::getEquivalentID(time_zone, i); + std::string equivalent_id; + u_equivalent_id.toUTF8String(equivalent_id); + time_zone_to_group.insert(std::make_pair(equivalent_id, group_id)); + } + } + + date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(group_id), std::forward_as_tuple(nullptr)); + } } DateLUTImpl & DateLUT::instance(const std::string & time_zone) @@ -44,16 +67,21 @@ DateLUTImpl & DateLUT::instance(const std::string & time_zone) DateLUTImpl & DateLUT::get(const std::string & time_zone) { - auto it = date_lut_impl_list.find(time_zone); - if (it == date_lut_impl_list.end()) + auto it = time_zone_to_group.find(time_zone); + if (it == time_zone_to_group.end()) throw Poco::Exception("Invalid time zone " + time_zone); + const auto & group_id = it->second; - auto & wrapper = it->second; + auto it2 = date_lut_impl_list.find(group_id); + if (it2 == date_lut_impl_list.end()) + throw Poco::Exception("Invalid group of equivalent time zones."); + + auto & wrapper = it2->second; DateLUTImpl * tmp = wrapper.load(std::memory_order_acquire); if (tmp == nullptr) { - std::lock_guard guard(mux); + std::lock_guard guard(mutex); tmp = wrapper.load(std::memory_order_acquire); if (tmp == nullptr) { diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index 33e4e07b48f..1045845dd7b 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -1,8 +1,9 @@ -#include #include #include -#include + #include +#include +#include namespace details { namespace { @@ -121,7 +122,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone) while (start_of_day <= DATE_LUT_MAX); /// Заполняем lookup таблицу для годов - memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0])); + ::memset(years_lut, 0, DATE_LUT_YEARS * sizeof(years_lut[0])); 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) From 81e654b89fffe12a45b9656b6405548eab59ce60 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Fri, 26 Jun 2015 21:09:00 +0300 Subject: [PATCH 03/24] dbms: Server: Support for multiple time zones: fix [#METR-15618] --- libs/libcommon/src/DateLUT.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index e992d8969cf..53e6ee2cfb8 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -46,7 +46,7 @@ DateLUT::DateLUT() auto it = time_zone_to_group.find(group_id); if (it == time_zone_to_group.end()) { - for (auto i = 1; i < count; ++i) + for (auto i = 0; i < count; ++i) { const UnicodeString & u_equivalent_id = TimeZone::getEquivalentID(time_zone, i); std::string equivalent_id; From 0b3887fbb3707987bc3421ce68ce0b782e9528eb Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Mon, 29 Jun 2015 12:56:04 +0300 Subject: [PATCH 04/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- libs/libcommon/src/DateLUT.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index 53e6ee2cfb8..bfbcd1d2279 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -51,7 +51,9 @@ DateLUT::DateLUT() const UnicodeString & u_equivalent_id = TimeZone::getEquivalentID(time_zone, i); std::string equivalent_id; u_equivalent_id.toUTF8String(equivalent_id); - time_zone_to_group.insert(std::make_pair(equivalent_id, group_id)); + auto res = time_zone_to_group.insert(std::make_pair(equivalent_id, group_id)); + if (!res.second) + throw Poco::Exception("Failed to initialize time zone information."); } } @@ -85,7 +87,7 @@ DateLUTImpl & DateLUT::get(const std::string & time_zone) tmp = wrapper.load(std::memory_order_acquire); if (tmp == nullptr) { - tmp = new DateLUTImpl(time_zone); + tmp = new DateLUTImpl(group_id); wrapper.store(tmp, std::memory_order_release); } } From b1788e7d05c21ff0abb17ec5fa2bd63acc9457d0 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Mon, 29 Jun 2015 14:54:57 +0300 Subject: [PATCH 05/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- libs/libcommon/src/DateLUT.cpp | 39 +++++++++++++++++++++++----------- 1 file changed, 27 insertions(+), 12 deletions(-) diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index bfbcd1d2279..ed4530a63c7 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -37,27 +37,42 @@ DateLUT::DateLUT() for (const auto & time_zone : time_zones) { - auto count = TimeZone::countEquivalentIDs(time_zone); - const UnicodeString & u_group_id = TimeZone::getEquivalentID(time_zone, 0); std::string group_id; - u_group_id.toUTF8String(group_id); - auto it = time_zone_to_group.find(group_id); - if (it == time_zone_to_group.end()) + if (u_group_id.isEmpty()) { - for (auto i = 0; i < count; ++i) + time_zone.toUTF8String(group_id); + + auto res = time_zone_to_group.insert(std::make_pair(group_id, group_id)); + if (!res.second) + throw Poco::Exception("Failed to initialize time zone information."); + auto res2 = date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(group_id), std::forward_as_tuple(nullptr)); + if (!res2.second) + throw Poco::Exception("Failed to initialize time zone information."); + } + else + { + u_group_id.toUTF8String(group_id); + + auto it = time_zone_to_group.find(group_id); + if (it == time_zone_to_group.end()) { - const UnicodeString & u_equivalent_id = TimeZone::getEquivalentID(time_zone, i); - std::string equivalent_id; - u_equivalent_id.toUTF8String(equivalent_id); - auto res = time_zone_to_group.insert(std::make_pair(equivalent_id, group_id)); + auto count = TimeZone::countEquivalentIDs(time_zone); + for (auto i = 0; i < count; ++i) + { + const UnicodeString & u_equivalent_id = TimeZone::getEquivalentID(time_zone, i); + std::string equivalent_id; + u_equivalent_id.toUTF8String(equivalent_id); + auto res = time_zone_to_group.insert(std::make_pair(equivalent_id, group_id)); + if (!res.second) + throw Poco::Exception("Failed to initialize time zone information."); + } + auto res = date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(group_id), std::forward_as_tuple(nullptr)); if (!res.second) throw Poco::Exception("Failed to initialize time zone information."); } } - - date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(group_id), std::forward_as_tuple(nullptr)); } } From f02d490f94bdd975517178cd0a6810548cb72c6f Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Mon, 29 Jun 2015 16:54:08 +0300 Subject: [PATCH 06/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- libs/libcommon/src/DateLUT.cpp | 51 +++++++++++++++++----------------- 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index ed4530a63c7..3e7409da548 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -3,7 +3,6 @@ #include #include -#include std::string DateLUT::default_time_zone; @@ -35,45 +34,51 @@ DateLUT::DateLUT() zone_id = time_zone_ids->snext(status); } + size_t group_id = 0; + for (const auto & time_zone : time_zones) { - const UnicodeString & u_group_id = TimeZone::getEquivalentID(time_zone, 0); - std::string group_id; + const UnicodeString & u_group_name = TimeZone::getEquivalentID(time_zone, 0); + std::string group_name; - if (u_group_id.isEmpty()) + if (u_group_name.isEmpty()) { - time_zone.toUTF8String(group_id); + time_zone.toUTF8String(group_name); - auto res = time_zone_to_group.insert(std::make_pair(group_id, group_id)); + auto res = time_zone_to_group.insert(std::make_pair(group_name, group_id)); if (!res.second) throw Poco::Exception("Failed to initialize time zone information."); - auto res2 = date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(group_id), std::forward_as_tuple(nullptr)); - if (!res2.second) - throw Poco::Exception("Failed to initialize time zone information."); + ++group_id; } else { - u_group_id.toUTF8String(group_id); + u_group_name.toUTF8String(group_name); - auto it = time_zone_to_group.find(group_id); + auto it = time_zone_to_group.find(group_name); if (it == time_zone_to_group.end()) { auto count = TimeZone::countEquivalentIDs(time_zone); + if (count == 0) + throw Poco::Exception("Inconsistent time zone information."); + for (auto i = 0; i < count; ++i) { - const UnicodeString & u_equivalent_id = TimeZone::getEquivalentID(time_zone, i); - std::string equivalent_id; - u_equivalent_id.toUTF8String(equivalent_id); - auto res = time_zone_to_group.insert(std::make_pair(equivalent_id, group_id)); + const UnicodeString & u_name = TimeZone::getEquivalentID(time_zone, i); + std::string name; + u_name.toUTF8String(name); + auto res = time_zone_to_group.insert(std::make_pair(name, group_id)); if (!res.second) throw Poco::Exception("Failed to initialize time zone information."); } - auto res = date_lut_impl_list.emplace(std::piecewise_construct, std::forward_as_tuple(group_id), std::forward_as_tuple(nullptr)); - if (!res.second) - throw Poco::Exception("Failed to initialize time zone information."); + ++group_id; } } } + + if (group_id == 0) + throw Poco::Exception("Could not find any time zone information."); + + date_lut_impl_list = std::make_unique(group_id); } DateLUTImpl & DateLUT::instance(const std::string & time_zone) @@ -87,13 +92,9 @@ DateLUTImpl & DateLUT::get(const std::string & time_zone) auto it = time_zone_to_group.find(time_zone); if (it == time_zone_to_group.end()) throw Poco::Exception("Invalid time zone " + time_zone); + const auto & group_id = it->second; - - auto it2 = date_lut_impl_list.find(group_id); - if (it2 == date_lut_impl_list.end()) - throw Poco::Exception("Invalid group of equivalent time zones."); - - auto & wrapper = it2->second; + auto & wrapper = (*date_lut_impl_list)[group_id]; DateLUTImpl * tmp = wrapper.load(std::memory_order_acquire); if (tmp == nullptr) @@ -102,7 +103,7 @@ DateLUTImpl & DateLUT::get(const std::string & time_zone) tmp = wrapper.load(std::memory_order_acquire); if (tmp == nullptr) { - tmp = new DateLUTImpl(group_id); + tmp = new DateLUTImpl(time_zone); wrapper.store(tmp, std::memory_order_release); } } From 79290d82a5c969666d90ceb0b6f94f873ee66bed Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Tue, 30 Jun 2015 16:42:08 +0300 Subject: [PATCH 07/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- libs/libcommon/src/DateLUT.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index 3e7409da548..4094223c583 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -89,9 +89,11 @@ DateLUTImpl & DateLUT::instance(const std::string & time_zone) DateLUTImpl & DateLUT::get(const std::string & time_zone) { - auto it = time_zone_to_group.find(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); if (it == time_zone_to_group.end()) - throw Poco::Exception("Invalid time zone " + time_zone); + throw Poco::Exception("Invalid time zone " + actual_time_zone); const auto & group_id = it->second; auto & wrapper = (*date_lut_impl_list)[group_id]; @@ -103,7 +105,7 @@ DateLUTImpl & DateLUT::get(const std::string & time_zone) tmp = wrapper.load(std::memory_order_acquire); if (tmp == nullptr) { - tmp = new DateLUTImpl(time_zone); + tmp = new DateLUTImpl(actual_time_zone); wrapper.store(tmp, std::memory_order_release); } } From eec6b89a4c8e2e2da776b5ac3fa23d0cde4bf3d8 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Tue, 30 Jun 2015 17:02:47 +0300 Subject: [PATCH 08/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- dbms/include/DB/Functions/FunctionsDateTime.h | 24 ++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 2e416863f10..fb921963698 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -217,7 +218,16 @@ struct DateTimeTransformImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - auto & date_lut = DateLUT::instance(); + std::string time_zone; + + if (arguments.size() == 2) + { + const ColumnPtr column = block.getByPosition(arguments[1]).column; + if (const ColumnConstString * col = typeid_cast(&*column)) + time_zone = col->getData(); + } + + auto & date_lut = DateLUT::instance(time_zone); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { @@ -260,11 +270,19 @@ public: /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnType(const DataTypes & arguments) const { - if (arguments.size() != 1) + if ((arguments.size() < 1) || (arguments.size() > 2)) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 1.", + + toString(arguments.size()) + ", should be 1 or 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + { + throw Exception{ + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + return new ToDataType; } From d8b05105187054785406c021bd130f0569ad9dd3 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Tue, 30 Jun 2015 18:03:22 +0300 Subject: [PATCH 09/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- dbms/include/DB/Functions/FunctionsDateTime.h | 85 ++++++++++++++----- 1 file changed, 64 insertions(+), 21 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index fb921963698..0d020c593a1 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -11,6 +11,7 @@ #include +#include namespace DB { @@ -212,11 +213,19 @@ struct ToRelativeSecondNumImpl } }; - -template -struct DateTimeTransformImpl +template +struct DateLUTAccessor { - static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + static DateLUTImpl & execute(Block & block, const ColumnNumbers & arguments) + { + return DateLUT::instance(); + } +}; + +template<> +struct DateLUTAccessor +{ + static DateLUTImpl & execute(Block & block, const ColumnNumbers & arguments) { std::string time_zone; @@ -227,7 +236,16 @@ struct DateTimeTransformImpl time_zone = col->getData(); } - auto & date_lut = DateLUT::instance(time_zone); + return DateLUT::instance(time_zone); + } +}; + +template +struct DateTimeTransformImpl +{ + static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + auto & date_lut = DateLUTAccessor::execute(block, arguments); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { @@ -267,23 +285,9 @@ public: return name; } - /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { - if ((arguments.size() < 1) || (arguments.size() > 2)) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 1 or 2.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) - { - throw Exception{ - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; - } - - return new ToDataType; + return getReturnTypeImpl(arguments); } /// Выполнить функцию над блоком. @@ -299,6 +303,45 @@ public: throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + +private: + /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, typename std::enable_if::value, void>::type * = nullptr) const + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return new ToDataType; + } + + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, typename std::enable_if::value, void>::type * = nullptr) const + { + if ((arguments.size() < 1) || (arguments.size() > 2)) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1 or 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (typeid_cast(&*arguments[0]) == nullptr) + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + { + throw Exception{ + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + return new ToDataType; + } }; From 557b0bf9cc512c3e6f93b05f57eb35dc400cda28 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Wed, 1 Jul 2015 17:35:02 +0300 Subject: [PATCH 10/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- dbms/include/DB/Functions/FunctionsDateTime.h | 151 +++++++++++------- 1 file changed, 96 insertions(+), 55 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 0d020c593a1..958b44a7a3f 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -50,32 +50,32 @@ namespace DB struct ToYearImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toYear(t); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toYear(DayNum_t(d)); } + 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)); } }; struct ToMonthImpl { - static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toMonth(t); } - static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toMonth(DayNum_t(d)); } + 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)); } }; struct ToDayOfMonthImpl { - static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toDayOfMonth(t); } - static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toDayOfMonth(DayNum_t(d)); } + 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)); } }; struct ToDayOfWeekImpl { - static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toDayOfWeek(t); } - static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toDayOfWeek(DayNum_t(d)); } + 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)); } }; struct ToHourImpl { - static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toHourInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -83,8 +83,8 @@ struct ToHourImpl struct ToMinuteImpl { - static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toMinuteInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -92,8 +92,8 @@ struct ToMinuteImpl struct ToSecondImpl { - static inline UInt8 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toSecondInaccurate(t); } - static inline UInt8 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -101,34 +101,49 @@ struct ToSecondImpl struct ToMondayImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfWeek(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfWeek(DayNum_t(d)); } + 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)); } }; struct ToStartOfMonthImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfMonth(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfMonth(DayNum_t(d)); } + 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)); } }; struct ToStartOfQuarterImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfQuarter(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); } + 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)); } }; struct ToStartOfYearImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfYear(date_lut.toDayNum(t)); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toFirstDayNumOfYear(DayNum_t(d)); } + 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)); } }; struct ToTimeImpl { /// При переводе во время, дату будем приравнивать к 1970-01-02. - static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toTimeInaccurate(t) + 86400; } - static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) + static inline UInt32 execute(UInt32 t, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) + { + time_t remote_t = remote_date_lut.toTimeInaccurate(t) + 86400; + + if (&remote_date_lut == &local_date_lut) + return remote_t; + else + { + const auto & values = remote_date_lut.getValues(remote_t); + return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month, + remote_date_lut.toHourInaccurate(remote_t), + remote_date_lut.toMinuteInaccurate(remote_t), + remote_date_lut.toSecondInaccurate(remote_t)); + } + } + + static inline UInt32 execute(UInt16 d, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) { throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -136,8 +151,8 @@ struct ToTimeImpl struct ToStartOfMinuteImpl { - static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toStartOfMinuteInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -145,8 +160,8 @@ struct ToStartOfMinuteImpl struct ToStartOfFiveMinuteImpl { - static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toStartOfFiveMinuteInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -154,8 +169,8 @@ struct ToStartOfFiveMinuteImpl struct ToStartOfHourImpl { - static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toStartOfHourInaccurate(t); } - static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -163,33 +178,33 @@ struct ToStartOfHourImpl struct ToRelativeYearNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toYear(t); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toYear(DayNum_t(d)); } + 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)); } }; struct ToRelativeMonthNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeMonthNum(t); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toRelativeMonthNum(DayNum_t(d)); } + 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)); } }; struct ToRelativeWeekNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeWeekNum(t); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return date_lut.toRelativeWeekNum(DayNum_t(d)); } + 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)); } }; struct ToRelativeDayNumImpl { - static inline UInt16 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toDayNum(t); } - static inline UInt16 execute(UInt16 d, DateLUTImpl & date_lut) { return static_cast(d); } + 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(d); } }; struct ToRelativeHourNumImpl { - static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeHourNum(t); } - static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -197,8 +212,8 @@ struct ToRelativeHourNumImpl struct ToRelativeMinuteNumImpl { - static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return date_lut.toRelativeMinuteNum(t); } - static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -206,15 +221,24 @@ struct ToRelativeMinuteNumImpl struct ToRelativeSecondNumImpl { - static inline UInt32 execute(UInt32 t, DateLUTImpl & date_lut) { return t; } - static inline UInt32 execute(UInt16 d, DateLUTImpl & date_lut) + 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) { throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } }; -template -struct DateLUTAccessor +template +struct DateLUTAccessor; + +template +struct DateLUTAccessor::value + && std::is_same::value) + || (std::is_same::value + && std::is_same::value + && std::is_same::value))>::type> { static DateLUTImpl & execute(Block & block, const ColumnNumbers & arguments) { @@ -222,8 +246,14 @@ struct DateLUTAccessor } }; -template<> -struct DateLUTAccessor +template +struct DateLUTAccessor::value + && std::is_same::value) + || (std::is_same::value + && std::is_same::value + && std::is_same::value)>::type> { static DateLUTImpl & execute(Block & block, const ColumnNumbers & arguments) { @@ -240,12 +270,14 @@ struct DateLUTAccessor } }; + template struct DateTimeTransformImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - auto & date_lut = DateLUTAccessor::execute(block, arguments); + auto & remote_date_lut = DateLUTAccessor::execute(block, arguments); + auto & local_date_lut = DateLUT::instance(); if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { @@ -258,11 +290,11 @@ struct DateTimeTransformImpl vec_to.resize(size); for (size_t i = 0; i < size; ++i) - vec_to[i] = Transform::execute(vec_from[i], date_lut); + vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut); } else if (const ColumnConst * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) { - block.getByPosition(result).column = new ColumnConst(col_from->size(), Transform::execute(col_from->getData(), date_lut)); + block.getByPosition(result).column = new ColumnConst(col_from->size(), Transform::execute(col_from->getData(), remote_date_lut, local_date_lut)); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() @@ -306,8 +338,13 @@ public: private: /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - template - DataTypePtr getReturnTypeImpl(const DataTypes & arguments, typename std::enable_if::value, void>::type * = nullptr) const + + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, + typename std::enable_if< + !(std::is_same::value + || (std::is_same::value && std::is_same::value)) + , void>::type * = nullptr) const { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -317,8 +354,12 @@ private: return new ToDataType; } - template - DataTypePtr getReturnTypeImpl(const DataTypes & arguments, typename std::enable_if::value, void>::type * = nullptr) const + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, + typename std::enable_if< + std::is_same::value + || (std::is_same::value && std::is_same::value) + , void>::type * = nullptr) const { if ((arguments.size() < 1) || (arguments.size() > 2)) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " From 419bc03713c5abd2f3a3f2d268498d0997a76d40 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Wed, 1 Jul 2015 19:07:29 +0300 Subject: [PATCH 11/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- .../DB/Functions/FunctionsConversion.h | 118 +++++++++++++++++- 1 file changed, 112 insertions(+), 6 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index 5d4e28809a1..e4721fcebac 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -242,6 +242,77 @@ struct ConvertImpl } }; +time_t convert_time(time_t remote_time, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) +{ + if (&remote_date_lut == &local_date_lut) + return remote_time; + else + { + const auto & values = remote_date_lut.getValues(remote_time); + return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month, + remote_date_lut.toHourInaccurate(remote_time), + remote_date_lut.toMinuteInaccurate(remote_time), + remote_date_lut.toSecondInaccurate(remote_time)); + } +} + +template +struct ConvertImpl +{ + typedef typename DataTypeDateTime::FieldType FromFieldType; + + static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + std::string time_zone; + + if (arguments.size() == 2) + { + const ColumnPtr column = block.getByPosition(arguments[1]).column; + if (const ColumnConstString * col = typeid_cast(&*column)) + time_zone = col->getData(); + } + + auto & remote_date_lut = DateLUT::instance(time_zone); + auto & local_date_lut = DateLUT::instance(); + + if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + { + ColumnString * col_to = new ColumnString; + block.getByPosition(result).column = col_to; + + const typename ColumnVector::Container_t & vec_from = col_from->getData(); + ColumnString::Chars_t & data_to = col_to->getChars(); + ColumnString::Offsets_t & offsets_to = col_to->getOffsets(); + size_t size = vec_from.size(); + data_to.resize(size * 2); + offsets_to.resize(size); + + WriteBufferFromVector write_buffer(data_to); + + for (size_t i = 0; i < size; ++i) + { + auto ti = convert_time(vec_from[i], remote_date_lut, local_date_lut); + formatImpl(ti, write_buffer); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + data_to.resize(write_buffer.count()); + } + else if (const ColumnConst * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + { + std::vector buf; + WriteBufferFromVector > write_buffer(buf); + auto ti = convert_time(col_from->getData(), remote_date_lut, local_date_lut); + formatImpl(ti, write_buffer); + block.getByPosition(result).column = new ColumnConstString(col_from->size(), std::string(&buf[0], write_buffer.count())); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } +}; + /** Преобразование строк в числа, даты, даты-с-временем: через парсинг. */ @@ -437,12 +508,7 @@ public: /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnType(const DataTypes & arguments) const { - if (arguments.size() != 1) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 1.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - return new ToDataType; + return getReturnTypeImpl(arguments); } /// Выполнить функцию над блоком. @@ -468,6 +534,46 @@ public: throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + +private: + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, + typename std::enable_if::value, void>::type * = nullptr) const + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return new ToDataType; + } + + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, + typename std::enable_if::value, void>::type * = nullptr) const + { + if ((arguments.size() < 1) || (arguments.size() > 2)) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1 or 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (typeid_cast(&*arguments[0]) == nullptr) + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + { + throw Exception{ + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + return new ToDataType2; + } }; From 2f56341f00c2ed34308ebe97f1f36cf349e8f096 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Wed, 1 Jul 2015 20:32:04 +0300 Subject: [PATCH 12/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- .../DB/Functions/FunctionsConversion.h | 98 ++++++++++++++++++- dbms/src/Functions/FunctionsConversion.cpp | 1 + 2 files changed, 94 insertions(+), 5 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index e4721fcebac..bcf8063bcb1 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -375,6 +375,67 @@ struct ConvertImpl } }; +struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; }; + +template<> +struct ConvertImpl +{ + typedef typename DataTypeInt32::FieldType ToFieldType; + + static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + std::string time_zone; + + if (arguments.size() == 2) + { + const ColumnPtr column = block.getByPosition(arguments[1]).column; + if (const ColumnConstString * col = typeid_cast(&*column)) + time_zone = col->getData(); + } + + auto & remote_date_lut = DateLUT::instance(time_zone); + auto & local_date_lut = DateLUT::instance(); + + if (const ColumnString * col_from = typeid_cast(&*block.getByPosition(arguments[0]).column)) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const ColumnString::Chars_t & data_from = col_from->getChars(); + typename ColumnVector::Container_t & vec_to = col_to->getData(); + size_t size = col_from->size(); + vec_to.resize(size); + + ReadBuffer read_buffer(const_cast(reinterpret_cast(&data_from[0])), data_from.size(), 0); + + char zero = 0; + for (size_t i = 0; i < size; ++i) + { + DataTypeDateTime::FieldType x = 0; + parseImpl(x, read_buffer); + auto ti = convert_time(x, local_date_lut, remote_date_lut); + vec_to[i] = ti; + readChar(zero, read_buffer); + if (zero != 0) + throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER); + } + } + else if (const ColumnConstString * col_from = typeid_cast(&*block.getByPosition(arguments[0]).column)) + { + const String & s = col_from->getData(); + ReadBufferFromString read_buffer(s); + DataTypeDateTime::FieldType x = 0; + parseImpl(x, read_buffer); + auto ti = convert_time(x, local_date_lut, remote_date_lut); + block.getByPosition(result).column = new ColumnConst(col_from->size(), ti); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + NameToUnixTimestamp::name, + ErrorCodes::ILLEGAL_COLUMN); + } +}; + /** Если типы совпадают - просто скопируем ссылку на столбец. */ template @@ -536,9 +597,9 @@ public: } private: - template + template DataTypePtr getReturnTypeImpl(const DataTypes & arguments, - typename std::enable_if::value, void>::type * = nullptr) const + typename std::enable_if::value || std::is_same::value), void>::type * = nullptr) const { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -548,9 +609,9 @@ private: return new ToDataType; } - template + template DataTypePtr getReturnTypeImpl(const DataTypes & arguments, - typename std::enable_if::value, void>::type * = nullptr) const + typename std::enable_if::value>::type * = nullptr) const { if ((arguments.size() < 1) || (arguments.size() > 2)) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -574,6 +635,33 @@ private: return new ToDataType2; } + + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, + typename std::enable_if::value, void>::type * = nullptr) const + { + if ((arguments.size() < 1) || (arguments.size() > 2)) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1 or 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (typeid_cast(&*arguments[0]) == nullptr) + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + { + throw Exception{ + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + return new ToDataType2; + } }; @@ -734,6 +822,6 @@ typedef FunctionConvert FunctionToFloat64; typedef FunctionConvert FunctionToDate; typedef FunctionConvert FunctionToDateTime; typedef FunctionConvert FunctionToString; - +typedef FunctionConvert FunctionToUnixTimestamp; } diff --git a/dbms/src/Functions/FunctionsConversion.cpp b/dbms/src/Functions/FunctionsConversion.cpp index 1e9ad9b42e6..e3f99046991 100644 --- a/dbms/src/Functions/FunctionsConversion.cpp +++ b/dbms/src/Functions/FunctionsConversion.cpp @@ -20,6 +20,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } From 17f681ba8cf8371b0499dbdd5b40b22ab937365d Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Wed, 1 Jul 2015 21:03:18 +0300 Subject: [PATCH 13/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- dbms/src/Interpreters/QueryLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/QueryLog.cpp b/dbms/src/Interpreters/QueryLog.cpp index 7e342b64343..77865140134 100644 --- a/dbms/src/Interpreters/QueryLog.cpp +++ b/dbms/src/Interpreters/QueryLog.cpp @@ -210,7 +210,7 @@ void QueryLog::flush() { LOG_TRACE(log, "Flushing query log"); - DateLUT & date_lut = DateLUT::instance(); + auto & date_lut = DateLUT::instance(); Block block = createBlock(); From ec8925aa4022ecac493e08021ec74a12246574e7 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 2 Jul 2015 03:19:31 +0300 Subject: [PATCH 14/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- .../0_stateless/00189_time_zones.reference | 35 ++++++++++++ .../queries/0_stateless/00189_time_zones.sql | 57 +++++++++++++++++++ 2 files changed, 92 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00189_time_zones.reference create mode 100644 dbms/tests/queries/0_stateless/00189_time_zones.sql diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference new file mode 100644 index 00000000000..0f401f2b805 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -0,0 +1,35 @@ +2014-12-29 +2014-12-22 +2014-12-22 +2014-12-29 +2014-12-22 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-07-01 +2014-07-01 +2014-07-01 +2014-10-01 +2014-07-01 +1970-01-02 12:00:00 1970-01-02 12:00:00 +1970-01-02 10:00:00 1970-01-02 11:00:00 +1970-01-02 09:00:00 1970-01-02 10:00:00 +1970-01-02 18:00:00 1970-01-02 18:00:00 +1970-01-02 01:30:00 1970-01-02 01:30:00 +2015-07-15 13:30:00 +2015-07-15 12:30:00 +2015-07-15 11:30:00 +2015-07-15 19:30:00 +2015-07-15 02:30:00 +1426415400 +1426422600 +1426426200 +1426393800 +1426455000 diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.sql b/dbms/tests/queries/0_stateless/00189_time_zones.sql new file mode 100644 index 00000000000..0a9f0a6accb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -0,0 +1,57 @@ + +/* toMonday */ + +SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Europe/Moscow'); +SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Europe/Paris'); +SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Europe/London'); +SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Asia/Tokyo'); +SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Pacific/Pitcairn'); + +/* toStartOfMonth */ + +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Moscow'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Paris'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/London'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Asia/Tokyo'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Pacific/Pitcairn'); + +/* toStartOfQuarter */ + +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Moscow'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Paris'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/London'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Asia/Tokyo'); +SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Pacific/Pitcairn'); + +/* toStartOfYear */ + +SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Europe/Moscow'); +SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Europe/Paris'); +SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Europe/London'); +SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Asia/Tokyo'); +SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Pacific/Pitcairn'); + +/* toTime */ + +SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Europe/Moscow'), toTime(toDateTime('2015-04-06 12:00:00'), 'Europe/Moscow'); +SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Europe/Paris'), toTime(toDateTime('2015-04-06 12:00:00'), 'Europe/Paris'); +SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Europe/London'), toTime(toDateTime('2015-04-06 12:00:00'), 'Europe/London'); +SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Asia/Tokyo'), toTime(toDateTime('2015-04-06 12:00:00'), 'Asia/Tokyo'); +SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Pacific/Pitcairn'), toTime(toDateTime('2015-04-06 12:00:00'), 'Pacific/Pitcairn'); + +/* toString */ + +SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Europe/Moscow'); +SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Europe/Paris'); +SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Europe/London'); +SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Asia/Tokyo'); +SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Pacific/Pitcairn'); + +/* toUnixTimestamp */ + +SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Europe/Moscow'); +SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Europe/Paris'); +SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Europe/London'); +SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Asia/Tokyo'); +SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Pacific/Pitcairn'); + From 7ef2b9595ab9b2b7139938681f9c79c2439ba470 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 2 Jul 2015 16:35:28 +0300 Subject: [PATCH 15/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- dbms/include/DB/Functions/FunctionsDateTime.h | 204 +++++++++++++----- .../0_stateless/00189_time_zones.reference | 5 + .../queries/0_stateless/00189_time_zones.sql | 94 +++++--- 3 files changed, 220 insertions(+), 83 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 958b44a7a3f..6a353a6108b 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -8,6 +8,7 @@ #include #include +#include #include @@ -228,82 +229,189 @@ struct ToRelativeSecondNumImpl } }; -template -struct DateLUTAccessor; - template -struct DateLUTAccessor::value - && std::is_same::value) - || (std::is_same::value - && std::is_same::value - && std::is_same::value))>::type> +struct Transformer { - static DateLUTImpl & execute(Block & block, const ColumnNumbers & arguments) + static void vector(const typename ColumnVector::Container_t & vec_from, typename ColumnVector::Container_t & vec_to) { - return DateLUT::instance(); + auto & local_date_lut = DateLUT::instance(); + for (size_t i = 0; i < vec_from.size(); ++i) + vec_to[i] = Transform::execute(vec_from[i], local_date_lut, local_date_lut); } -}; -template -struct DateLUTAccessor::value - && std::is_same::value) - || (std::is_same::value - && std::is_same::value - && std::is_same::value)>::type> -{ - static DateLUTImpl & execute(Block & block, const ColumnNumbers & arguments) + static void constant(const FromType & from, ToType & to) { - std::string time_zone; + auto & local_date_lut = DateLUT::instance(); + to = Transform::execute(from, local_date_lut, local_date_lut); + } - if (arguments.size() == 2) + static void vector_vector(const typename ColumnVector::Container_t & vec_from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, typename ColumnVector::Container_t & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + ColumnString::Offset_t prev_offset = 0; + + for (size_t i = 0; i < vec_from.size(); ++i) { - const ColumnPtr column = block.getByPosition(arguments[1]).column; - if (const ColumnConstString * col = typeid_cast(&*column)) - time_zone = col->getData(); + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + 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; } + } - return DateLUT::instance(time_zone); + static void vector_fixed(const typename ColumnVector::Container_t & vec_from, const ColumnString::Chars_t & data, + size_t n, typename ColumnVector::Container_t & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + auto & remote_date_lut = DateLUT::instance(std::string(reinterpret_cast(&data[0]), n)); + for (size_t i = 0; i < vec_from.size(); ++i) + vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut); + } + + static void vector_constant(const typename ColumnVector::Container_t & vec_from, const std::string & data, + typename ColumnVector::Container_t & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + 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); + } + + static void constant_vector(const FromType & from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, typename ColumnVector::Container_t & vec_to) + { + 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(&data[prev_offset]), cur_offset - prev_offset - 1); + auto & remote_date_lut = DateLUT::instance(time_zone); + vec_to[i] = Transform::execute(from, remote_date_lut, local_date_lut); + prev_offset = cur_offset; + } + } + + static void constant_fixed(const FromType & from, const ColumnString::Chars_t & data, size_t n, ToType & to) + { + auto & local_date_lut = DateLUT::instance(); + auto & remote_date_lut = DateLUT::instance(std::string(reinterpret_cast(&data[0]), n)); + to = Transform::execute(from, remote_date_lut, local_date_lut); + } + + 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); + to = Transform::execute(from, remote_date_lut, local_date_lut); } }; - template struct DateTimeTransformImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - auto & remote_date_lut = DateLUTAccessor::execute(block, arguments); - auto & local_date_lut = DateLUT::instance(); + using Op = Transformer; - if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + const ColumnPtr source_col = block.getByPosition(arguments[0]).column; + const ColumnVector * sources = typeid_cast *>(&*source_col); + const ColumnConst * const_source = typeid_cast *>(&*source_col); + + if (arguments.size() == 1) { - ColumnVector * col_to = new ColumnVector; - block.getByPosition(result).column = col_to; + if (sources) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; - const typename ColumnVector::Container_t & vec_from = col_from->getData(); - typename ColumnVector::Container_t & vec_to = col_to->getData(); - size_t size = vec_from.size(); - vec_to.resize(size); + auto & vec_from = sources->getData(); + auto & vec_to = col_to->getData(); + size_t size = vec_from.size(); + vec_to.resize(size); - for (size_t i = 0; i < size; ++i) - vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut); + Op::vector(vec_from, vec_to); + } + else if (const_source) + { + ToType res; + Op::constant(const_source->getData(), res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + } + else + { + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } } - else if (const ColumnConst * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + else if (arguments.size() == 2) { - block.getByPosition(result).column = new ColumnConst(col_from->size(), Transform::execute(col_from->getData(), remote_date_lut, local_date_lut)); + const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; + const ColumnString * time_zones = typeid_cast(&*time_zone_col); + const ColumnFixedString * fixed_time_zone = typeid_cast(&*time_zone_col); + const ColumnConstString * const_time_zone = typeid_cast(&*time_zone_col); + + if (sources) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + auto & vec_from = sources->getData(); + auto & vec_to = col_to->getData(); + vec_to.resize(vec_from.size()); + + if (time_zones) + Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); + else if (fixed_time_zone) + Op::vector_fixed(vec_from, fixed_time_zone->getChars(), fixed_time_zone->getN(), vec_to); + else if (const_time_zone) + Op::vector_constant(vec_from, const_time_zone->getData(), vec_to); + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else if (const_source) + { + if (time_zones) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + auto & vec_to = col_to->getData(); + vec_to.resize(time_zones->getOffsets().size()); + + Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); + } + else if (fixed_time_zone) + { + ToType res; + Op::constant_fixed(const_source->getData(), fixed_time_zone->getChars(), fixed_time_zone->getN(), res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + } + else if (const_time_zone) + { + ToType res; + Op::constant_constant(const_source->getData(), const_time_zone->getData(), res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + Name::name, - ErrorCodes::ILLEGAL_COLUMN); } }; - template class FunctionDateOrDateTimeToSomething : public IFunction { diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index 0f401f2b805..c9bccf34454 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -33,3 +33,8 @@ 1426426200 1426393800 1426455000 +1426415400 +1426415400 +1426415400 +1426415400 +1426415400 diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.sql b/dbms/tests/queries/0_stateless/00189_time_zones.sql index 0a9f0a6accb..90e2d4e2027 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.sql +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -1,57 +1,81 @@ +/* timestamp 1419800400 == 2014-12-29 00:00:00 (Europe/Moscow) */ +/* timestamp 1412106600 == 2014-09-30 23:50:00 (Europe/Moscow) */ +/* timestamp 1420102800 == 2015-01-01 12:00:00 (Europe/Moscow) */ +/* timestamp 1428310800 == 2015-04-06 12:00:00 (Europe/Moscow) */ +/* timestamp 1436956200 == 2015-07-15 13:30:00 (Europe/Moscow) */ +/* timestamp 1426415400 == 2015-03-15 13:30:00 (Europe/Moscow) */ + /* toMonday */ -SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Europe/Moscow'); -SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Europe/Paris'); -SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Europe/London'); -SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Asia/Tokyo'); -SELECT toMonday(toDateTime('2014-12-29 00:00:00'), 'Pacific/Pitcairn'); +SELECT toMonday(toDateTime(1419800400), 'Europe/Moscow'); +SELECT toMonday(toDateTime(1419800400), 'Europe/Paris'); +SELECT toMonday(toDateTime(1419800400), 'Europe/London'); +SELECT toMonday(toDateTime(1419800400), 'Asia/Tokyo'); +SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn'); /* toStartOfMonth */ -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Moscow'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Paris'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/London'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Asia/Tokyo'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Pacific/Pitcairn'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn'); /* toStartOfQuarter */ -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Moscow'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/Paris'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Europe/London'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Asia/Tokyo'); -SELECT toStartOfMonth(toDateTime('2014-12-29 00:00:00'), 'Pacific/Pitcairn'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo'); +SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn'); /* toStartOfYear */ -SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Europe/Moscow'); -SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Europe/Paris'); -SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Europe/London'); -SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Asia/Tokyo'); -SELECT toStartOfQuarter(toDateTime('2014-09-30 23:50:00'), 'Pacific/Pitcairn'); +SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Paris'); +SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/London'); +SELECT toStartOfQuarter(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toStartOfQuarter(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toTime */ -SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Europe/Moscow'), toTime(toDateTime('2015-04-06 12:00:00'), 'Europe/Moscow'); -SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Europe/Paris'), toTime(toDateTime('2015-04-06 12:00:00'), 'Europe/Paris'); -SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Europe/London'), toTime(toDateTime('2015-04-06 12:00:00'), 'Europe/London'); -SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Asia/Tokyo'), toTime(toDateTime('2015-04-06 12:00:00'), 'Asia/Tokyo'); -SELECT toTime(toDateTime('2015-01-01 12:00:00'), 'Pacific/Pitcairn'), toTime(toDateTime('2015-04-06 12:00:00'), 'Pacific/Pitcairn'); +SELECT toTime(toDateTime(1420102800), 'Europe/Moscow'), toTime(toDateTime(1428310800), 'Europe/Moscow'); +SELECT toTime(toDateTime(1420102800), 'Europe/Paris'), toTime(toDateTime(1428310800), 'Europe/Paris'); +SELECT toTime(toDateTime(1420102800), 'Europe/London'), toTime(toDateTime(1428310800), 'Europe/London'); +SELECT toTime(toDateTime(1420102800), 'Asia/Tokyo'), toTime(toDateTime(1428310800), 'Asia/Tokyo'); +SELECT toTime(toDateTime(1420102800), 'Pacific/Pitcairn'), toTime(toDateTime(1428310800), 'Pacific/Pitcairn'); + +DROP TABLE IF EXISTS foo; +CREATE TABLE foo(x Int32, y String) ENGINE=Memory; +INSERT INTO foo(x, y) VALUES(1420102800, 'Europe/Moscow'); +INSERT INTO foo(x, y) VALUES(1412106600, 'Europe/Paris'); +INSERT INTO foo(x, y) VALUES(1419800400, 'Europe/London'); +INSERT INTO foo(x, y) VALUES(1436956200, 'Asia/Tokyo'); +INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn'); + +SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo; +SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo; +SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo; /* toString */ -SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Europe/Moscow'); -SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Europe/Paris'); -SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Europe/London'); -SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Asia/Tokyo'); -SELECT toString(toDateTime('2015-07-15 13:30:00'), 'Pacific/Pitcairn'); +SELECT toString(toDateTime(1436956200), 'Europe/Moscow'); +SELECT toString(toDateTime(1436956200), 'Europe/Paris'); +SELECT toString(toDateTime(1436956200), 'Europe/London'); +SELECT toString(toDateTime(1436956200), 'Asia/Tokyo'); +SELECT toString(toDateTime(1436956200), 'Pacific/Pitcairn'); /* toUnixTimestamp */ -SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Europe/Moscow'); -SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Europe/Paris'); -SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Europe/London'); -SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Asia/Tokyo'); -SELECT toUnixTimestamp('2015-03-15 13:30:00', 'Pacific/Pitcairn'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/Moscow'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/Paris'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/London'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Asia/Tokyo'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Pacific/Pitcairn'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Moscow'), 'Europe/Moscow'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Paris'), 'Europe/Paris'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/London'), 'Europe/London'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Asia/Tokyo'), 'Asia/Tokyo'); +SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); From 426e27d98ec83564de7810864dd10eb585dfbc51 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 2 Jul 2015 17:14:07 +0300 Subject: [PATCH 16/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- dbms/include/DB/Functions/FunctionsDateTime.h | 25 ------------------- .../0_stateless/00189_time_zones.reference | 15 +++++++++++ 2 files changed, 15 insertions(+), 25 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 6a353a6108b..236f1f68775 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -261,15 +261,6 @@ struct Transformer } } - static void vector_fixed(const typename ColumnVector::Container_t & vec_from, const ColumnString::Chars_t & data, - size_t n, typename ColumnVector::Container_t & vec_to) - { - auto & local_date_lut = DateLUT::instance(); - auto & remote_date_lut = DateLUT::instance(std::string(reinterpret_cast(&data[0]), n)); - for (size_t i = 0; i < vec_from.size(); ++i) - vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut); - } - static void vector_constant(const typename ColumnVector::Container_t & vec_from, const std::string & data, typename ColumnVector::Container_t & vec_to) { @@ -295,13 +286,6 @@ struct Transformer } } - static void constant_fixed(const FromType & from, const ColumnString::Chars_t & data, size_t n, ToType & to) - { - auto & local_date_lut = DateLUT::instance(); - auto & remote_date_lut = DateLUT::instance(std::string(reinterpret_cast(&data[0]), n)); - to = Transform::execute(from, remote_date_lut, local_date_lut); - } - static void constant_constant(const FromType & from, const std::string & data, ToType & to) { auto & local_date_lut = DateLUT::instance(); @@ -352,7 +336,6 @@ struct DateTimeTransformImpl { const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; const ColumnString * time_zones = typeid_cast(&*time_zone_col); - const ColumnFixedString * fixed_time_zone = typeid_cast(&*time_zone_col); const ColumnConstString * const_time_zone = typeid_cast(&*time_zone_col); if (sources) @@ -366,8 +349,6 @@ struct DateTimeTransformImpl if (time_zones) Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); - else if (fixed_time_zone) - Op::vector_fixed(vec_from, fixed_time_zone->getChars(), fixed_time_zone->getN(), vec_to); else if (const_time_zone) Op::vector_constant(vec_from, const_time_zone->getData(), vec_to); else @@ -387,12 +368,6 @@ struct DateTimeTransformImpl Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); } - else if (fixed_time_zone) - { - ToType res; - Op::constant_fixed(const_source->getData(), fixed_time_zone->getChars(), fixed_time_zone->getN(), res); - block.getByPosition(result).column = new ColumnConst(const_source->size(), res); - } else if (const_time_zone) { ToType res; diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index c9bccf34454..02ad1862f2f 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -23,6 +23,21 @@ 1970-01-02 09:00:00 1970-01-02 10:00:00 1970-01-02 18:00:00 1970-01-02 18:00:00 1970-01-02 01:30:00 1970-01-02 01:30:00 +2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00 +2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00 +2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00 +2014-12-22 2014-12-01 2014-10-01 1970-01-02 21:00:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 +2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00 +2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00 +2014-09-29 2014-09-01 2014-10-01 1970-01-02 11:20:00 +2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 10:30:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 2015-07-15 13:30:00 2015-07-15 12:30:00 2015-07-15 11:30:00 From 91ca58d0d3b0ba8186fac52e2020f5964641c56f Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 2 Jul 2015 17:22:15 +0300 Subject: [PATCH 17/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- .../queries/0_stateless/00189_time_zones.reference | 14 +++++++------- .../tests/queries/0_stateless/00189_time_zones.sql | 6 +++--- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index 02ad1862f2f..78c2c72ac55 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -23,21 +23,21 @@ 1970-01-02 09:00:00 1970-01-02 10:00:00 1970-01-02 18:00:00 1970-01-02 18:00:00 1970-01-02 01:30:00 1970-01-02 01:30:00 -2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00 2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00 -2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00 2014-12-22 2014-12-01 2014-10-01 1970-01-02 21:00:00 +2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 -2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00 -2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00 +2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00 2014-09-29 2014-09-01 2014-10-01 1970-01-02 11:20:00 -2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00 +2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00 +2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 -2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00 +2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 10:30:00 -2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00 2015-07-15 13:30:00 2015-07-15 12:30:00 2015-07-15 11:30:00 diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.sql b/dbms/tests/queries/0_stateless/00189_time_zones.sql index 90e2d4e2027..4dc7cec6324 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.sql +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -54,9 +54,9 @@ INSERT INTO foo(x, y) VALUES(1419800400, 'Europe/London'); INSERT INTO foo(x, y) VALUES(1436956200, 'Asia/Tokyo'); INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn'); -SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo; -SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo; -SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo; +SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY x ASC; +SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY x ASC; +SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY x ASC; /* toString */ From 7968c06699dfc8b25912f15bf13c8b2de7934b01 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 2 Jul 2015 17:26:15 +0300 Subject: [PATCH 18/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- dbms/include/DB/Functions/FunctionsDateTime.h | 17 ++--------------- 1 file changed, 2 insertions(+), 15 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 236f1f68775..eb4076f94b2 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -232,19 +232,6 @@ struct ToRelativeSecondNumImpl template struct Transformer { - static void vector(const typename ColumnVector::Container_t & vec_from, typename ColumnVector::Container_t & vec_to) - { - auto & local_date_lut = DateLUT::instance(); - for (size_t i = 0; i < vec_from.size(); ++i) - vec_to[i] = Transform::execute(vec_from[i], local_date_lut, local_date_lut); - } - - static void constant(const FromType & from, ToType & to) - { - auto & local_date_lut = DateLUT::instance(); - to = Transform::execute(from, local_date_lut, local_date_lut); - } - static void vector_vector(const typename ColumnVector::Container_t & vec_from, const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, typename ColumnVector::Container_t & vec_to) { @@ -317,12 +304,12 @@ struct DateTimeTransformImpl size_t size = vec_from.size(); vec_to.resize(size); - Op::vector(vec_from, vec_to); + Op::vector_constant(vec_from, "", vec_to); } else if (const_source) { ToType res; - Op::constant(const_source->getData(), res); + Op::constant_constant(const_source->getData(), "", res); block.getByPosition(result).column = new ColumnConst(const_source->size(), res); } else From 5a3d5970c53810f74e3d47a7ddc94e8f1dce6854 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Fri, 3 Jul 2015 03:09:57 +0300 Subject: [PATCH 19/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- .../DB/Functions/FunctionsConversion.h | 429 ++++++++++++++---- .../0_stateless/00189_time_zones.reference | 34 +- .../queries/0_stateless/00189_time_zones.sql | 10 +- 3 files changed, 387 insertions(+), 86 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index bcf8063bcb1..e8c466e2391 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -256,6 +256,111 @@ time_t convert_time(time_t remote_time, DateLUTImpl & remote_date_lut, DateLUTIm } } +struct DateTimeConverter +{ + static void vector_vector(const typename ColumnVector::Container_t & vec_from, + const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, + ColumnString & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + + ColumnString::Chars_t & data_to = vec_to.getChars(); + ColumnString::Offsets_t & offsets_to = vec_to.getOffsets(); + size_t size = vec_from.size(); + data_to.resize(size * 2); + offsets_to.resize(size); + + WriteBufferFromVector write_buffer(data_to); + + ColumnString::Offset_t prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + auto & remote_date_lut = DateLUT::instance(time_zone); + + auto ti = convert_time(vec_from[i], remote_date_lut, local_date_lut); + formatImpl(ti, write_buffer); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + + prev_offset = cur_offset; + } + data_to.resize(write_buffer.count()); + } + + static void vector_constant(const typename ColumnVector::Container_t & vec_from, + const std::string & data, + ColumnString & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + auto & remote_date_lut = DateLUT::instance(data); + + ColumnString::Chars_t & data_to = vec_to.getChars(); + ColumnString::Offsets_t & offsets_to = vec_to.getOffsets(); + size_t size = vec_from.size(); + data_to.resize(size * 2); + offsets_to.resize(size); + + WriteBufferFromVector write_buffer(data_to); + + for (size_t i = 0; i < size; ++i) + { + auto ti = convert_time(vec_from[i], remote_date_lut, local_date_lut); + formatImpl(ti, write_buffer); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + data_to.resize(write_buffer.count()); + } + + static void constant_vector(DataTypeDateTime::FieldType from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, + ColumnString & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + + ColumnString::Chars_t & data_to = vec_to.getChars(); + ColumnString::Offsets_t & offsets_to = vec_to.getOffsets(); + size_t size = offsets.size(); + data_to.resize(size * 2); + offsets_to.resize(size); + + WriteBufferFromVector write_buffer(data_to); + + ColumnString::Offset_t prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + auto & remote_date_lut = DateLUT::instance(time_zone); + + auto ti = convert_time(from, remote_date_lut, local_date_lut); + formatImpl(ti, write_buffer); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + + prev_offset = cur_offset; + } + data_to.resize(write_buffer.count()); + } + + static void constant_constant(DataTypeDateTime::FieldType from, const std::string & data, std::string & to) + { + auto & local_date_lut = DateLUT::instance(); + auto & remote_date_lut = DateLUT::instance(data); + + std::vector buf; + WriteBufferFromVector > write_buffer(buf); + auto ti = convert_time(from, remote_date_lut, local_date_lut); + formatImpl(ti, write_buffer); + to = std::string(&buf[0], write_buffer.count()); + } +}; + template struct ConvertImpl { @@ -263,57 +368,87 @@ struct ConvertImpl static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - std::string time_zone; + const ColumnPtr source_col = block.getByPosition(arguments[0]).column; + const ColumnVector * sources = typeid_cast *>(&*source_col); + const ColumnConst * const_source = typeid_cast *>(&*source_col); - if (arguments.size() == 2) + if (arguments.size() == 1) { - const ColumnPtr column = block.getByPosition(arguments[1]).column; - if (const ColumnConstString * col = typeid_cast(&*column)) - time_zone = col->getData(); - } - - auto & remote_date_lut = DateLUT::instance(time_zone); - auto & local_date_lut = DateLUT::instance(); - - if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) - { - ColumnString * col_to = new ColumnString; - block.getByPosition(result).column = col_to; - - const typename ColumnVector::Container_t & vec_from = col_from->getData(); - ColumnString::Chars_t & data_to = col_to->getChars(); - ColumnString::Offsets_t & offsets_to = col_to->getOffsets(); - size_t size = vec_from.size(); - data_to.resize(size * 2); - offsets_to.resize(size); - - WriteBufferFromVector write_buffer(data_to); - - for (size_t i = 0; i < size; ++i) + if (sources) { - auto ti = convert_time(vec_from[i], remote_date_lut, local_date_lut); - formatImpl(ti, write_buffer); - writeChar(0, write_buffer); - offsets_to[i] = write_buffer.count(); + ColumnString * col_to = new ColumnString; + block.getByPosition(result).column = col_to; + + auto & vec_from = sources->getData(); + auto & vec_to = *col_to; + + DateTimeConverter::vector_constant(vec_from, "", vec_to); + } + else if (const_source) + { + std::string res; + DateTimeConverter::constant_constant(const_source->getData(), "", res); + block.getByPosition(result).column = new ColumnConstString(const_source->size(), res); + } + else + { + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); } - data_to.resize(write_buffer.count()); } - else if (const ColumnConst * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + else if (arguments.size() == 2) { - std::vector buf; - WriteBufferFromVector > write_buffer(buf); - auto ti = convert_time(col_from->getData(), remote_date_lut, local_date_lut); - formatImpl(ti, write_buffer); - block.getByPosition(result).column = new ColumnConstString(col_from->size(), std::string(&buf[0], write_buffer.count())); + const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; + const ColumnString * time_zones = typeid_cast(&*time_zone_col); + const ColumnConstString * const_time_zone = typeid_cast(&*time_zone_col); + + if (sources) + { + ColumnString * col_to = new ColumnString; + block.getByPosition(result).column = col_to; + + auto & vec_from = sources->getData(); + auto & vec_to = *col_to; + + if (time_zones) + DateTimeConverter::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); + else if (const_time_zone) + DateTimeConverter::vector_constant(vec_from, const_time_zone->getData(), vec_to); + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else if (const_source) + { + if (time_zones) + { + ColumnString * col_to = new ColumnString; + block.getByPosition(result).column = col_to; + auto & vec_to = *col_to; + + DateTimeConverter::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); + } + else if (const_time_zone) + { + std::string res; + DateTimeConverter::constant_constant(const_source->getData(), const_time_zone->getData(), res); + block.getByPosition(result).column = new ColumnConstString(const_source->size(), res); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + Name::name, - ErrorCodes::ILLEGAL_COLUMN); } }; - /** Преобразование строк в числа, даты, даты-с-временем: через парсинг. */ template void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb) { readText(x,rb); } @@ -375,6 +510,98 @@ struct ConvertImpl } }; +struct TimeConverter +{ + static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, + typename ColumnVector::Container_t & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + ReadBuffer read_buffer(const_cast(reinterpret_cast(&vec_from[0])), vec_from.size(), 0); + + ColumnString::Offset_t prev_offset = 0; + + char zero = 0; + for (size_t i = 0; i < vec_to.size(); ++i) + { + DataTypeDateTime::FieldType x = 0; + parseImpl(x, read_buffer); + + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + auto & remote_date_lut = DateLUT::instance(time_zone); + + auto ti = convert_time(x, local_date_lut, remote_date_lut); + + vec_to[i] = ti; + readChar(zero, read_buffer); + if (zero != 0) + throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER); + + prev_offset = cur_offset; + } + } + + static void vector_constant(const ColumnString::Chars_t & vec_from, const std::string & data, + typename ColumnVector::Container_t & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + auto & remote_date_lut = DateLUT::instance(data); + ReadBuffer read_buffer(const_cast(reinterpret_cast(&vec_from[0])), vec_from.size(), 0); + + char zero = 0; + for (size_t i = 0; i < vec_to.size(); ++i) + { + DataTypeDateTime::FieldType x = 0; + parseImpl(x, read_buffer); + + auto ti = convert_time(x, local_date_lut, remote_date_lut); + + vec_to[i] = ti; + readChar(zero, read_buffer); + if (zero != 0) + throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER); + } + } + + static void constant_vector(const std::string & from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, + typename ColumnVector::Container_t & vec_to) + { + auto & local_date_lut = DateLUT::instance(); + + ReadBufferFromString read_buffer(from); + DataTypeDateTime::FieldType x = 0; + parseImpl(x, read_buffer); + + 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(&data[prev_offset]), cur_offset - prev_offset - 1); + auto & remote_date_lut = DateLUT::instance(time_zone); + + auto ti = convert_time(x, local_date_lut, remote_date_lut); + + vec_to[i] = ti; + prev_offset = cur_offset; + } + } + + static void constant_constant(const std::string & from, const std::string & data, DataTypeInt32::FieldType & to) + { + auto & local_date_lut = DateLUT::instance(); + auto & remote_date_lut = DateLUT::instance(data); + + ReadBufferFromString read_buffer(from); + DataTypeDateTime::FieldType x = 0; + parseImpl(x, read_buffer); + + to = convert_time(x, local_date_lut, remote_date_lut); + } +}; + struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; }; template<> @@ -384,58 +611,94 @@ struct ConvertImpl static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - std::string time_zone; + const ColumnPtr source_col = block.getByPosition(arguments[0]).column; + const ColumnString * sources = typeid_cast(&*source_col); + const ColumnConstString * const_source = typeid_cast(&*source_col); - if (arguments.size() == 2) + if (arguments.size() == 1) { - const ColumnPtr column = block.getByPosition(arguments[1]).column; - if (const ColumnConstString * col = typeid_cast(&*column)) - time_zone = col->getData(); - } - - auto & remote_date_lut = DateLUT::instance(time_zone); - auto & local_date_lut = DateLUT::instance(); - - if (const ColumnString * col_from = typeid_cast(&*block.getByPosition(arguments[0]).column)) - { - ColumnVector * col_to = new ColumnVector; - block.getByPosition(result).column = col_to; - - const ColumnString::Chars_t & data_from = col_from->getChars(); - typename ColumnVector::Container_t & vec_to = col_to->getData(); - size_t size = col_from->size(); - vec_to.resize(size); - - ReadBuffer read_buffer(const_cast(reinterpret_cast(&data_from[0])), data_from.size(), 0); - - char zero = 0; - for (size_t i = 0; i < size; ++i) + if (sources) { - DataTypeDateTime::FieldType x = 0; - parseImpl(x, read_buffer); - auto ti = convert_time(x, local_date_lut, remote_date_lut); - vec_to[i] = ti; - readChar(zero, read_buffer); - if (zero != 0) - throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER); + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + auto & vec_from = sources->getChars(); + auto & vec_to = col_to->getData(); + size_t size = sources->size(); + vec_to.resize(size); + + TimeConverter::vector_constant(vec_from, "", vec_to); + } + else if (const_source) + { + DataTypeInt32::FieldType res; + TimeConverter::constant_constant(const_source->getData(), "", res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + } + else + { + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + NameToUnixTimestamp::name, + ErrorCodes::ILLEGAL_COLUMN); } } - else if (const ColumnConstString * col_from = typeid_cast(&*block.getByPosition(arguments[0]).column)) + else if (arguments.size() == 2) { - const String & s = col_from->getData(); - ReadBufferFromString read_buffer(s); - DataTypeDateTime::FieldType x = 0; - parseImpl(x, read_buffer); - auto ti = convert_time(x, local_date_lut, remote_date_lut); - block.getByPosition(result).column = new ColumnConst(col_from->size(), ti); + const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; + const ColumnString * time_zones = typeid_cast(&*time_zone_col); + const ColumnConstString * const_time_zone = typeid_cast(&*time_zone_col); + + if (sources) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + auto & vec_from = sources->getChars(); + auto & vec_to = col_to->getData(); + size_t size = sources->size(); + vec_to.resize(size); + + if (time_zones) + TimeConverter::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); + else if (const_time_zone) + TimeConverter::vector_constant(vec_from, const_time_zone->getData(), vec_to); + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + NameToUnixTimestamp::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else if (const_source) + { + if (time_zones) + { + ColumnVector * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + auto & vec_to = col_to->getData(); + vec_to.resize(time_zones->getOffsets().size()); + + TimeConverter::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); + } + else if (const_time_zone) + { + DataTypeInt32::FieldType res; + TimeConverter::constant_constant(const_source->getData(), const_time_zone->getData(), res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of second argument of function " + NameToUnixTimestamp::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + NameToUnixTimestamp::name, + ErrorCodes::ILLEGAL_COLUMN); } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + NameToUnixTimestamp::name, - ErrorCodes::ILLEGAL_COLUMN); } }; + /** Если типы совпадают - просто скопируем ссылку на столбец. */ template diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index 78c2c72ac55..32c941f69ff 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -33,16 +33,31 @@ 2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 2015-07-13 2015-07-01 2015-07-01 1970-01-02 02:00:00 -2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 10:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 13:30:00 +2015-03-09 2015-03-01 2015-01-01 1970-01-02 11:30:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 -2015-03-09 2015-03-01 2015-01-01 1970-01-02 19:30:00 2015-07-15 13:30:00 2015-07-15 12:30:00 2015-07-15 11:30:00 2015-07-15 19:30:00 2015-07-15 02:30:00 +2015-07-15 19:30:00 +2014-12-28 21:00:00 +2015-01-01 12:00:00 +2014-09-30 21:50:00 +2015-03-15 02:30:00 +2015-07-15 19:30:00 +2015-07-15 11:30:00 +2015-07-15 13:30:00 +2015-07-15 12:30:00 +2015-07-15 02:30:00 +2014-09-30 20:50:00 +2014-12-28 21:00:00 +2015-01-01 09:00:00 +2015-03-15 10:30:00 +2015-07-15 11:30:00 1426415400 1426422600 1426426200 @@ -53,3 +68,18 @@ 1426415400 1426415400 1426415400 +1412113800 +1419811200 +1420102800 +1426455000 +1436934600 +1426393800 +1426426200 +1426415400 +1426422600 +1426455000 +1412113800 +1419807600 +1420110000 +1426422600 +1436959800 diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.sql b/dbms/tests/queries/0_stateless/00189_time_zones.sql index 4dc7cec6324..a2ea41b195e 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.sql +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -56,7 +56,7 @@ INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn'); SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY x ASC; SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY x ASC; -SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY x ASC; +SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; /* toString */ @@ -66,6 +66,10 @@ SELECT toString(toDateTime(1436956200), 'Europe/London'); SELECT toString(toDateTime(1436956200), 'Asia/Tokyo'); SELECT toString(toDateTime(1436956200), 'Pacific/Pitcairn'); +SELECT toString(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toString(toDateTime(1436956200), y) FROM foo ORDER BY y ASC; +SELECT toString(toDateTime(x), 'Europe/London') FROM foo ORDER BY x ASC; + /* toUnixTimestamp */ SELECT toUnixTimestamp(toString(toDateTime(1426415400)), 'Europe/Moscow'); @@ -79,3 +83,7 @@ SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Paris'), 'Europe SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/London'), 'Europe/London'); SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Asia/Tokyo'), 'Asia/Tokyo'); SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); + +SELECT toUnixTimestamp(toString(toDateTime(x)), y) FROM foo ORDER BY x ASC; +SELECT toUnixTimestamp(toString(toDateTime(1426415400)), y) FROM foo ORDER BY y ASC; +SELECT toUnixTimestamp(toString(toDateTime(x)), 'Europe/Paris') FROM foo ORDER BY x ASC; From 90d85c9a778b3fb46561a9d71e005839dec8092b Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Fri, 3 Jul 2015 14:55:51 +0300 Subject: [PATCH 20/24] dbms: Server: Support for multiple time zones: development [#METR-15618] --- .../DB/Functions/FunctionsConversion.h | 143 ++++++++++-------- dbms/include/DB/Functions/FunctionsDateTime.h | 26 ++-- .../0_stateless/00189_time_zones.reference | 8 +- .../queries/0_stateless/00189_time_zones.sql | 4 +- libs/libcommon/src/DateLUT.cpp | 2 +- libs/libcommon/src/DateLUTImpl.cpp | 11 +- 6 files changed, 110 insertions(+), 84 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index e8c466e2391..447a5489105 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -242,26 +242,34 @@ struct ConvertImpl } }; -time_t convert_time(time_t remote_time, DateLUTImpl & remote_date_lut, DateLUTImpl & local_date_lut) +namespace details { namespace { + +/** Пусть source_timestamp представляет дату и время в исходном часовом поясе соответствующем + * объекту from_date_lut. Эта функция возвращает timestamp представлящий те же дату и время + * в часовом поясе соответствующем объекту to_date_lut. + */ +time_t convertTimestamp(time_t source_timestamp, DateLUTImpl & from_date_lut, DateLUTImpl & to_date_lut) { - if (&remote_date_lut == &local_date_lut) - return remote_time; + if (&from_date_lut == &to_date_lut) + return source_timestamp; else { - const auto & values = remote_date_lut.getValues(remote_time); - return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month, - remote_date_lut.toHourInaccurate(remote_time), - remote_date_lut.toMinuteInaccurate(remote_time), - remote_date_lut.toSecondInaccurate(remote_time)); + const auto & values = from_date_lut.getValues(source_timestamp); + return to_date_lut.makeDateTime(values.year, values.month, values.day_of_month, + from_date_lut.toHourInaccurate(source_timestamp), + from_date_lut.toMinuteInaccurate(source_timestamp), + from_date_lut.toSecondInaccurate(source_timestamp)); } } -struct DateTimeConverter +/** Функции для преобразования даты + времени в строку. + */ +struct DateTimeToStringConverter { - static void vector_vector(const typename ColumnVector::Container_t & vec_from, - const ColumnString::Chars_t & data, - const ColumnString::Offsets_t & offsets, - ColumnString & vec_to) + using FromFieldType = typename DataTypeDateTime::FieldType; + + static void vector_vector(const PODArray & vec_from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, ColumnString & vec_to) { auto & local_date_lut = DateLUT::instance(); @@ -281,7 +289,7 @@ struct DateTimeConverter const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); auto & remote_date_lut = DateLUT::instance(time_zone); - auto ti = convert_time(vec_from[i], remote_date_lut, local_date_lut); + auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut); formatImpl(ti, write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); @@ -291,8 +299,7 @@ struct DateTimeConverter data_to.resize(write_buffer.count()); } - static void vector_constant(const typename ColumnVector::Container_t & vec_from, - const std::string & data, + static void vector_constant(const PODArray & vec_from, const std::string & data, ColumnString & vec_to) { auto & local_date_lut = DateLUT::instance(); @@ -308,7 +315,7 @@ struct DateTimeConverter for (size_t i = 0; i < size; ++i) { - auto ti = convert_time(vec_from[i], remote_date_lut, local_date_lut); + auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut); formatImpl(ti, write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); @@ -316,7 +323,7 @@ struct DateTimeConverter data_to.resize(write_buffer.count()); } - static void constant_vector(DataTypeDateTime::FieldType from, const ColumnString::Chars_t & data, + static void constant_vector(FromFieldType from, const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, ColumnString & vec_to) { @@ -338,7 +345,7 @@ struct DateTimeConverter const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); auto & remote_date_lut = DateLUT::instance(time_zone); - auto ti = convert_time(from, remote_date_lut, local_date_lut); + auto ti = convertTimestamp(from, remote_date_lut, local_date_lut); formatImpl(ti, write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); @@ -348,29 +355,32 @@ struct DateTimeConverter data_to.resize(write_buffer.count()); } - static void constant_constant(DataTypeDateTime::FieldType from, const std::string & data, std::string & to) + 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); std::vector buf; WriteBufferFromVector > write_buffer(buf); - auto ti = convert_time(from, remote_date_lut, local_date_lut); + auto ti = convertTimestamp(from, remote_date_lut, local_date_lut); formatImpl(ti, write_buffer); to = std::string(&buf[0], write_buffer.count()); } }; +}} + template struct ConvertImpl { - typedef typename DataTypeDateTime::FieldType FromFieldType; + using Op = details::DateTimeToStringConverter; + using FromFieldType = Op::FromFieldType; static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnPtr source_col = block.getByPosition(arguments[0]).column; - const ColumnVector * sources = typeid_cast *>(&*source_col); - const ColumnConst * const_source = typeid_cast *>(&*source_col); + const auto * sources = typeid_cast *>(&*source_col); + const auto * const_source = typeid_cast *>(&*source_col); if (arguments.size() == 1) { @@ -382,12 +392,12 @@ struct ConvertImpl auto & vec_from = sources->getData(); auto & vec_to = *col_to; - DateTimeConverter::vector_constant(vec_from, "", vec_to); + Op::vector_constant(vec_from, "", vec_to); } else if (const_source) { std::string res; - DateTimeConverter::constant_constant(const_source->getData(), "", res); + Op::constant_constant(const_source->getData(), "", res); block.getByPosition(result).column = new ColumnConstString(const_source->size(), res); } else @@ -400,8 +410,8 @@ struct ConvertImpl else if (arguments.size() == 2) { const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; - const ColumnString * time_zones = typeid_cast(&*time_zone_col); - const ColumnConstString * const_time_zone = typeid_cast(&*time_zone_col); + const auto * time_zones = typeid_cast(&*time_zone_col); + const auto * const_time_zone = typeid_cast(&*time_zone_col); if (sources) { @@ -412,9 +422,9 @@ struct ConvertImpl auto & vec_to = *col_to; if (time_zones) - DateTimeConverter::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); + Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); else if (const_time_zone) - DateTimeConverter::vector_constant(vec_from, const_time_zone->getData(), vec_to); + Op::vector_constant(vec_from, const_time_zone->getData(), vec_to); else throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + " of second argument of function " + Name::name, @@ -428,12 +438,12 @@ struct ConvertImpl block.getByPosition(result).column = col_to; auto & vec_to = *col_to; - DateTimeConverter::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); + Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); } else if (const_time_zone) { std::string res; - DateTimeConverter::constant_constant(const_source->getData(), const_time_zone->getData(), res); + Op::constant_constant(const_source->getData(), const_time_zone->getData(), res); block.getByPosition(result).column = new ColumnConstString(const_source->size(), res); } else @@ -446,6 +456,8 @@ struct ConvertImpl + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); } + else + throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR); } }; @@ -510,11 +522,16 @@ struct ConvertImpl } }; -struct TimeConverter +namespace details { namespace { + +/** Функции для преобразования строк в timestamp. + */ +struct StringToTimestampConverter { + using ToFieldType = typename DataTypeInt32::FieldType; + static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data, - const ColumnString::Offsets_t & offsets, - typename ColumnVector::Container_t & vec_to) + const ColumnString::Offsets_t & offsets, PODArray & vec_to) { auto & local_date_lut = DateLUT::instance(); ReadBuffer read_buffer(const_cast(reinterpret_cast(&vec_from[0])), vec_from.size(), 0); @@ -531,7 +548,7 @@ struct TimeConverter const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); auto & remote_date_lut = DateLUT::instance(time_zone); - auto ti = convert_time(x, local_date_lut, remote_date_lut); + auto ti = convertTimestamp(x, local_date_lut, remote_date_lut); vec_to[i] = ti; readChar(zero, read_buffer); @@ -543,7 +560,7 @@ struct TimeConverter } static void vector_constant(const ColumnString::Chars_t & vec_from, const std::string & data, - typename ColumnVector::Container_t & vec_to) + PODArray & vec_to) { auto & local_date_lut = DateLUT::instance(); auto & remote_date_lut = DateLUT::instance(data); @@ -555,7 +572,7 @@ struct TimeConverter DataTypeDateTime::FieldType x = 0; parseImpl(x, read_buffer); - auto ti = convert_time(x, local_date_lut, remote_date_lut); + auto ti = convertTimestamp(x, local_date_lut, remote_date_lut); vec_to[i] = ti; readChar(zero, read_buffer); @@ -565,8 +582,7 @@ struct TimeConverter } static void constant_vector(const std::string & from, const ColumnString::Chars_t & data, - const ColumnString::Offsets_t & offsets, - typename ColumnVector::Container_t & vec_to) + const ColumnString::Offsets_t & offsets, PODArray & vec_to) { auto & local_date_lut = DateLUT::instance(); @@ -582,14 +598,14 @@ struct TimeConverter const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); auto & remote_date_lut = DateLUT::instance(time_zone); - auto ti = convert_time(x, local_date_lut, remote_date_lut); + auto ti = convertTimestamp(x, local_date_lut, remote_date_lut); vec_to[i] = ti; prev_offset = cur_offset; } } - static void constant_constant(const std::string & from, const std::string & data, DataTypeInt32::FieldType & to) + 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); @@ -598,28 +614,31 @@ struct TimeConverter DataTypeDateTime::FieldType x = 0; parseImpl(x, read_buffer); - to = convert_time(x, local_date_lut, remote_date_lut); + to = convertTimestamp(x, local_date_lut, remote_date_lut); } }; +}} + struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; }; template<> struct ConvertImpl { - typedef typename DataTypeInt32::FieldType ToFieldType; + using Op = details::StringToTimestampConverter; + using ToFieldType = Op::ToFieldType; static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnPtr source_col = block.getByPosition(arguments[0]).column; - const ColumnString * sources = typeid_cast(&*source_col); - const ColumnConstString * const_source = typeid_cast(&*source_col); + const auto * sources = typeid_cast(&*source_col); + const auto * const_source = typeid_cast(&*source_col); if (arguments.size() == 1) { if (sources) { - ColumnVector * col_to = new ColumnVector; + auto * col_to = new ColumnVector; block.getByPosition(result).column = col_to; auto & vec_from = sources->getChars(); @@ -627,13 +646,13 @@ struct ConvertImpl size_t size = sources->size(); vec_to.resize(size); - TimeConverter::vector_constant(vec_from, "", vec_to); + Op::vector_constant(vec_from, "", vec_to); } else if (const_source) { - DataTypeInt32::FieldType res; - TimeConverter::constant_constant(const_source->getData(), "", res); - block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + ToFieldType res; + Op::constant_constant(const_source->getData(), "", res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); } else { @@ -645,12 +664,12 @@ struct ConvertImpl else if (arguments.size() == 2) { const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; - const ColumnString * time_zones = typeid_cast(&*time_zone_col); - const ColumnConstString * const_time_zone = typeid_cast(&*time_zone_col); + const auto * time_zones = typeid_cast(&*time_zone_col); + const auto * const_time_zone = typeid_cast(&*time_zone_col); if (sources) { - ColumnVector * col_to = new ColumnVector; + auto * col_to = new ColumnVector; block.getByPosition(result).column = col_to; auto & vec_from = sources->getChars(); @@ -659,9 +678,9 @@ struct ConvertImpl vec_to.resize(size); if (time_zones) - TimeConverter::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); + Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to); else if (const_time_zone) - TimeConverter::vector_constant(vec_from, const_time_zone->getData(), vec_to); + Op::vector_constant(vec_from, const_time_zone->getData(), vec_to); else throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + " of second argument of function " + NameToUnixTimestamp::name, @@ -671,19 +690,19 @@ struct ConvertImpl { if (time_zones) { - ColumnVector * col_to = new ColumnVector; + auto * col_to = new ColumnVector; block.getByPosition(result).column = col_to; auto & vec_to = col_to->getData(); vec_to.resize(time_zones->getOffsets().size()); - TimeConverter::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); + Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to); } else if (const_time_zone) { - DataTypeInt32::FieldType res; - TimeConverter::constant_constant(const_source->getData(), const_time_zone->getData(), res); - block.getByPosition(result).column = new ColumnConst(const_source->size(), res); + ToFieldType res; + Op::constant_constant(const_source->getData(), const_time_zone->getData(), res); + block.getByPosition(result).column = new ColumnConst(const_source->size(), res); } else throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() @@ -695,6 +714,8 @@ struct ConvertImpl + " of first argument of function " + NameToUnixTimestamp::name, ErrorCodes::ILLEGAL_COLUMN); } + else + throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR); } }; diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index eb4076f94b2..8e104c6729d 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -232,8 +232,8 @@ struct ToRelativeSecondNumImpl template struct Transformer { - static void vector_vector(const typename ColumnVector::Container_t & vec_from, const ColumnString::Chars_t & data, - const ColumnString::Offsets_t & offsets, typename ColumnVector::Container_t & vec_to) + static void vector_vector(const PODArray & vec_from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, PODArray & vec_to) { auto & local_date_lut = DateLUT::instance(); ColumnString::Offset_t prev_offset = 0; @@ -248,8 +248,8 @@ struct Transformer } } - static void vector_constant(const typename ColumnVector::Container_t & vec_from, const std::string & data, - typename ColumnVector::Container_t & vec_to) + static void vector_constant(const PODArray & vec_from, const std::string & data, + PODArray & vec_to) { auto & local_date_lut = DateLUT::instance(); auto & remote_date_lut = DateLUT::instance(data); @@ -258,7 +258,7 @@ struct Transformer } static void constant_vector(const FromType & from, const ColumnString::Chars_t & data, - const ColumnString::Offsets_t & offsets, typename ColumnVector::Container_t & vec_to) + const ColumnString::Offsets_t & offsets, PODArray & vec_to) { auto & local_date_lut = DateLUT::instance(); ColumnString::Offset_t prev_offset = 0; @@ -289,14 +289,14 @@ struct DateTimeTransformImpl using Op = Transformer; const ColumnPtr source_col = block.getByPosition(arguments[0]).column; - const ColumnVector * sources = typeid_cast *>(&*source_col); - const ColumnConst * const_source = typeid_cast *>(&*source_col); + const auto * sources = typeid_cast *>(&*source_col); + const auto * const_source = typeid_cast *>(&*source_col); if (arguments.size() == 1) { if (sources) { - ColumnVector * col_to = new ColumnVector; + auto * col_to = new ColumnVector; block.getByPosition(result).column = col_to; auto & vec_from = sources->getData(); @@ -322,12 +322,12 @@ struct DateTimeTransformImpl else if (arguments.size() == 2) { const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; - const ColumnString * time_zones = typeid_cast(&*time_zone_col); - const ColumnConstString * const_time_zone = typeid_cast(&*time_zone_col); + const auto * time_zones = typeid_cast(&*time_zone_col); + const auto * const_time_zone = typeid_cast(&*time_zone_col); if (sources) { - ColumnVector * col_to = new ColumnVector; + auto * col_to = new ColumnVector; block.getByPosition(result).column = col_to; auto & vec_from = sources->getData(); @@ -347,7 +347,7 @@ struct DateTimeTransformImpl { if (time_zones) { - ColumnVector * col_to = new ColumnVector; + auto * col_to = new ColumnVector; block.getByPosition(result).column = col_to; auto & vec_to = col_to->getData(); @@ -371,6 +371,8 @@ struct DateTimeTransformImpl + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); } + else + throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR); } }; diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index 32c941f69ff..f1d2ea9c2a7 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -23,11 +23,11 @@ 1970-01-02 09:00:00 1970-01-02 10:00:00 1970-01-02 18:00:00 1970-01-02 18:00:00 1970-01-02 01:30:00 1970-01-02 01:30:00 -2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00 +2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00 2014-12-22 2014-12-01 2014-10-01 1970-01-02 21:00:00 2014-12-29 2015-01-01 2015-01-01 1970-01-02 12:00:00 +2014-09-29 2014-09-01 2014-07-01 1970-01-02 21:50:00 2015-03-09 2015-03-01 2015-01-01 1970-01-02 02:00:00 -2015-07-13 2015-07-01 2015-07-01 1970-01-02 19:30:00 2014-09-29 2014-09-01 2014-10-01 1970-01-02 11:20:00 2014-12-22 2014-12-01 2014-10-01 1970-01-02 13:30:00 2014-12-29 2015-01-01 2015-01-01 1970-01-02 01:30:00 @@ -68,11 +68,11 @@ 1426415400 1426415400 1426415400 -1412113800 +1436934600 1419811200 1420102800 +1412113800 1426455000 -1436934600 1426393800 1426426200 1426415400 diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.sql b/dbms/tests/queries/0_stateless/00189_time_zones.sql index a2ea41b195e..fef2ef11c6b 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.sql +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -54,7 +54,7 @@ INSERT INTO foo(x, y) VALUES(1419800400, 'Europe/London'); INSERT INTO foo(x, y) VALUES(1436956200, 'Asia/Tokyo'); INSERT INTO foo(x, y) VALUES(1426415400, 'Pacific/Pitcairn'); -SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY x ASC; +SELECT toMonday(toDateTime(x), y), toStartOfMonth(toDateTime(x), y), toStartOfQuarter(toDateTime(x), y), toTime(toDateTime(x), y) FROM foo ORDER BY y ASC; SELECT toMonday(toDateTime(x), 'Europe/Paris'), toStartOfMonth(toDateTime(x), 'Europe/London'), toStartOfQuarter(toDateTime(x), 'Asia/Tokyo'), toTime(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY x ASC; SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; @@ -84,6 +84,6 @@ SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/London'), 'Europ SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Asia/Tokyo'), 'Asia/Tokyo'); SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); -SELECT toUnixTimestamp(toString(toDateTime(x)), y) FROM foo ORDER BY x ASC; +SELECT toUnixTimestamp(toString(toDateTime(x)), y) FROM foo ORDER BY y ASC; SELECT toUnixTimestamp(toString(toDateTime(1426415400)), y) FROM foo ORDER BY y ASC; SELECT toUnixTimestamp(toString(toDateTime(x)), 'Europe/Paris') FROM foo ORDER BY x ASC; diff --git a/libs/libcommon/src/DateLUT.cpp b/libs/libcommon/src/DateLUT.cpp index 4094223c583..f9596b47fdb 100644 --- a/libs/libcommon/src/DateLUT.cpp +++ b/libs/libcommon/src/DateLUT.cpp @@ -12,7 +12,7 @@ DateLUT::DateLUT() std::unique_ptr tz(TimeZone::createDefault()); if (tz == nullptr) - throw Poco::Exception("Failed to query the host time zone."); + throw Poco::Exception("Failed to determine the host time zone."); UnicodeString u_out; tz->getID(u_out); diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index 1045845dd7b..9b40cebea3b 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -72,12 +72,15 @@ GDateTimePtr toNextDay(const GTimeZonePtr & p_tz, const GDateTimePtr & p_dt) dt = g_date_time_add_days(dt, 1); if (dt == nullptr) - throw Poco::Exception("Null pointer"); + throw Poco::Exception("Failed to create GDateTime object."); + + GDateTimePtr p_next_dt = GDateTimePtr(dt); + GDateTime * next_dt = p_next_dt.get(); dt = g_date_time_new(p_tz.get(), - g_date_time_get_year(dt), - g_date_time_get_month(dt), - g_date_time_get_day_of_month(dt), + g_date_time_get_year(next_dt), + g_date_time_get_month(next_dt), + g_date_time_get_day_of_month(next_dt), 0, 0, 0); if (dt == nullptr) throw Poco::Exception("Failed to create GDateTime object."); From 1c8251ad109a0e42e0b26217a7f6b4ac7975c079 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Mon, 6 Jul 2015 16:53:13 +0300 Subject: [PATCH 21/24] Merge --- .../00046_stored_aggregates_simple.reference | 2 +- .../00047_stored_aggregates_complex.reference | 222 +++++++++--------- .../00048_a_stored_aggregates_merge.reference | 86 +++---- .../00048_b_stored_aggregates_merge.reference | 86 +++---- ...84_distributed_group_by_no_merge.reference | 4 +- ...arguments_of_aggregate_functions.reference | 2 +- 6 files changed, 201 insertions(+), 201 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00046_stored_aggregates_simple.reference b/dbms/tests/queries/0_stateless/00046_stored_aggregates_simple.reference index 83b33d238da..6e1fc58dbfa 100644 --- a/dbms/tests/queries/0_stateless/00046_stored_aggregates_simple.reference +++ b/dbms/tests/queries/0_stateless/00046_stored_aggregates_simple.reference @@ -1 +1 @@ -1000 +995 diff --git a/dbms/tests/queries/0_stateless/00047_stored_aggregates_complex.reference b/dbms/tests/queries/0_stateless/00047_stored_aggregates_complex.reference index e3609c9c4f1..0f3e332bd49 100644 --- a/dbms/tests/queries/0_stateless/00047_stored_aggregates_complex.reference +++ b/dbms/tests/queries/0_stateless/00047_stored_aggregates_complex.reference @@ -1,111 +1,111 @@ -2014-06-01 0 0 45 4.5 7 0 0 [4.5,8.1] ['0','1','2','3','4','5','6','7','8','9'] -2014-06-01 0 1 145 14.5 7 10 14 [14.5,18.1] ['10','11','12','13','14','15','16','17','18','19'] -2014-06-01 0 2 245 24.5 7 20 21 [24.5,28.1] ['20','21','22','23','24','25','26','27','28','29'] -2014-06-01 0 3 345 34.5 7 30 35 [34.5,38.1] ['30','31','32','33','34','35','36','37','38','39'] -2014-06-01 0 4 445 44.5 7 40 42 [44.5,48.1] ['40','41','42','43','44','45','46','47','48','49'] -2014-06-01 0 5 545 54.5 7 50 56 [54.5,58.099999999999994] ['50','51','52','53','54','55','56','57','58','59'] -2014-06-01 0 6 645 64.5 7 60 63 [64.5,68.1] ['60','61','62','63','64','65','66','67','68','69'] -2014-06-01 0 7 745 74.5 7 70 70 [74.5,78.10000000000001] ['70','71','72','73','74','75','76','77','78','79'] -2014-06-01 0 8 845 84.5 7 80 84 [84.5,88.1] ['80','81','82','83','84','85','86','87','88','89'] -2014-06-01 0 9 945 94.5 7 90 91 [94.5,98.1] ['90','91','92','93','94','95','96','97','98','99'] -2014-06-01 1 10 1045 104.5 7 100 105 [104.5,108.10000000000001] ['100','101','102','103','104','105','106','107','108','109'] -2014-06-01 1 11 1145 114.5 7 110 112 [114.5,118.10000000000001] ['110','111','112','113','114','115','116','117','118','119'] -2014-06-01 1 12 1245 124.5 7 120 126 [124.5,128.1] ['120','121','122','123','124','125','126','127','128','129'] -2014-06-01 1 13 1345 134.5 7 130 133 [134.5,138.1] ['130','131','132','133','134','135','136','137','138','139'] -2014-06-01 1 14 1445 144.5 7 140 140 [144.5,148.1] ['140','141','142','143','144','145','146','147','148','149'] -2014-06-01 1 15 1545 154.5 7 150 154 [154.5,158.1] ['150','151','152','153','154','155','156','157','158','159'] -2014-06-01 1 16 1645 164.5 7 160 161 [164.5,168.1] ['160','161','162','163','164','165','166','167','168','169'] -2014-06-01 1 17 1745 174.5 7 170 175 [174.5,178.10000000000002] ['170','171','172','173','174','175','176','177','178','179'] -2014-06-01 1 18 1845 184.5 7 180 182 [184.5,188.10000000000002] ['180','181','182','183','184','185','186','187','188','189'] -2014-06-01 1 19 1945 194.5 7 190 196 [194.5,198.1] ['190','191','192','193','194','195','196','197','198','199'] -2014-06-01 2 20 2045 204.5 7 200 203 [204.5,208.1] ['200','201','202','203','204','205','206','207','208','209'] -2014-06-01 2 21 2145 214.5 7 210 210 [214.5,218.1] ['210','211','212','213','214','215','216','217','218','219'] -2014-06-01 2 22 2245 224.5 7 220 224 [224.5,228.1] ['220','221','222','223','224','225','226','227','228','229'] -2014-06-01 2 23 2345 234.5 7 230 231 [234.5,238.1] ['230','231','232','233','234','235','236','237','238','239'] -2014-06-01 2 24 2445 244.5 7 240 245 [244.5,248.10000000000002] ['240','241','242','243','244','245','246','247','248','249'] -2014-06-01 2 25 2545 254.5 7 250 252 [254.5,258.1] ['250','251','252','253','254','255','256','257','258','259'] -2014-06-01 2 26 2645 264.5 7 260 266 [264.5,268.1] ['260','261','262','263','264','265','266','267','268','269'] -2014-06-01 2 27 2745 274.5 7 270 273 [274.5,278.1] ['270','271','272','273','274','275','276','277','278','279'] -2014-06-01 2 28 2845 284.5 7 280 280 [284.5,288.1] ['280','281','282','283','284','285','286','287','288','289'] -2014-06-01 2 29 2945 294.5 7 290 294 [294.5,298.1] ['290','291','292','293','294','295','296','297','298','299'] -2014-06-01 3 30 3045 304.5 7 300 301 [304.5,308.1] ['300','301','302','303','304','305','306','307','308','309'] -2014-06-01 3 31 3145 314.5 7 310 315 [314.5,318.09999999999997] ['310','311','312','313','314','315','316','317','318','319'] -2014-06-01 3 32 3245 324.5 7 320 322 [324.5,328.09999999999997] ['320','321','322','323','324','325','326','327','328','329'] -2014-06-01 3 33 3345 334.5 7 330 336 [334.5,338.09999999999997] ['330','331','332','333','334','335','336','337','338','339'] -2014-06-01 3 34 3445 344.5 7 340 343 [344.5,348.09999999999997] ['340','341','342','343','344','345','346','347','348','349'] -2014-06-01 3 35 3545 354.5 7 350 350 [354.5,358.09999999999997] ['350','351','352','353','354','355','356','357','358','359'] -2014-06-01 3 36 3645 364.5 7 360 364 [364.5,368.1] ['360','361','362','363','364','365','366','367','368','369'] -2014-06-01 3 37 3745 374.5 7 370 371 [374.5,378.1] ['370','371','372','373','374','375','376','377','378','379'] -2014-06-01 3 38 3845 384.5 7 380 385 [384.5,388.1] ['380','381','382','383','384','385','386','387','388','389'] -2014-06-01 3 39 3945 394.5 7 390 392 [394.5,398.1] ['390','391','392','393','394','395','396','397','398','399'] -2014-06-01 4 40 4045 404.5 7 400 406 [404.5,408.1] ['400','401','402','403','404','405','406','407','408','409'] -2014-06-01 4 41 4145 414.5 7 410 413 [414.5,418.1] ['410','411','412','413','414','415','416','417','418','419'] -2014-06-01 4 42 4245 424.5 7 420 420 [424.5,428.1] ['420','421','422','423','424','425','426','427','428','429'] -2014-06-01 4 43 4345 434.5 7 430 434 [434.5,438.1] ['430','431','432','433','434','435','436','437','438','439'] -2014-06-01 4 44 4445 444.5 7 440 441 [444.5,448.1] ['440','441','442','443','444','445','446','447','448','449'] -2014-06-01 4 45 4545 454.5 7 450 455 [454.5,458.1] ['450','451','452','453','454','455','456','457','458','459'] -2014-06-01 4 46 4645 464.5 7 460 462 [464.5,468.1] ['460','461','462','463','464','465','466','467','468','469'] -2014-06-01 4 47 4745 474.5 7 470 476 [474.5,478.09999999999997] ['470','471','472','473','474','475','476','477','478','479'] -2014-06-01 4 48 4845 484.5 7 480 483 [484.5,488.09999999999997] ['480','481','482','483','484','485','486','487','488','489'] -2014-06-01 4 49 4945 494.5 7 490 490 [494.5,498.09999999999997] ['490','491','492','493','494','495','496','497','498','499'] -2014-06-01 5 50 5045 504.5 7 500 504 [504.5,508.09999999999997] ['500','501','502','503','504','505','506','507','508','509'] -2014-06-01 5 51 5145 514.5 7 510 511 [514.5,518.1] ['510','511','512','513','514','515','516','517','518','519'] -2014-06-01 5 52 5245 524.5 7 520 525 [524.5,528.1] ['520','521','522','523','524','525','526','527','528','529'] -2014-06-01 5 53 5345 534.5 7 530 532 [534.5,538.1] ['530','531','532','533','534','535','536','537','538','539'] -2014-06-01 5 54 5445 544.5 7 540 546 [544.5,548.1] ['540','541','542','543','544','545','546','547','548','549'] -2014-06-01 5 55 5545 554.5 7 550 553 [554.5,558.1] ['550','551','552','553','554','555','556','557','558','559'] -2014-06-01 5 56 5645 564.5 7 560 560 [564.5,568.1] ['560','561','562','563','564','565','566','567','568','569'] -2014-06-01 5 57 5745 574.5 7 570 574 [574.5,578.0999999999999] ['570','571','572','573','574','575','576','577','578','579'] -2014-06-01 5 58 5845 584.5 7 580 581 [584.5,588.0999999999999] ['580','581','582','583','584','585','586','587','588','589'] -2014-06-01 5 59 5945 594.5 7 590 595 [594.5,598.0999999999999] ['590','591','592','593','594','595','596','597','598','599'] -2014-06-01 6 60 6045 604.5 7 600 602 [604.5,608.1] ['600','601','602','603','604','605','606','607','608','609'] -2014-06-01 6 61 6145 614.5 7 610 616 [614.5,618.1] ['610','611','612','613','614','615','616','617','618','619'] -2014-06-01 6 62 6245 624.5 7 620 623 [624.5,628.1] ['620','621','622','623','624','625','626','627','628','629'] -2014-06-01 6 63 6345 634.5 7 630 630 [634.5,638.1] ['630','631','632','633','634','635','636','637','638','639'] -2014-06-01 6 64 6445 644.5 7 640 644 [644.5,648.1] ['640','641','642','643','644','645','646','647','648','649'] -2014-06-01 6 65 6545 654.5 7 650 651 [654.5,658.1] ['650','651','652','653','654','655','656','657','658','659'] -2014-06-01 6 66 6645 664.5 7 660 665 [664.5,668.1] ['660','661','662','663','664','665','666','667','668','669'] -2014-06-01 6 67 6745 674.5 7 670 672 [674.5,678.1] ['670','671','672','673','674','675','676','677','678','679'] -2014-06-01 6 68 6845 684.5 7 680 686 [684.5,688.1] ['680','681','682','683','684','685','686','687','688','689'] -2014-06-01 6 69 6945 694.5 7 690 693 [694.5,698.1] ['690','691','692','693','694','695','696','697','698','699'] -2014-06-01 7 70 7045 704.5 7 700 700 [704.5,708.1] ['700','701','702','703','704','705','706','707','708','709'] -2014-06-01 7 71 7145 714.5 7 710 714 [714.5,718.1] ['710','711','712','713','714','715','716','717','718','719'] -2014-06-01 7 72 7245 724.5 7 720 721 [724.5,728.1] ['720','721','722','723','724','725','726','727','728','729'] -2014-06-01 7 73 7345 734.5 7 730 735 [734.5,738.1] ['730','731','732','733','734','735','736','737','738','739'] -2014-06-01 7 74 7445 744.5 7 740 742 [744.5,748.1] ['740','741','742','743','744','745','746','747','748','749'] -2014-06-01 7 75 7545 754.5 7 750 756 [754.5,758.1] ['750','751','752','753','754','755','756','757','758','759'] -2014-06-01 7 76 7645 764.5 7 760 763 [764.5,768.1] ['760','761','762','763','764','765','766','767','768','769'] -2014-06-01 7 77 7745 774.5 7 770 770 [774.5,778.1] ['770','771','772','773','774','775','776','777','778','779'] -2014-06-01 7 78 7845 784.5 7 780 784 [784.5,788.1] ['780','781','782','783','784','785','786','787','788','789'] -2014-06-01 7 79 7945 794.5 7 790 791 [794.5,798.1] ['790','791','792','793','794','795','796','797','798','799'] -2014-06-01 8 80 8045 804.5 7 800 805 [804.5,808.1] ['800','801','802','803','804','805','806','807','808','809'] -2014-06-01 8 81 8145 814.5 7 810 812 [814.5,818.1] ['810','811','812','813','814','815','816','817','818','819'] -2014-06-01 8 82 8245 824.5 7 820 826 [824.5,828.1] ['820','821','822','823','824','825','826','827','828','829'] -2014-06-01 8 83 8345 834.5 7 830 833 [834.5,838.1] ['830','831','832','833','834','835','836','837','838','839'] -2014-06-01 8 84 8445 844.5 7 840 840 [844.5,848.0999999999999] ['840','841','842','843','844','845','846','847','848','849'] -2014-06-01 8 85 8545 854.5 7 850 854 [854.5,858.0999999999999] ['850','851','852','853','854','855','856','857','858','859'] -2014-06-01 8 86 8645 864.5 7 860 861 [864.5,868.0999999999999] ['860','861','862','863','864','865','866','867','868','869'] -2014-06-01 8 87 8745 874.5 7 870 875 [874.5,878.0999999999999] ['870','871','872','873','874','875','876','877','878','879'] -2014-06-01 8 88 8845 884.5 7 880 882 [884.5,888.0999999999999] ['880','881','882','883','884','885','886','887','888','889'] -2014-06-01 8 89 8945 894.5 7 890 896 [894.5,898.0999999999999] ['890','891','892','893','894','895','896','897','898','899'] -2014-06-01 9 90 9045 904.5 7 900 903 [904.5,908.0999999999999] ['900','901','902','903','904','905','906','907','908','909'] -2014-06-01 9 91 9145 914.5 7 910 910 [914.5,918.0999999999999] ['910','911','912','913','914','915','916','917','918','919'] -2014-06-01 9 92 9245 924.5 7 920 924 [924.5,928.0999999999999] ['920','921','922','923','924','925','926','927','928','929'] -2014-06-01 9 93 9345 934.5 7 930 931 [934.5,938.1] ['930','931','932','933','934','935','936','937','938','939'] -2014-06-01 9 94 9445 944.5 7 940 945 [944.5,948.1] ['940','941','942','943','944','945','946','947','948','949'] -2014-06-01 9 95 9545 954.5 7 950 952 [954.5,958.1] ['950','951','952','953','954','955','956','957','958','959'] -2014-06-01 9 96 9645 964.5 7 960 966 [964.5,968.1] ['960','961','962','963','964','965','966','967','968','969'] -2014-06-01 9 97 9745 974.5 7 970 973 [974.5,978.1] ['970','971','972','973','974','975','976','977','978','979'] -2014-06-01 9 98 9845 984.5 7 980 980 [984.5,988.1] ['980','981','982','983','984','985','986','987','988','989'] -2014-06-01 9 99 9945 994.5 7 990 994 [994.5,998.1] ['990','991','992','993','994','995','996','997','998','999'] -2014-06-01 0 4950 49.5 7 0 0 [49.5,89.10000000000001] ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19','20','21','22','23','24','25','26','27','28','29','30','31','32','33','34','35','36','37','38','39','40','41','42','43','44','45','46','47','48','49','50','51','52','53','54','55','56','57','58','59','60','61','62','63','64','65','66','67','68','69','70','71','72','73','74','75','76','77','78','79','80','81','82','83','84','85','86','87','88','89','90','91','92','93','94','95','96','97','98','99'] -2014-06-01 1 14950 149.5 7 100 105 [149.5,189.1] ['100','101','102','103','104','105','106','107','108','109','110','111','112','113','114','115','116','117','118','119','120','121','122','123','124','125','126','127','128','129','130','131','132','133','134','135','136','137','138','139','140','141','142','143','144','145','146','147','148','149','150','151','152','153','154','155','156','157','158','159','160','161','162','163','164','165','166','167','168','169','170','171','172','173','174','175','176','177','178','179','180','181','182','183','184','185','186','187','188','189','190','191','192','193','194','195','196','197','198','199'] -2014-06-01 2 24950 249.5 7 200 203 [249.5,289.1] ['200','201','202','203','204','205','206','207','208','209','210','211','212','213','214','215','216','217','218','219','220','221','222','223','224','225','226','227','228','229','230','231','232','233','234','235','236','237','238','239','240','241','242','243','244','245','246','247','248','249','250','251','252','253','254','255','256','257','258','259','260','261','262','263','264','265','266','267','268','269','270','271','272','273','274','275','276','277','278','279','280','281','282','283','284','285','286','287','288','289','290','291','292','293','294','295','296','297','298','299'] -2014-06-01 3 34950 349.5 7 300 301 [349.5,389.1] ['300','301','302','303','304','305','306','307','308','309','310','311','312','313','314','315','316','317','318','319','320','321','322','323','324','325','326','327','328','329','330','331','332','333','334','335','336','337','338','339','340','341','342','343','344','345','346','347','348','349','350','351','352','353','354','355','356','357','358','359','360','361','362','363','364','365','366','367','368','369','370','371','372','373','374','375','376','377','378','379','380','381','382','383','384','385','386','387','388','389','390','391','392','393','394','395','396','397','398','399'] -2014-06-01 4 44950 449.5 7 400 406 [449.5,489.1] ['400','401','402','403','404','405','406','407','408','409','410','411','412','413','414','415','416','417','418','419','420','421','422','423','424','425','426','427','428','429','430','431','432','433','434','435','436','437','438','439','440','441','442','443','444','445','446','447','448','449','450','451','452','453','454','455','456','457','458','459','460','461','462','463','464','465','466','467','468','469','470','471','472','473','474','475','476','477','478','479','480','481','482','483','484','485','486','487','488','489','490','491','492','493','494','495','496','497','498','499'] -2014-06-01 5 54950 549.5 7 500 504 [549.5,589.1] ['500','501','502','503','504','505','506','507','508','509','510','511','512','513','514','515','516','517','518','519','520','521','522','523','524','525','526','527','528','529','530','531','532','533','534','535','536','537','538','539','540','541','542','543','544','545','546','547','548','549','550','551','552','553','554','555','556','557','558','559','560','561','562','563','564','565','566','567','568','569','570','571','572','573','574','575','576','577','578','579','580','581','582','583','584','585','586','587','588','589','590','591','592','593','594','595','596','597','598','599'] -2014-06-01 6 64950 649.5 7 600 602 [649.5,689.1] ['600','601','602','603','604','605','606','607','608','609','610','611','612','613','614','615','616','617','618','619','620','621','622','623','624','625','626','627','628','629','630','631','632','633','634','635','636','637','638','639','640','641','642','643','644','645','646','647','648','649','650','651','652','653','654','655','656','657','658','659','660','661','662','663','664','665','666','667','668','669','670','671','672','673','674','675','676','677','678','679','680','681','682','683','684','685','686','687','688','689','690','691','692','693','694','695','696','697','698','699'] -2014-06-01 7 74950 749.5 7 700 700 [749.5,789.1] ['700','701','702','703','704','705','706','707','708','709','710','711','712','713','714','715','716','717','718','719','720','721','722','723','724','725','726','727','728','729','730','731','732','733','734','735','736','737','738','739','740','741','742','743','744','745','746','747','748','749','750','751','752','753','754','755','756','757','758','759','760','761','762','763','764','765','766','767','768','769','770','771','772','773','774','775','776','777','778','779','780','781','782','783','784','785','786','787','788','789','790','791','792','793','794','795','796','797','798','799'] -2014-06-01 8 84950 849.5 7 800 805 [849.5,889.1] ['800','801','802','803','804','805','806','807','808','809','810','811','812','813','814','815','816','817','818','819','820','821','822','823','824','825','826','827','828','829','830','831','832','833','834','835','836','837','838','839','840','841','842','843','844','845','846','847','848','849','850','851','852','853','854','855','856','857','858','859','860','861','862','863','864','865','866','867','868','869','870','871','872','873','874','875','876','877','878','879','880','881','882','883','884','885','886','887','888','889','890','891','892','893','894','895','896','897','898','899'] -2014-06-01 9 94950 949.5 7 900 903 [949.5,989.1] ['900','901','902','903','904','905','906','907','908','909','910','911','912','913','914','915','916','917','918','919','920','921','922','923','924','925','926','927','928','929','930','931','932','933','934','935','936','937','938','939','940','941','942','943','944','945','946','947','948','949','950','951','952','953','954','955','956','957','958','959','960','961','962','963','964','965','966','967','968','969','970','971','972','973','974','975','976','977','978','979','980','981','982','983','984','985','986','987','988','989','990','991','992','993','994','995','996','997','998','999'] -2014-06-01 499500 499.5 7 0 0 [499.5,899.1] ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19','20','21','22','23','24','25','26','27','28','29','30','31','32','33','34','35','36','37','38','39','40','41','42','43','44','45','46','47','48','49','50','51','52','53','54','55','56','57','58','59','60','61','62','63','64','65','66','67','68','69','70','71','72','73','74','75','76','77','78','79','80','81','82','83','84','85','86','87','88','89','90','91','92','93','94','95','96','97','98','99','100','101','102','103','104','105','106','107','108','109','110','111','112','113','114','115','116','117','118','119','120','121','122','123','124','125','126','127','128','129','130','131','132','133','134','135','136','137','138','139','140','141','142','143','144','145','146','147','148','149','150','151','152','153','154','155','156','157','158','159','160','161','162','163','164','165','166','167','168','169','170','171','172','173','174','175','176','177','178','179','180','181','182','183','184','185','186','187','188','189','190','191','192','193','194','195','196','197','198','199','200','201','202','203','204','205','206','207','208','209','210','211','212','213','214','215','216','217','218','219','220','221','222','223','224','225','226','227','228','229','230','231','232','233','234','235','236','237','238','239','240','241','242','243','244','245','246','247','248','249','250','251','252','253','254','255','256','257','258','259','260','261','262','263','264','265','266','267','268','269','270','271','272','273','274','275','276','277','278','279','280','281','282','283','284','285','286','287','288','289','290','291','292','293','294','295','296','297','298','299','300','301','302','303','304','305','306','307','308','309','310','311','312','313','314','315','316','317','318','319','320','321','322','323','324','325','326','327','328','329','330','331','332','333','334','335','336','337','338','339','340','341','342','343','344','345','346','347','348','349','350','351','352','353','354','355','356','357','358','359','360','361','362','363','364','365','366','367','368','369','370','371','372','373','374','375','376','377','378','379','380','381','382','383','384','385','386','387','388','389','390','391','392','393','394','395','396','397','398','399','400','401','402','403','404','405','406','407','408','409','410','411','412','413','414','415','416','417','418','419','420','421','422','423','424','425','426','427','428','429','430','431','432','433','434','435','436','437','438','439','440','441','442','443','444','445','446','447','448','449','450','451','452','453','454','455','456','457','458','459','460','461','462','463','464','465','466','467','468','469','470','471','472','473','474','475','476','477','478','479','480','481','482','483','484','485','486','487','488','489','490','491','492','493','494','495','496','497','498','499','500','501','502','503','504','505','506','507','508','509','510','511','512','513','514','515','516','517','518','519','520','521','522','523','524','525','526','527','528','529','530','531','532','533','534','535','536','537','538','539','540','541','542','543','544','545','546','547','548','549','550','551','552','553','554','555','556','557','558','559','560','561','562','563','564','565','566','567','568','569','570','571','572','573','574','575','576','577','578','579','580','581','582','583','584','585','586','587','588','589','590','591','592','593','594','595','596','597','598','599','600','601','602','603','604','605','606','607','608','609','610','611','612','613','614','615','616','617','618','619','620','621','622','623','624','625','626','627','628','629','630','631','632','633','634','635','636','637','638','639','640','641','642','643','644','645','646','647','648','649','650','651','652','653','654','655','656','657','658','659','660','661','662','663','664','665','666','667','668','669','670','671','672','673','674','675','676','677','678','679','680','681','682','683','684','685','686','687','688','689','690','691','692','693','694','695','696','697','698','699','700','701','702','703','704','705','706','707','708','709','710','711','712','713','714','715','716','717','718','719','720','721','722','723','724','725','726','727','728','729','730','731','732','733','734','735','736','737','738','739','740','741','742','743','744','745','746','747','748','749','750','751','752','753','754','755','756','757','758','759','760','761','762','763','764','765','766','767','768','769','770','771','772','773','774','775','776','777','778','779','780','781','782','783','784','785','786','787','788','789','790','791','792','793','794','795','796','797','798','799','800','801','802','803','804','805','806','807','808','809','810','811','812','813','814','815','816','817','818','819','820','821','822','823','824','825','826','827','828','829','830','831','832','833','834','835','836','837','838','839','840','841','842','843','844','845','846','847','848','849','850','851','852','853','854','855','856','857','858','859','860','861','862','863','864','865','866','867','868','869','870','871','872','873','874','875','876','877','878','879','880','881','882','883','884','885','886','887','888','889','890','891','892','893','894','895','896','897','898','899','900','901','902','903','904','905','906','907','908','909','910','911','912','913','914','915','916','917','918','919','920','921','922','923','924','925','926','927','928','929','930','931','932','933','934','935','936','937','938','939','940','941','942','943','944','945','946','947','948','949','950','951','952','953','954','955','956','957','958','959','960','961','962','963','964','965','966','967','968','969','970','971','972','973','974','975','976','977','978','979','980','981','982','983','984','985','986','987','988','989','990','991','992','993','994','995','996','997','998','999'] +2014-06-01 0 0 45 4.5 6 0 0 [4.5,8.1] ['0','1','2','3','4','5','6','7','8','9'] +2014-06-01 0 1 145 14.5 6 10 14 [14.5,18.1] ['10','11','12','13','14','15','16','17','18','19'] +2014-06-01 0 2 245 24.5 6 20 21 [24.5,28.1] ['20','21','22','23','24','25','26','27','28','29'] +2014-06-01 0 3 345 34.5 6 30 35 [34.5,38.1] ['30','31','32','33','34','35','36','37','38','39'] +2014-06-01 0 4 445 44.5 6 40 42 [44.5,48.1] ['40','41','42','43','44','45','46','47','48','49'] +2014-06-01 0 5 545 54.5 6 50 56 [54.5,58.099999999999994] ['50','51','52','53','54','55','56','57','58','59'] +2014-06-01 0 6 645 64.5 6 60 63 [64.5,68.1] ['60','61','62','63','64','65','66','67','68','69'] +2014-06-01 0 7 745 74.5 6 70 70 [74.5,78.10000000000001] ['70','71','72','73','74','75','76','77','78','79'] +2014-06-01 0 8 845 84.5 6 80 84 [84.5,88.1] ['80','81','82','83','84','85','86','87','88','89'] +2014-06-01 0 9 945 94.5 6 90 91 [94.5,98.1] ['90','91','92','93','94','95','96','97','98','99'] +2014-06-01 1 10 1045 104.5 6 100 105 [104.5,108.10000000000001] ['100','101','102','103','104','105','106','107','108','109'] +2014-06-01 1 11 1145 114.5 6 110 112 [114.5,118.10000000000001] ['110','111','112','113','114','115','116','117','118','119'] +2014-06-01 1 12 1245 124.5 6 120 126 [124.5,128.1] ['120','121','122','123','124','125','126','127','128','129'] +2014-06-01 1 13 1345 134.5 6 130 133 [134.5,138.1] ['130','131','132','133','134','135','136','137','138','139'] +2014-06-01 1 14 1445 144.5 6 140 140 [144.5,148.1] ['140','141','142','143','144','145','146','147','148','149'] +2014-06-01 1 15 1545 154.5 6 150 154 [154.5,158.1] ['150','151','152','153','154','155','156','157','158','159'] +2014-06-01 1 16 1645 164.5 6 160 161 [164.5,168.1] ['160','161','162','163','164','165','166','167','168','169'] +2014-06-01 1 17 1745 174.5 6 170 175 [174.5,178.10000000000002] ['170','171','172','173','174','175','176','177','178','179'] +2014-06-01 1 18 1845 184.5 6 180 182 [184.5,188.10000000000002] ['180','181','182','183','184','185','186','187','188','189'] +2014-06-01 1 19 1945 194.5 6 190 196 [194.5,198.1] ['190','191','192','193','194','195','196','197','198','199'] +2014-06-01 2 20 2045 204.5 6 200 203 [204.5,208.1] ['200','201','202','203','204','205','206','207','208','209'] +2014-06-01 2 21 2145 214.5 6 210 210 [214.5,218.1] ['210','211','212','213','214','215','216','217','218','219'] +2014-06-01 2 22 2245 224.5 6 220 224 [224.5,228.1] ['220','221','222','223','224','225','226','227','228','229'] +2014-06-01 2 23 2345 234.5 6 230 231 [234.5,238.1] ['230','231','232','233','234','235','236','237','238','239'] +2014-06-01 2 24 2445 244.5 6 240 245 [244.5,248.10000000000002] ['240','241','242','243','244','245','246','247','248','249'] +2014-06-01 2 25 2545 254.5 6 250 252 [254.5,258.1] ['250','251','252','253','254','255','256','257','258','259'] +2014-06-01 2 26 2645 264.5 6 260 266 [264.5,268.1] ['260','261','262','263','264','265','266','267','268','269'] +2014-06-01 2 27 2745 274.5 6 270 273 [274.5,278.1] ['270','271','272','273','274','275','276','277','278','279'] +2014-06-01 2 28 2845 284.5 6 280 280 [284.5,288.1] ['280','281','282','283','284','285','286','287','288','289'] +2014-06-01 2 29 2945 294.5 6 290 294 [294.5,298.1] ['290','291','292','293','294','295','296','297','298','299'] +2014-06-01 3 30 3045 304.5 6 300 301 [304.5,308.1] ['300','301','302','303','304','305','306','307','308','309'] +2014-06-01 3 31 3145 314.5 6 310 315 [314.5,318.09999999999997] ['310','311','312','313','314','315','316','317','318','319'] +2014-06-01 3 32 3245 324.5 6 320 322 [324.5,328.09999999999997] ['320','321','322','323','324','325','326','327','328','329'] +2014-06-01 3 33 3345 334.5 6 330 336 [334.5,338.09999999999997] ['330','331','332','333','334','335','336','337','338','339'] +2014-06-01 3 34 3445 344.5 6 340 343 [344.5,348.09999999999997] ['340','341','342','343','344','345','346','347','348','349'] +2014-06-01 3 35 3545 354.5 6 350 350 [354.5,358.09999999999997] ['350','351','352','353','354','355','356','357','358','359'] +2014-06-01 3 36 3645 364.5 6 360 364 [364.5,368.1] ['360','361','362','363','364','365','366','367','368','369'] +2014-06-01 3 37 3745 374.5 6 370 371 [374.5,378.1] ['370','371','372','373','374','375','376','377','378','379'] +2014-06-01 3 38 3845 384.5 6 380 385 [384.5,388.1] ['380','381','382','383','384','385','386','387','388','389'] +2014-06-01 3 39 3945 394.5 6 390 392 [394.5,398.1] ['390','391','392','393','394','395','396','397','398','399'] +2014-06-01 4 40 4045 404.5 6 400 406 [404.5,408.1] ['400','401','402','403','404','405','406','407','408','409'] +2014-06-01 4 41 4145 414.5 6 410 413 [414.5,418.1] ['410','411','412','413','414','415','416','417','418','419'] +2014-06-01 4 42 4245 424.5 6 420 420 [424.5,428.1] ['420','421','422','423','424','425','426','427','428','429'] +2014-06-01 4 43 4345 434.5 6 430 434 [434.5,438.1] ['430','431','432','433','434','435','436','437','438','439'] +2014-06-01 4 44 4445 444.5 6 440 441 [444.5,448.1] ['440','441','442','443','444','445','446','447','448','449'] +2014-06-01 4 45 4545 454.5 6 450 455 [454.5,458.1] ['450','451','452','453','454','455','456','457','458','459'] +2014-06-01 4 46 4645 464.5 6 460 462 [464.5,468.1] ['460','461','462','463','464','465','466','467','468','469'] +2014-06-01 4 47 4745 474.5 6 470 476 [474.5,478.09999999999997] ['470','471','472','473','474','475','476','477','478','479'] +2014-06-01 4 48 4845 484.5 6 480 483 [484.5,488.09999999999997] ['480','481','482','483','484','485','486','487','488','489'] +2014-06-01 4 49 4945 494.5 6 490 490 [494.5,498.09999999999997] ['490','491','492','493','494','495','496','497','498','499'] +2014-06-01 5 50 5045 504.5 6 500 504 [504.5,508.09999999999997] ['500','501','502','503','504','505','506','507','508','509'] +2014-06-01 5 51 5145 514.5 6 510 511 [514.5,518.1] ['510','511','512','513','514','515','516','517','518','519'] +2014-06-01 5 52 5245 524.5 6 520 525 [524.5,528.1] ['520','521','522','523','524','525','526','527','528','529'] +2014-06-01 5 53 5345 534.5 6 530 532 [534.5,538.1] ['530','531','532','533','534','535','536','537','538','539'] +2014-06-01 5 54 5445 544.5 6 540 546 [544.5,548.1] ['540','541','542','543','544','545','546','547','548','549'] +2014-06-01 5 55 5545 554.5 6 550 553 [554.5,558.1] ['550','551','552','553','554','555','556','557','558','559'] +2014-06-01 5 56 5645 564.5 6 560 560 [564.5,568.1] ['560','561','562','563','564','565','566','567','568','569'] +2014-06-01 5 57 5745 574.5 6 570 574 [574.5,578.0999999999999] ['570','571','572','573','574','575','576','577','578','579'] +2014-06-01 5 58 5845 584.5 6 580 581 [584.5,588.0999999999999] ['580','581','582','583','584','585','586','587','588','589'] +2014-06-01 5 59 5945 594.5 6 590 595 [594.5,598.0999999999999] ['590','591','592','593','594','595','596','597','598','599'] +2014-06-01 6 60 6045 604.5 6 600 602 [604.5,608.1] ['600','601','602','603','604','605','606','607','608','609'] +2014-06-01 6 61 6145 614.5 6 610 616 [614.5,618.1] ['610','611','612','613','614','615','616','617','618','619'] +2014-06-01 6 62 6245 624.5 6 620 623 [624.5,628.1] ['620','621','622','623','624','625','626','627','628','629'] +2014-06-01 6 63 6345 634.5 6 630 630 [634.5,638.1] ['630','631','632','633','634','635','636','637','638','639'] +2014-06-01 6 64 6445 644.5 6 640 644 [644.5,648.1] ['640','641','642','643','644','645','646','647','648','649'] +2014-06-01 6 65 6545 654.5 6 650 651 [654.5,658.1] ['650','651','652','653','654','655','656','657','658','659'] +2014-06-01 6 66 6645 664.5 6 660 665 [664.5,668.1] ['660','661','662','663','664','665','666','667','668','669'] +2014-06-01 6 67 6745 674.5 6 670 672 [674.5,678.1] ['670','671','672','673','674','675','676','677','678','679'] +2014-06-01 6 68 6845 684.5 6 680 686 [684.5,688.1] ['680','681','682','683','684','685','686','687','688','689'] +2014-06-01 6 69 6945 694.5 6 690 693 [694.5,698.1] ['690','691','692','693','694','695','696','697','698','699'] +2014-06-01 7 70 7045 704.5 6 700 700 [704.5,708.1] ['700','701','702','703','704','705','706','707','708','709'] +2014-06-01 7 71 7145 714.5 6 710 714 [714.5,718.1] ['710','711','712','713','714','715','716','717','718','719'] +2014-06-01 7 72 7245 724.5 6 720 721 [724.5,728.1] ['720','721','722','723','724','725','726','727','728','729'] +2014-06-01 7 73 7345 734.5 6 730 735 [734.5,738.1] ['730','731','732','733','734','735','736','737','738','739'] +2014-06-01 7 74 7445 744.5 6 740 742 [744.5,748.1] ['740','741','742','743','744','745','746','747','748','749'] +2014-06-01 7 75 7545 754.5 6 750 756 [754.5,758.1] ['750','751','752','753','754','755','756','757','758','759'] +2014-06-01 7 76 7645 764.5 6 760 763 [764.5,768.1] ['760','761','762','763','764','765','766','767','768','769'] +2014-06-01 7 77 7745 774.5 6 770 770 [774.5,778.1] ['770','771','772','773','774','775','776','777','778','779'] +2014-06-01 7 78 7845 784.5 6 780 784 [784.5,788.1] ['780','781','782','783','784','785','786','787','788','789'] +2014-06-01 7 79 7945 794.5 6 790 791 [794.5,798.1] ['790','791','792','793','794','795','796','797','798','799'] +2014-06-01 8 80 8045 804.5 6 800 805 [804.5,808.1] ['800','801','802','803','804','805','806','807','808','809'] +2014-06-01 8 81 8145 814.5 6 810 812 [814.5,818.1] ['810','811','812','813','814','815','816','817','818','819'] +2014-06-01 8 82 8245 824.5 6 820 826 [824.5,828.1] ['820','821','822','823','824','825','826','827','828','829'] +2014-06-01 8 83 8345 834.5 6 830 833 [834.5,838.1] ['830','831','832','833','834','835','836','837','838','839'] +2014-06-01 8 84 8445 844.5 6 840 840 [844.5,848.0999999999999] ['840','841','842','843','844','845','846','847','848','849'] +2014-06-01 8 85 8545 854.5 6 850 854 [854.5,858.0999999999999] ['850','851','852','853','854','855','856','857','858','859'] +2014-06-01 8 86 8645 864.5 6 860 861 [864.5,868.0999999999999] ['860','861','862','863','864','865','866','867','868','869'] +2014-06-01 8 87 8745 874.5 6 870 875 [874.5,878.0999999999999] ['870','871','872','873','874','875','876','877','878','879'] +2014-06-01 8 88 8845 884.5 6 880 882 [884.5,888.0999999999999] ['880','881','882','883','884','885','886','887','888','889'] +2014-06-01 8 89 8945 894.5 6 890 896 [894.5,898.0999999999999] ['890','891','892','893','894','895','896','897','898','899'] +2014-06-01 9 90 9045 904.5 6 900 903 [904.5,908.0999999999999] ['900','901','902','903','904','905','906','907','908','909'] +2014-06-01 9 91 9145 914.5 6 910 910 [914.5,918.0999999999999] ['910','911','912','913','914','915','916','917','918','919'] +2014-06-01 9 92 9245 924.5 6 920 924 [924.5,928.0999999999999] ['920','921','922','923','924','925','926','927','928','929'] +2014-06-01 9 93 9345 934.5 6 930 931 [934.5,938.1] ['930','931','932','933','934','935','936','937','938','939'] +2014-06-01 9 94 9445 944.5 6 940 945 [944.5,948.1] ['940','941','942','943','944','945','946','947','948','949'] +2014-06-01 9 95 9545 954.5 6 950 952 [954.5,958.1] ['950','951','952','953','954','955','956','957','958','959'] +2014-06-01 9 96 9645 964.5 6 960 966 [964.5,968.1] ['960','961','962','963','964','965','966','967','968','969'] +2014-06-01 9 97 9745 974.5 6 970 973 [974.5,978.1] ['970','971','972','973','974','975','976','977','978','979'] +2014-06-01 9 98 9845 984.5 6 980 980 [984.5,988.1] ['980','981','982','983','984','985','986','987','988','989'] +2014-06-01 9 99 9945 994.5 6 990 994 [994.5,998.1] ['990','991','992','993','994','995','996','997','998','999'] +2014-06-01 0 4950 49.5 6 0 0 [49.5,89.10000000000001] ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19','20','21','22','23','24','25','26','27','28','29','30','31','32','33','34','35','36','37','38','39','40','41','42','43','44','45','46','47','48','49','50','51','52','53','54','55','56','57','58','59','60','61','62','63','64','65','66','67','68','69','70','71','72','73','74','75','76','77','78','79','80','81','82','83','84','85','86','87','88','89','90','91','92','93','94','95','96','97','98','99'] +2014-06-01 1 14950 149.5 6 100 105 [149.5,189.1] ['100','101','102','103','104','105','106','107','108','109','110','111','112','113','114','115','116','117','118','119','120','121','122','123','124','125','126','127','128','129','130','131','132','133','134','135','136','137','138','139','140','141','142','143','144','145','146','147','148','149','150','151','152','153','154','155','156','157','158','159','160','161','162','163','164','165','166','167','168','169','170','171','172','173','174','175','176','177','178','179','180','181','182','183','184','185','186','187','188','189','190','191','192','193','194','195','196','197','198','199'] +2014-06-01 2 24950 249.5 6 200 203 [249.5,289.1] ['200','201','202','203','204','205','206','207','208','209','210','211','212','213','214','215','216','217','218','219','220','221','222','223','224','225','226','227','228','229','230','231','232','233','234','235','236','237','238','239','240','241','242','243','244','245','246','247','248','249','250','251','252','253','254','255','256','257','258','259','260','261','262','263','264','265','266','267','268','269','270','271','272','273','274','275','276','277','278','279','280','281','282','283','284','285','286','287','288','289','290','291','292','293','294','295','296','297','298','299'] +2014-06-01 3 34950 349.5 6 300 301 [349.5,389.1] ['300','301','302','303','304','305','306','307','308','309','310','311','312','313','314','315','316','317','318','319','320','321','322','323','324','325','326','327','328','329','330','331','332','333','334','335','336','337','338','339','340','341','342','343','344','345','346','347','348','349','350','351','352','353','354','355','356','357','358','359','360','361','362','363','364','365','366','367','368','369','370','371','372','373','374','375','376','377','378','379','380','381','382','383','384','385','386','387','388','389','390','391','392','393','394','395','396','397','398','399'] +2014-06-01 4 44950 449.5 6 400 406 [449.5,489.1] ['400','401','402','403','404','405','406','407','408','409','410','411','412','413','414','415','416','417','418','419','420','421','422','423','424','425','426','427','428','429','430','431','432','433','434','435','436','437','438','439','440','441','442','443','444','445','446','447','448','449','450','451','452','453','454','455','456','457','458','459','460','461','462','463','464','465','466','467','468','469','470','471','472','473','474','475','476','477','478','479','480','481','482','483','484','485','486','487','488','489','490','491','492','493','494','495','496','497','498','499'] +2014-06-01 5 54950 549.5 6 500 504 [549.5,589.1] ['500','501','502','503','504','505','506','507','508','509','510','511','512','513','514','515','516','517','518','519','520','521','522','523','524','525','526','527','528','529','530','531','532','533','534','535','536','537','538','539','540','541','542','543','544','545','546','547','548','549','550','551','552','553','554','555','556','557','558','559','560','561','562','563','564','565','566','567','568','569','570','571','572','573','574','575','576','577','578','579','580','581','582','583','584','585','586','587','588','589','590','591','592','593','594','595','596','597','598','599'] +2014-06-01 6 64950 649.5 6 600 602 [649.5,689.1] ['600','601','602','603','604','605','606','607','608','609','610','611','612','613','614','615','616','617','618','619','620','621','622','623','624','625','626','627','628','629','630','631','632','633','634','635','636','637','638','639','640','641','642','643','644','645','646','647','648','649','650','651','652','653','654','655','656','657','658','659','660','661','662','663','664','665','666','667','668','669','670','671','672','673','674','675','676','677','678','679','680','681','682','683','684','685','686','687','688','689','690','691','692','693','694','695','696','697','698','699'] +2014-06-01 7 74950 749.5 6 700 700 [749.5,789.1] ['700','701','702','703','704','705','706','707','708','709','710','711','712','713','714','715','716','717','718','719','720','721','722','723','724','725','726','727','728','729','730','731','732','733','734','735','736','737','738','739','740','741','742','743','744','745','746','747','748','749','750','751','752','753','754','755','756','757','758','759','760','761','762','763','764','765','766','767','768','769','770','771','772','773','774','775','776','777','778','779','780','781','782','783','784','785','786','787','788','789','790','791','792','793','794','795','796','797','798','799'] +2014-06-01 8 84950 849.5 6 800 805 [849.5,889.1] ['800','801','802','803','804','805','806','807','808','809','810','811','812','813','814','815','816','817','818','819','820','821','822','823','824','825','826','827','828','829','830','831','832','833','834','835','836','837','838','839','840','841','842','843','844','845','846','847','848','849','850','851','852','853','854','855','856','857','858','859','860','861','862','863','864','865','866','867','868','869','870','871','872','873','874','875','876','877','878','879','880','881','882','883','884','885','886','887','888','889','890','891','892','893','894','895','896','897','898','899'] +2014-06-01 9 94950 949.5 6 900 903 [949.5,989.1] ['900','901','902','903','904','905','906','907','908','909','910','911','912','913','914','915','916','917','918','919','920','921','922','923','924','925','926','927','928','929','930','931','932','933','934','935','936','937','938','939','940','941','942','943','944','945','946','947','948','949','950','951','952','953','954','955','956','957','958','959','960','961','962','963','964','965','966','967','968','969','970','971','972','973','974','975','976','977','978','979','980','981','982','983','984','985','986','987','988','989','990','991','992','993','994','995','996','997','998','999'] +2014-06-01 499500 499.5 6 0 0 [499.5,899.1] ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19','20','21','22','23','24','25','26','27','28','29','30','31','32','33','34','35','36','37','38','39','40','41','42','43','44','45','46','47','48','49','50','51','52','53','54','55','56','57','58','59','60','61','62','63','64','65','66','67','68','69','70','71','72','73','74','75','76','77','78','79','80','81','82','83','84','85','86','87','88','89','90','91','92','93','94','95','96','97','98','99','100','101','102','103','104','105','106','107','108','109','110','111','112','113','114','115','116','117','118','119','120','121','122','123','124','125','126','127','128','129','130','131','132','133','134','135','136','137','138','139','140','141','142','143','144','145','146','147','148','149','150','151','152','153','154','155','156','157','158','159','160','161','162','163','164','165','166','167','168','169','170','171','172','173','174','175','176','177','178','179','180','181','182','183','184','185','186','187','188','189','190','191','192','193','194','195','196','197','198','199','200','201','202','203','204','205','206','207','208','209','210','211','212','213','214','215','216','217','218','219','220','221','222','223','224','225','226','227','228','229','230','231','232','233','234','235','236','237','238','239','240','241','242','243','244','245','246','247','248','249','250','251','252','253','254','255','256','257','258','259','260','261','262','263','264','265','266','267','268','269','270','271','272','273','274','275','276','277','278','279','280','281','282','283','284','285','286','287','288','289','290','291','292','293','294','295','296','297','298','299','300','301','302','303','304','305','306','307','308','309','310','311','312','313','314','315','316','317','318','319','320','321','322','323','324','325','326','327','328','329','330','331','332','333','334','335','336','337','338','339','340','341','342','343','344','345','346','347','348','349','350','351','352','353','354','355','356','357','358','359','360','361','362','363','364','365','366','367','368','369','370','371','372','373','374','375','376','377','378','379','380','381','382','383','384','385','386','387','388','389','390','391','392','393','394','395','396','397','398','399','400','401','402','403','404','405','406','407','408','409','410','411','412','413','414','415','416','417','418','419','420','421','422','423','424','425','426','427','428','429','430','431','432','433','434','435','436','437','438','439','440','441','442','443','444','445','446','447','448','449','450','451','452','453','454','455','456','457','458','459','460','461','462','463','464','465','466','467','468','469','470','471','472','473','474','475','476','477','478','479','480','481','482','483','484','485','486','487','488','489','490','491','492','493','494','495','496','497','498','499','500','501','502','503','504','505','506','507','508','509','510','511','512','513','514','515','516','517','518','519','520','521','522','523','524','525','526','527','528','529','530','531','532','533','534','535','536','537','538','539','540','541','542','543','544','545','546','547','548','549','550','551','552','553','554','555','556','557','558','559','560','561','562','563','564','565','566','567','568','569','570','571','572','573','574','575','576','577','578','579','580','581','582','583','584','585','586','587','588','589','590','591','592','593','594','595','596','597','598','599','600','601','602','603','604','605','606','607','608','609','610','611','612','613','614','615','616','617','618','619','620','621','622','623','624','625','626','627','628','629','630','631','632','633','634','635','636','637','638','639','640','641','642','643','644','645','646','647','648','649','650','651','652','653','654','655','656','657','658','659','660','661','662','663','664','665','666','667','668','669','670','671','672','673','674','675','676','677','678','679','680','681','682','683','684','685','686','687','688','689','690','691','692','693','694','695','696','697','698','699','700','701','702','703','704','705','706','707','708','709','710','711','712','713','714','715','716','717','718','719','720','721','722','723','724','725','726','727','728','729','730','731','732','733','734','735','736','737','738','739','740','741','742','743','744','745','746','747','748','749','750','751','752','753','754','755','756','757','758','759','760','761','762','763','764','765','766','767','768','769','770','771','772','773','774','775','776','777','778','779','780','781','782','783','784','785','786','787','788','789','790','791','792','793','794','795','796','797','798','799','800','801','802','803','804','805','806','807','808','809','810','811','812','813','814','815','816','817','818','819','820','821','822','823','824','825','826','827','828','829','830','831','832','833','834','835','836','837','838','839','840','841','842','843','844','845','846','847','848','849','850','851','852','853','854','855','856','857','858','859','860','861','862','863','864','865','866','867','868','869','870','871','872','873','874','875','876','877','878','879','880','881','882','883','884','885','886','887','888','889','890','891','892','893','894','895','896','897','898','899','900','901','902','903','904','905','906','907','908','909','910','911','912','913','914','915','916','917','918','919','920','921','922','923','924','925','926','927','928','929','930','931','932','933','934','935','936','937','938','939','940','941','942','943','944','945','946','947','948','949','950','951','952','953','954','955','956','957','958','959','960','961','962','963','964','965','966','967','968','969','970','971','972','973','974','975','976','977','978','979','980','981','982','983','984','985','986','987','988','989','990','991','992','993','994','995','996','997','998','999'] diff --git a/dbms/tests/queries/0_stateless/00048_a_stored_aggregates_merge.reference b/dbms/tests/queries/0_stateless/00048_a_stored_aggregates_merge.reference index e48dcb7503e..4ab320160e4 100644 --- a/dbms/tests/queries/0_stateless/00048_a_stored_aggregates_merge.reference +++ b/dbms/tests/queries/0_stateless/00048_a_stored_aggregates_merge.reference @@ -1,43 +1,43 @@ -100 -2014-06-01 10 -2014-06-02 10 -2014-06-03 10 -2014-06-04 10 -2014-06-05 10 -2014-06-06 10 -2014-06-07 10 -2014-06-08 10 -2014-06-09 10 -2014-06-10 10 -155 -2014-06-01 10 -2014-06-02 10 -2014-06-03 10 -2014-06-04 10 -2014-06-05 10 -2014-06-06 15 -2014-06-07 15 -2014-06-08 15 -2014-06-09 15 -2014-06-10 15 -2014-06-11 10 -2014-06-12 10 -2014-06-13 10 -2014-06-14 10 -2014-06-15 10 -155 -2014-06-01 10 -2014-06-02 10 -2014-06-03 10 -2014-06-04 10 -2014-06-05 10 -2014-06-06 15 -2014-06-07 15 -2014-06-08 15 -2014-06-09 15 -2014-06-10 15 -2014-06-11 10 -2014-06-12 10 -2014-06-13 10 -2014-06-14 10 -2014-06-15 10 +99 +2014-06-01 9 +2014-06-02 9 +2014-06-03 9 +2014-06-04 9 +2014-06-05 9 +2014-06-06 9 +2014-06-07 9 +2014-06-08 9 +2014-06-09 9 +2014-06-10 9 +154 +2014-06-01 9 +2014-06-02 9 +2014-06-03 9 +2014-06-04 9 +2014-06-05 9 +2014-06-06 14 +2014-06-07 14 +2014-06-08 14 +2014-06-09 14 +2014-06-10 14 +2014-06-11 9 +2014-06-12 9 +2014-06-13 9 +2014-06-14 9 +2014-06-15 9 +154 +2014-06-01 9 +2014-06-02 9 +2014-06-03 9 +2014-06-04 9 +2014-06-05 9 +2014-06-06 14 +2014-06-07 14 +2014-06-08 14 +2014-06-09 14 +2014-06-10 14 +2014-06-11 9 +2014-06-12 9 +2014-06-13 9 +2014-06-14 9 +2014-06-15 9 diff --git a/dbms/tests/queries/0_stateless/00048_b_stored_aggregates_merge.reference b/dbms/tests/queries/0_stateless/00048_b_stored_aggregates_merge.reference index e48dcb7503e..4ab320160e4 100644 --- a/dbms/tests/queries/0_stateless/00048_b_stored_aggregates_merge.reference +++ b/dbms/tests/queries/0_stateless/00048_b_stored_aggregates_merge.reference @@ -1,43 +1,43 @@ -100 -2014-06-01 10 -2014-06-02 10 -2014-06-03 10 -2014-06-04 10 -2014-06-05 10 -2014-06-06 10 -2014-06-07 10 -2014-06-08 10 -2014-06-09 10 -2014-06-10 10 -155 -2014-06-01 10 -2014-06-02 10 -2014-06-03 10 -2014-06-04 10 -2014-06-05 10 -2014-06-06 15 -2014-06-07 15 -2014-06-08 15 -2014-06-09 15 -2014-06-10 15 -2014-06-11 10 -2014-06-12 10 -2014-06-13 10 -2014-06-14 10 -2014-06-15 10 -155 -2014-06-01 10 -2014-06-02 10 -2014-06-03 10 -2014-06-04 10 -2014-06-05 10 -2014-06-06 15 -2014-06-07 15 -2014-06-08 15 -2014-06-09 15 -2014-06-10 15 -2014-06-11 10 -2014-06-12 10 -2014-06-13 10 -2014-06-14 10 -2014-06-15 10 +99 +2014-06-01 9 +2014-06-02 9 +2014-06-03 9 +2014-06-04 9 +2014-06-05 9 +2014-06-06 9 +2014-06-07 9 +2014-06-08 9 +2014-06-09 9 +2014-06-10 9 +154 +2014-06-01 9 +2014-06-02 9 +2014-06-03 9 +2014-06-04 9 +2014-06-05 9 +2014-06-06 14 +2014-06-07 14 +2014-06-08 14 +2014-06-09 14 +2014-06-10 14 +2014-06-11 9 +2014-06-12 9 +2014-06-13 9 +2014-06-14 9 +2014-06-15 9 +154 +2014-06-01 9 +2014-06-02 9 +2014-06-03 9 +2014-06-04 9 +2014-06-05 9 +2014-06-06 14 +2014-06-07 14 +2014-06-08 14 +2014-06-09 14 +2014-06-10 14 +2014-06-11 9 +2014-06-12 9 +2014-06-13 9 +2014-06-14 9 +2014-06-15 9 diff --git a/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.reference b/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.reference index 1957f3a9604..2e50651146b 100644 --- a/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.reference +++ b/dbms/tests/queries/0_stateless/00184_distributed_group_by_no_merge.reference @@ -1,2 +1,2 @@ -1 1 -1 1 +1 0 +1 0 diff --git a/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.reference b/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.reference index c696b78280f..c53c45191b1 100644 --- a/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.reference +++ b/dbms/tests/queries/0_stateless/00188_constants_as_arguments_of_aggregate_functions.reference @@ -1 +1 @@ -10 10 1 +10 10 0 From 0296dc2b8817bba5ac7c1c928f99bcbb1da7fe05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Jul 2015 22:24:51 +0300 Subject: [PATCH 22/24] dbms: fixing error with function replicate of arrays of consts [#METR-17110]. --- dbms/include/DB/Columns/ColumnArray.h | 283 +-------------------- dbms/src/Columns/ColumnArray.cpp | 338 ++++++++++++++++++++++++++ 2 files changed, 351 insertions(+), 270 deletions(-) create mode 100644 dbms/src/Columns/ColumnArray.cpp diff --git a/dbms/include/DB/Columns/ColumnArray.h b/dbms/include/DB/Columns/ColumnArray.h index 42fae9fbc3c..170dafce004 100644 --- a/dbms/include/DB/Columns/ColumnArray.h +++ b/dbms/include/DB/Columns/ColumnArray.h @@ -119,38 +119,7 @@ public: getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems); } - ColumnPtr cut(size_t start, size_t length) const override - { - if (length == 0) - return new ColumnArray(data); - - if (start + length > getOffsets().size()) - throw Exception("Parameter out of bound in IColumnArray::cut() method.", - ErrorCodes::PARAMETER_OUT_OF_BOUND); - - size_t nested_offset = offsetAt(start); - size_t nested_length = getOffsets()[start + length - 1] - nested_offset; - - ColumnArray * res_ = new ColumnArray(data); - ColumnPtr res = res_; - - res_->data = data->cut(nested_offset, nested_length); - Offsets_t & res_offsets = res_->getOffsets(); - - if (start == 0) - { - res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length); - } - else - { - res_offsets.resize(length); - - for (size_t i = 0; i < length; ++i) - res_offsets[i] = getOffsets()[start + i] - nested_offset; - } - - return res; - } + ColumnPtr cut(size_t start, size_t length) const override; void insert(const Field & x) override { @@ -178,82 +147,9 @@ public: getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back()); } - ColumnPtr filter(const Filter & filt) const override - { - size_t size = getOffsets().size(); - if (size != filt.size()) - throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + ColumnPtr filter(const Filter & filt) const override; - if (size == 0) - return new ColumnArray(data); - - /// Не слишком оптимально. Можно сделать специализацию для массивов известных типов. - Filter nested_filt(getOffsets().back()); - for (size_t i = 0; i < size; ++i) - { - if (filt[i]) - memset(&nested_filt[offsetAt(i)], 1, sizeAt(i)); - else - memset(&nested_filt[offsetAt(i)], 0, sizeAt(i)); - } - - ColumnArray * res_ = new ColumnArray(data); - ColumnPtr res = res_; - res_->data = data->filter(nested_filt); - - Offsets_t & res_offsets = res_->getOffsets(); - res_offsets.reserve(size); - - size_t current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - if (filt[i]) - { - current_offset += sizeAt(i); - res_offsets.push_back(current_offset); - } - } - - return res; - } - - ColumnPtr permute(const Permutation & perm, size_t limit) const override - { - size_t size = getOffsets().size(); - - if (limit == 0) - limit = size; - else - limit = std::min(size, limit); - - if (perm.size() < limit) - throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - if (limit == 0) - return new ColumnArray(data); - - Permutation nested_perm(getOffsets().back()); - - ColumnArray * res_ = new ColumnArray(data->cloneEmpty()); - ColumnPtr res = res_; - - Offsets_t & res_offsets = res_->getOffsets(); - res_offsets.resize(limit); - size_t current_offset = 0; - - for (size_t i = 0; i < limit; ++i) - { - for (size_t j = 0; j < sizeAt(perm[i]); ++j) - nested_perm[current_offset + j] = offsetAt(perm[i]) + j; - current_offset += sizeAt(perm[i]); - res_offsets[i] = current_offset; - } - - if (current_offset != 0) - res_->data = data->permute(nested_perm, current_offset); - - return res; - } + ColumnPtr permute(const Permutation & perm, size_t limit) const override; int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override { @@ -290,31 +186,7 @@ public: } }; - void getPermutation(bool reverse, size_t limit, Permutation & res) const override - { - size_t s = size(); - if (limit >= s) - limit = 0; - - res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; - - if (limit) - { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); - } - else - { - if (reverse) - std::sort(res.begin(), res.end(), less(*this)); - else - std::sort(res.begin(), res.end(), less(*this)); - } - } + void getPermutation(bool reverse, size_t limit, Permutation & res) const override; void reserve(size_t n) override { @@ -364,24 +236,7 @@ public: const ColumnPtr & getOffsetsColumn() const { return offsets; } - ColumnPtr replicate(const Offsets_t & replicate_offsets) const override - { - /// Не получается реализовать в общем случае. - - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicate(replicate_offsets); - if (typeid_cast(&*data)) return replicateString(replicate_offsets); - - throw Exception("Replication of column " + getName() + " is not implemented.", ErrorCodes::NOT_IMPLEMENTED); - } + ColumnPtr replicate(const Offsets_t & replicate_offsets) const override; private: ColumnPtr data; @@ -393,129 +248,17 @@ private: /// Размножить значения, если вложенный столбец - ColumnArray. template - ColumnPtr replicate(const Offsets_t & replicate_offsets) const - { - size_t col_size = size(); - if (col_size != replicate_offsets.size()) - throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - ColumnPtr res = cloneEmpty(); - - if (0 == col_size) - return res; - - ColumnArray & res_ = typeid_cast(*res); - - const typename ColumnVector::Container_t & cur_data = typeid_cast &>(*data).getData(); - const Offsets_t & cur_offsets = getOffsets(); - - typename ColumnVector::Container_t & res_data = typeid_cast &>(res_.getData()).getData(); - Offsets_t & res_offsets = res_.getOffsets(); - - res_data.reserve(data->size() / col_size * replicate_offsets.back()); - res_offsets.reserve(replicate_offsets.back()); - - Offset_t prev_replicate_offset = 0; - Offset_t prev_data_offset = 0; - Offset_t current_new_offset = 0; - - for (size_t i = 0; i < col_size; ++i) - { - size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; - size_t value_size = cur_offsets[i] - prev_data_offset; - - for (size_t j = 0; j < size_to_replicate; ++j) - { - current_new_offset += value_size; - res_offsets.push_back(current_new_offset); - - res_data.resize(res_data.size() + value_size); - memcpy(&res_data[res_data.size() - value_size], &cur_data[prev_data_offset], value_size * sizeof(T)); - } - - prev_replicate_offset = replicate_offsets[i]; - prev_data_offset = cur_offsets[i]; - } - - return res; - } + ColumnPtr replicate(const Offsets_t & replicate_offsets) const; /// Размножить значения, если вложенный столбец - ColumnString. Код слишком сложный. - ColumnPtr replicateString(const Offsets_t & replicate_offsets) const - { - size_t col_size = size(); - if (col_size != replicate_offsets.size()) - throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + ColumnPtr replicateString(const Offsets_t & replicate_offsets) const; - ColumnPtr res = cloneEmpty(); - - if (0 == col_size) - return res; - - ColumnArray & res_ = typeid_cast(*res); - - const ColumnString & cur_string = typeid_cast(*data); - const ColumnString::Chars_t & cur_chars = cur_string.getChars(); - const Offsets_t & cur_string_offsets = cur_string.getOffsets(); - const Offsets_t & cur_offsets = getOffsets(); - - ColumnString::Chars_t & res_chars = typeid_cast(res_.getData()).getChars(); - Offsets_t & res_string_offsets = typeid_cast(res_.getData()).getOffsets(); - Offsets_t & res_offsets = res_.getOffsets(); - - res_chars.reserve(cur_chars.size() / col_size * replicate_offsets.back()); - res_string_offsets.reserve(cur_string_offsets.size() / col_size * replicate_offsets.back()); - res_offsets.reserve(replicate_offsets.back()); - - Offset_t prev_replicate_offset = 0; - - Offset_t prev_cur_offset = 0; - Offset_t prev_cur_string_offset = 0; - - Offset_t current_res_offset = 0; - Offset_t current_res_string_offset = 0; - - for (size_t i = 0; i < col_size; ++i) - { - /// Насколько размножить массив. - size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; - /// Количество строк в массиве. - size_t value_size = cur_offsets[i] - prev_cur_offset; - - size_t sum_chars_size = 0; - - for (size_t j = 0; j < size_to_replicate; ++j) - { - current_res_offset += value_size; - res_offsets.push_back(current_res_offset); - - sum_chars_size = 0; - - size_t prev_cur_string_offset_local = prev_cur_string_offset; - for (size_t k = 0; k < value_size; ++k) - { - /// Размер одной строки. - size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local; - - current_res_string_offset += chars_size; - res_string_offsets.push_back(current_res_string_offset); - - /// Копирование символов одной строки. - res_chars.resize(res_chars.size() + chars_size); - memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size); - - sum_chars_size += chars_size; - prev_cur_string_offset_local += chars_size; - } - } - - prev_replicate_offset = replicate_offsets[i]; - prev_cur_offset = cur_offsets[i]; - prev_cur_string_offset += sum_chars_size; - } - - return res; - } + /** Неконстантные массивы константных значений - довольно редкое явление. + * Большинство функций не умеет с ними работать, и не создаёт такие столбцы в качестве результата. + * Исключение - функция replicate (см. FunctionsMiscellaneous.h), которая имеет служебное значение для реализации лямбда-функций. + * Только ради неё сделана реализация метода replicate для ColumnArray(ColumnConst). + */ + ColumnPtr replicateConst(const Offsets_t & replicate_offsets) const; }; diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp new file mode 100644 index 00000000000..2f9a7945ae7 --- /dev/null +++ b/dbms/src/Columns/ColumnArray.cpp @@ -0,0 +1,338 @@ +#include + + +namespace DB +{ + + +ColumnPtr ColumnArray::cut(size_t start, size_t length) const +{ + if (length == 0) + return new ColumnArray(data); + + if (start + length > getOffsets().size()) + throw Exception("Parameter out of bound in IColumnArray::cut() method.", + ErrorCodes::PARAMETER_OUT_OF_BOUND); + + size_t nested_offset = offsetAt(start); + size_t nested_length = getOffsets()[start + length - 1] - nested_offset; + + ColumnArray * res_ = new ColumnArray(data); + ColumnPtr res = res_; + + res_->data = data->cut(nested_offset, nested_length); + Offsets_t & res_offsets = res_->getOffsets(); + + if (start == 0) + { + res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length); + } + else + { + res_offsets.resize(length); + + for (size_t i = 0; i < length; ++i) + res_offsets[i] = getOffsets()[start + i] - nested_offset; + } + + return res; +} + + +ColumnPtr ColumnArray::filter(const Filter & filt) const +{ + size_t size = getOffsets().size(); + if (size != filt.size()) + throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + if (size == 0) + return new ColumnArray(data); + + /// Не слишком оптимально. Можно сделать специализацию для массивов известных типов. + Filter nested_filt(getOffsets().back()); + for (size_t i = 0; i < size; ++i) + { + if (filt[i]) + memset(&nested_filt[offsetAt(i)], 1, sizeAt(i)); + else + memset(&nested_filt[offsetAt(i)], 0, sizeAt(i)); + } + + ColumnArray * res_ = new ColumnArray(data); + ColumnPtr res = res_; + res_->data = data->filter(nested_filt); + + Offsets_t & res_offsets = res_->getOffsets(); + res_offsets.reserve(size); + + size_t current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + if (filt[i]) + { + current_offset += sizeAt(i); + res_offsets.push_back(current_offset); + } + } + + return res; +} + + +ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const +{ + size_t size = getOffsets().size(); + + if (limit == 0) + limit = size; + else + limit = std::min(size, limit); + + if (perm.size() < limit) + throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + if (limit == 0) + return new ColumnArray(data); + + Permutation nested_perm(getOffsets().back()); + + ColumnArray * res_ = new ColumnArray(data->cloneEmpty()); + ColumnPtr res = res_; + + Offsets_t & res_offsets = res_->getOffsets(); + res_offsets.resize(limit); + size_t current_offset = 0; + + for (size_t i = 0; i < limit; ++i) + { + for (size_t j = 0; j < sizeAt(perm[i]); ++j) + nested_perm[current_offset + j] = offsetAt(perm[i]) + j; + current_offset += sizeAt(perm[i]); + res_offsets[i] = current_offset; + } + + if (current_offset != 0) + res_->data = data->permute(nested_perm, current_offset); + + return res; +} + + +void ColumnArray::getPermutation(bool reverse, size_t limit, Permutation & res) const +{ + size_t s = size(); + if (limit >= s) + limit = 0; + + res.resize(s); + for (size_t i = 0; i < s; ++i) + res[i] = i; + + if (limit) + { + if (reverse) + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); + else + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); + } + else + { + if (reverse) + std::sort(res.begin(), res.end(), less(*this)); + else + std::sort(res.begin(), res.end(), less(*this)); + } +} + + +ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const +{ + /// Не получается реализовать в общем случае. + + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicate(replicate_offsets); + if (typeid_cast(&*data)) return replicateString(replicate_offsets); + if (dynamic_cast(&*data)) return replicateConst(replicate_offsets); + + throw Exception("Replication of column " + getName() + " is not implemented.", ErrorCodes::NOT_IMPLEMENTED); +} + + +template +ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const +{ + size_t col_size = size(); + if (col_size != replicate_offsets.size()) + throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + ColumnPtr res = cloneEmpty(); + + if (0 == col_size) + return res; + + ColumnArray & res_ = typeid_cast(*res); + + const typename ColumnVector::Container_t & cur_data = typeid_cast &>(*data).getData(); + const Offsets_t & cur_offsets = getOffsets(); + + typename ColumnVector::Container_t & res_data = typeid_cast &>(res_.getData()).getData(); + Offsets_t & res_offsets = res_.getOffsets(); + + res_data.reserve(data->size() / col_size * replicate_offsets.back()); + res_offsets.reserve(replicate_offsets.back()); + + Offset_t prev_replicate_offset = 0; + Offset_t prev_data_offset = 0; + Offset_t current_new_offset = 0; + + for (size_t i = 0; i < col_size; ++i) + { + size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; + size_t value_size = cur_offsets[i] - prev_data_offset; + + for (size_t j = 0; j < size_to_replicate; ++j) + { + current_new_offset += value_size; + res_offsets.push_back(current_new_offset); + + res_data.resize(res_data.size() + value_size); + memcpy(&res_data[res_data.size() - value_size], &cur_data[prev_data_offset], value_size * sizeof(T)); + } + + prev_replicate_offset = replicate_offsets[i]; + prev_data_offset = cur_offsets[i]; + } + + return res; +} + + +ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) const +{ + size_t col_size = size(); + if (col_size != replicate_offsets.size()) + throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + ColumnPtr res = cloneEmpty(); + + if (0 == col_size) + return res; + + ColumnArray & res_ = typeid_cast(*res); + + const ColumnString & cur_string = typeid_cast(*data); + const ColumnString::Chars_t & cur_chars = cur_string.getChars(); + const Offsets_t & cur_string_offsets = cur_string.getOffsets(); + const Offsets_t & cur_offsets = getOffsets(); + + ColumnString::Chars_t & res_chars = typeid_cast(res_.getData()).getChars(); + Offsets_t & res_string_offsets = typeid_cast(res_.getData()).getOffsets(); + Offsets_t & res_offsets = res_.getOffsets(); + + res_chars.reserve(cur_chars.size() / col_size * replicate_offsets.back()); + res_string_offsets.reserve(cur_string_offsets.size() / col_size * replicate_offsets.back()); + res_offsets.reserve(replicate_offsets.back()); + + Offset_t prev_replicate_offset = 0; + + Offset_t prev_cur_offset = 0; + Offset_t prev_cur_string_offset = 0; + + Offset_t current_res_offset = 0; + Offset_t current_res_string_offset = 0; + + for (size_t i = 0; i < col_size; ++i) + { + /// Насколько размножить массив. + size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; + /// Количество строк в массиве. + size_t value_size = cur_offsets[i] - prev_cur_offset; + + size_t sum_chars_size = 0; + + for (size_t j = 0; j < size_to_replicate; ++j) + { + current_res_offset += value_size; + res_offsets.push_back(current_res_offset); + + sum_chars_size = 0; + + size_t prev_cur_string_offset_local = prev_cur_string_offset; + for (size_t k = 0; k < value_size; ++k) + { + /// Размер одной строки. + size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local; + + current_res_string_offset += chars_size; + res_string_offsets.push_back(current_res_string_offset); + + /// Копирование символов одной строки. + res_chars.resize(res_chars.size() + chars_size); + memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size); + + sum_chars_size += chars_size; + prev_cur_string_offset_local += chars_size; + } + } + + prev_replicate_offset = replicate_offsets[i]; + prev_cur_offset = cur_offsets[i]; + prev_cur_string_offset += sum_chars_size; + } + + return res; +} + + +ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const +{ + size_t col_size = size(); + if (col_size != replicate_offsets.size()) + throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + ColumnPtr res = cloneEmpty(); + + if (0 == col_size) + return res; + + ColumnArray & res_ = typeid_cast(*res); + + const Offsets_t & cur_offsets = getOffsets(); + Offsets_t & res_offsets = res_.getOffsets(); + + res_offsets.reserve(replicate_offsets.back()); + + Offset_t prev_replicate_offset = 0; + Offset_t prev_data_offset = 0; + Offset_t current_new_offset = 0; + + for (size_t i = 0; i < col_size; ++i) + { + size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; + size_t value_size = cur_offsets[i] - prev_data_offset; + + for (size_t j = 0; j < size_to_replicate; ++j) + { + current_new_offset += value_size; + res_offsets.push_back(current_new_offset); + } + + prev_replicate_offset = replicate_offsets[i]; + prev_data_offset = cur_offsets[i]; + } + + res_.getDataPtr() = res_.getData().cloneResized(res_offsets.back()); + + return res; +} + + +} From bbdfd33f47a971ddb650fdf3e90f5c4892d31796 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Jul 2015 23:25:50 +0300 Subject: [PATCH 23/24] dbms: fixed error with arrays; added test [#METR-17110]. --- dbms/src/Columns/ColumnArray.cpp | 14 +++++--------- ...0_non_constant_array_of_constant_data.reference | 8 ++++++++ .../00190_non_constant_array_of_constant_data.sql | 11 +++++++++++ 3 files changed, 24 insertions(+), 9 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.reference create mode 100644 dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.sql diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 2f9a7945ae7..663040eff08 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -298,16 +298,14 @@ ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const if (col_size != replicate_offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - ColumnPtr res = cloneEmpty(); - if (0 == col_size) - return res; - - ColumnArray & res_ = typeid_cast(*res); + return cloneEmpty(); const Offsets_t & cur_offsets = getOffsets(); - Offsets_t & res_offsets = res_.getOffsets(); + ColumnOffsets_t * res_column_offsets = new ColumnOffsets_t; + ColumnPtr res_column_offsets_holder = res_column_offsets; + Offsets_t & res_offsets = res_column_offsets->getData(); res_offsets.reserve(replicate_offsets.back()); Offset_t prev_replicate_offset = 0; @@ -329,9 +327,7 @@ ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const prev_data_offset = cur_offsets[i]; } - res_.getDataPtr() = res_.getData().cloneResized(res_offsets.back()); - - return res; + return new ColumnArray(getData().cloneResized(current_new_offset), res_column_offsets_holder); } diff --git a/dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.reference b/dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.reference new file mode 100644 index 00000000000..38593582115 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.reference @@ -0,0 +1,8 @@ +[''] +--- +--- +0 ['hello'] +--- +--- +0 [] +--- diff --git a/dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.sql b/dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.sql new file mode 100644 index 00000000000..2d95410ebfa --- /dev/null +++ b/dbms/tests/queries/0_stateless/00190_non_constant_array_of_constant_data.sql @@ -0,0 +1,11 @@ +SELECT arrayFilter(x -> notEmpty(concat(x, 'hello')), ['']) ARRAY JOIN [0] AS elem, arrayMap(x -> concat(x, 'hello'), ['']) AS unused WHERE NOT ignore(elem); +SELECT '---'; +SELECT arrayFilter(x -> x = 'hello', ['']) ARRAY JOIN [0] AS elem WHERE NOT ignore(elem) AND arrayExists(x -> x = 'hello', ['']); +SELECT '---'; +SELECT arrayJoin([0]), replicate('hello', [1]) WHERE NOT ignore(replicate('hello', [1])); +SELECT '---'; +SELECT arrayJoin([0]), replicate('hello', emptyArrayString()) ARRAY JOIN emptyArrayString() AS unused WHERE NOT ignore(replicate('hello', emptyArrayString())); +SELECT '---'; +SELECT arrayJoin([0]), replicate('hello', emptyArrayString()) WHERE NOT ignore(replicate('hello', emptyArrayString())); +SELECT '---'; +SELECT replicate('hello', emptyArrayString()) ARRAY JOIN emptyArrayString() AS unused WHERE NOT ignore(replicate('hello', emptyArrayString())); From ba715bdde04a33d10e541e7d3202728dbfc1bf42 Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Tue, 7 Jul 2015 12:29:29 +0300 Subject: [PATCH 24/24] dbms: writeDateTimeText pass arguments as template parameters --- dbms/include/DB/IO/WriteHelpers.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/include/DB/IO/WriteHelpers.h b/dbms/include/DB/IO/WriteHelpers.h index 4e0530ab34e..aeb7573fe4b 100644 --- a/dbms/include/DB/IO/WriteHelpers.h +++ b/dbms/include/DB/IO/WriteHelpers.h @@ -364,7 +364,8 @@ inline void writeDateText(mysqlxx::Date date, WriteBuffer & buf) /// в формате YYYY-MM-DD HH:MM:SS, согласно текущему часовому поясу -inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_delimeter = '-', char time_delimeter = ':') +template +inline void writeDateTimeText(time_t datetime, WriteBuffer & buf) { char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'}; @@ -400,7 +401,8 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, char date_deli buf.write(s, 19); } -inline void writeDateTimeText(mysqlxx::DateTime datetime, WriteBuffer & buf, char date_delimeter = '-', char time_delimeter = ':') +template +inline void writeDateTimeText(mysqlxx::DateTime datetime, WriteBuffer & buf) { char s[19] = {'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0', ' ', '0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0'};