diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index ae7ade63bff..1dfc844a2f4 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -97,43 +97,195 @@ struct ConvertImpl } }; +/// Реализация функции toDate. -/** Преобразование даты-с-временем в дату: отбрасывание времени. - */ -template -struct ConvertImpl +namespace details { namespace { + +template class Transformation> +class Transformer { - typedef DataTypeDateTime::FieldType FromFieldType; - typedef DataTypeDate::FieldType ToFieldType; +private: + using Op = Transformation; - static void execute(Block & block, const ColumnNumbers & arguments, size_t result) +public: + static void vector_vector(const PODArray & vec_from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, PODArray & vec_to) { - const auto & date_lut = DateLUT::instance(); + ColumnString::Offset_t prev_offset = 0; - if (const ColumnVector * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + for (size_t i = 0; i < vec_from.size(); ++i) { - 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); - - for (size_t i = 0; i < size; ++i) - vec_to[i] = date_lut.toDayNum(vec_from[i]); + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + const auto & remote_date_lut = DateLUT::instance(time_zone); + vec_to[i] = Op::execute(vec_from[i], remote_date_lut); + prev_offset = cur_offset; } - else if (const ColumnConst * col_from = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) + } + + static void vector_constant(const PODArray & vec_from, const std::string & data, + PODArray & vec_to) + { + const auto & remote_date_lut = DateLUT::instance(data); + for (size_t i = 0; i < vec_from.size(); ++i) + vec_to[i] = Op::execute(vec_from[i], remote_date_lut); + } + + static void vector_constant(const PODArray & vec_from, PODArray & vec_to) + { + const auto & local_date_lut = DateLUT::instance(); + for (size_t i = 0; i < vec_from.size(); ++i) + vec_to[i] = Op::execute(vec_from[i], local_date_lut); + } + + static void constant_vector(const FromType & from, const ColumnString::Chars_t & data, + const ColumnString::Offsets_t & offsets, PODArray & vec_to) + { + ColumnString::Offset_t prev_offset = 0; + + for (size_t i = 0; i < offsets.size(); ++i) { - block.getByPosition(result).column = new ColumnConst(col_from->size(), date_lut.toDayNum(col_from->getData())); + ColumnString::Offset_t cur_offset = offsets[i]; + const std::string time_zone(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1); + const auto & remote_date_lut = DateLUT::instance(time_zone); + vec_to[i] = Op::execute(from, remote_date_lut); + prev_offset = cur_offset; } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + Name::name, - ErrorCodes::ILLEGAL_COLUMN); + } + + static void constant_constant(const FromType & from, const std::string & data, ToType & to) + { + const auto & remote_date_lut = DateLUT::instance(data); + to = Op::execute(from, remote_date_lut); + } + + static void constant_constant(const FromType & from, ToType & to) + { + const auto & local_date_lut = DateLUT::instance(); + to = Op::execute(from, local_date_lut); } }; +template class Transformation, typename Name> +class ToDateConverter +{ +private: + using FromFieldType = typename FromType::FieldType; + using ToFieldType = typename DataTypeDate::FieldType; + using Op = Transformer; + +public: + static void execute(Block & block, const ColumnNumbers & arguments, size_t result) + { + const ColumnPtr source_col = block.getByPosition(arguments[0]).column; + const auto * sources = typeid_cast *>(&*source_col); + const auto * const_source = typeid_cast *>(&*source_col); + + if (arguments.size() == 1) + { + if (sources) + { + auto * col_to = new ColumnVector; + block.getByPosition(result).column = col_to; + + const auto & vec_from = sources->getData(); + auto & vec_to = col_to->getData(); + size_t size = vec_from.size(); + vec_to.resize(size); + + Op::vector_constant(vec_from, vec_to); + } + else if (const_source) + { + ToFieldType res; + Op::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 argument of function " + Name::name, + ErrorCodes::ILLEGAL_COLUMN); + } + else if (arguments.size() == 2) + { + const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column; + const auto * time_zones = typeid_cast(&*time_zone_col); + const auto * const_time_zone = typeid_cast(&*time_zone_col); + + if (sources) + { + auto * 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 (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) + { + auto * 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 (const_time_zone) + { + 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() + + " 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("FunctionsConversion: Internal error", ErrorCodes::LOGICAL_ERROR); + } +}; + +template +struct ToDateTransform +{ + static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut) + { + return date_lut.toDayNum(from); + } +}; + +template +struct ToDateTransform32Or64 +{ + static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut) + { + return (from < 0xFFFF) ? from : date_lut.toDayNum(from); + } +}; + +}} + +/** Преобразование даты-с-временем в дату: отбрасывание времени. + */ +template struct ConvertImpl : details::ToDateConverter {}; /** Отдельный случай для преобразования (U)Int32 или (U)Int64 в Date. * Если число меньше 65536, то оно понимается, как DayNum, а если больше или равно - как unix timestamp. @@ -142,56 +294,10 @@ struct ConvertImpl * когда пользователь пишет toDate(UInt32), ожидая, что это - перевод unix timestamp в дату * (иначе такое использование было бы распространённой ошибкой). */ -template -struct ConvertImpl32Or64ToDate -{ - typedef typename FromDataType::FieldType FromFieldType; - typedef DataTypeDate::FieldType ToFieldType; - - template - static To convert(const From & from, const DateLUTImpl & date_lut) - { - return from < 0xFFFF - ? from - : date_lut.toDayNum(from); - } - - static void execute(Block & block, const ColumnNumbers & arguments, size_t result) - { - const auto & date_lut = DateLUT::instance(); - - if (const ColumnVector * col_from - = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) - { - 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); - - for (size_t i = 0; i < size; ++i) - vec_to[i] = convert(vec_from[i], date_lut); - } - else if (const ColumnConst * col_from - = typeid_cast *>(&*block.getByPosition(arguments[0]).column)) - { - block.getByPosition(result).column = new ColumnConst(col_from->size(), - convert(col_from->getData(), date_lut)); - } - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + Name::name, - ErrorCodes::ILLEGAL_COLUMN); - } -}; - -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; -template struct ConvertImpl : ConvertImpl32Or64ToDate {}; - +template struct ConvertImpl : details::ToDateConverter {}; +template struct ConvertImpl : details::ToDateConverter {}; +template struct ConvertImpl : details::ToDateConverter {}; +template struct ConvertImpl : details::ToDateConverter {}; /** Преобразование чисел, дат, дат-с-временем в строки: через форматирование. */ @@ -887,6 +993,8 @@ struct ConvertImpl } }; +/// Предварительное объявление. +struct NameToDate { static constexpr auto name = "toDate"; }; template class FunctionConvert : public IFunction @@ -934,7 +1042,9 @@ public: private: template DataTypePtr getReturnTypeImpl(const DataTypes & arguments, - typename std::enable_if::value || std::is_same::value), void>::type * = nullptr) const + typename std::enable_if::value || + std::is_same::value || + std::is_same::value)>::type * = nullptr) const { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -960,7 +1070,7 @@ private: + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - else if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + else if ((arguments.size() == 2) && (typeid_cast(&*arguments[1]) == nullptr)) { throw Exception{ "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), @@ -987,7 +1097,7 @@ private: + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - else if ((arguments.size()) == 2 && typeid_cast(&*arguments[1]) == nullptr) + else if ((arguments.size() == 2) && (typeid_cast(&*arguments[1]) == nullptr)) { throw Exception{ "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), @@ -997,6 +1107,26 @@ private: return new ToDataType2; } + + template + DataTypePtr getReturnTypeImpl(const DataTypes & arguments, + 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 " + + 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 2nd argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + return new ToDataType2; + } }; @@ -1140,7 +1270,6 @@ struct NameToInt32 { static constexpr auto name = "toInt32"; }; struct NameToInt64 { static constexpr auto name = "toInt64"; }; struct NameToFloat32 { static constexpr auto name = "toFloat32"; }; struct NameToFloat64 { static constexpr auto name = "toFloat64"; }; -struct NameToDate { static constexpr auto name = "toDate"; }; struct NameToDateTime { static constexpr auto name = "toDateTime"; }; struct NameToString { static constexpr auto name = "toString"; }; diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 9d270044810..f831f37edb4 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -402,49 +402,8 @@ public: return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const override - { - return getReturnTypeImpl(arguments); - } - - /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) override - { - IDataType * from_type = &*block.getByPosition(arguments[0]).type; - - if (typeid_cast(from_type)) - DateTimeTransformImpl::execute(block, arguments, result); - else if (typeid_cast(from_type)) - DateTimeTransformImpl::execute(block, arguments, result); - else - 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< - !(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 " - + 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< - std::is_same::value - || (std::is_same::value && std::is_same::value) - , void>::type * = nullptr) 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 " @@ -468,6 +427,20 @@ private: return new ToDataType; } + + /// Выполнить функцию над блоком. + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override + { + IDataType * from_type = &*block.getByPosition(arguments[0]).type; + + if (typeid_cast(from_type)) + DateTimeTransformImpl::execute(block, arguments, result); + else if (typeid_cast(from_type)) + DateTimeTransformImpl::execute(block, arguments, result); + else + throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } }; diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index f1d2ea9c2a7..d4b61d3da17 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -23,21 +23,186 @@ 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 +2014 +2014 +2014 +2014 +9 +9 +9 +10 +9 +30 +30 +30 +1 +30 +2 +2 +2 +3 +2 +23 +21 +20 +4 +11 +50 +50 +50 +50 +50 +0 +0 +0 +0 +0 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:50:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014-09-30 23:00:00 +2014 +2014 +2014 +2014 +2014 +24177 +24177 +24177 +24178 +24177 +2334 +2334 +2334 +2334 +2334 +16343 +16343 +16343 +16344 +16343 +392251 +392251 +392251 +392251 +392251 +23535110 +23535110 +23535110 +23535110 +23535110 +1412106600 +1412106600 +1412106600 +1412106600 +1412106600 +2014-09-30 +2014-09-30 +2014-09-30 +2014-10-01 +2014-09-30 +2014-09-30 +2014-09-30 +2014-09-30 +2014-10-01 +2014-09-30 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 7 15 3 +2014 12 28 7 +2015 1 1 4 +2014 9 30 2 +2015 3 15 7 +19 30 0 2015-07-15 13:30:00 +21 0 0 2014-12-29 00:00:00 +12 0 0 2015-01-01 12:00:00 +21 50 0 2014-09-30 23:50:00 +2 30 0 2015-03-15 13:30:00 +2015-07-15 13:00:00 2015 24187 2375 +2014-12-29 00:00:00 2014 24180 2346 +2015-01-01 12:00:00 2015 24181 2347 +2014-09-30 23:00:00 2014 24177 2334 +2015-03-15 13:00:00 2015 24183 2357 +16631 399154 23949270 1436956200 +16432 394389 23663340 1419800400 +16436 394473 23668380 1420102800 +16343 392251 23535110 1412106600 +16509 396226 23773590 1426415400 +2015-07-15 2015-07-15 +2014-12-28 2014-12-28 +2015-01-01 2015-01-01 +2014-09-30 2014-09-30 +2015-03-15 2015-03-15 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 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 7 15 3 +2014 12 29 7 +2015 1 1 4 +2014 9 1 2 +2015 3 15 7 +12 30 0 2015-07-15 13:30:00 +22 0 0 2014-12-29 00:00:00 +10 0 0 2015-01-01 12:00:00 +21 50 0 2014-09-30 23:50:00 +11 30 0 2015-03-15 13:30:00 +2015-07-15 13:00:00 2015 24187 2375 +2014-12-29 00:00:00 2014 24180 2346 +2015-01-01 12:00:00 2015 24181 2347 +2014-09-30 23:00:00 2014 24178 2334 +2015-03-15 13:00:00 2015 24183 2357 +16631 399154 23949270 1436956200 +16432 394389 23663340 1419800400 +16436 394473 23668380 1420102800 +16343 392251 23535110 1412106600 +16509 396226 23773590 1426415400 +2015-07-15 2015-07-15 +2014-12-28 2014-12-28 +2015-01-01 2015-01-01 +2014-09-30 2014-09-30 +2015-03-15 2015-03-15 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 3 15 7 +2015 3 15 7 +2015 3 15 7 +2015 3 15 7 +2015 3 15 7 +19 30 0 2015-03-15 13:30:00 +10 30 0 2015-03-15 13:30:00 +13 30 0 2015-03-15 13:30:00 +11 30 0 2015-03-15 13:30:00 +2 30 0 2015-03-15 13:30:00 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +2015-03-15 13:00:00 2015 24183 2357 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +16509 396226 23773590 1426415400 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 +2015-03-15 2015-03-15 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 fef2ef11c6b..e694d74befc 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.sql +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -46,6 +46,148 @@ SELECT toTime(toDateTime(1420102800), 'Europe/London'), toTime(toDateTime(142831 SELECT toTime(toDateTime(1420102800), 'Asia/Tokyo'), toTime(toDateTime(1428310800), 'Asia/Tokyo'); SELECT toTime(toDateTime(1420102800), 'Pacific/Pitcairn'), toTime(toDateTime(1428310800), 'Pacific/Pitcairn'); +/* toYear */ + +SELECT toYear(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toYear(toDateTime(1412106600), 'Europe/Paris'); +SELECT toYear(toDateTime(1412106600), 'Europe/London'); +SELECT toYear(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toYear(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toMonth */ + +SELECT toMonth(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toMonth(toDateTime(1412106600), 'Europe/Paris'); +SELECT toMonth(toDateTime(1412106600), 'Europe/London'); +SELECT toMonth(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toMonth(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toDayOfMonth */ + +SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Paris'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/London'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toDayOfMonth(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toDayOfWeek */ + +SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Paris'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/London'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toDayOfWeek(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toHour */ + +SELECT toHour(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toHour(toDateTime(1412106600), 'Europe/Paris'); +SELECT toHour(toDateTime(1412106600), 'Europe/London'); +SELECT toHour(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toHour(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toMinute */ + +SELECT toMinute(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toMinute(toDateTime(1412106600), 'Europe/Paris'); +SELECT toMinute(toDateTime(1412106600), 'Europe/London'); +SELECT toMinute(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toMinute(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toSecond */ + +SELECT toSecond(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toSecond(toDateTime(1412106600), 'Europe/Paris'); +SELECT toSecond(toDateTime(1412106600), 'Europe/London'); +SELECT toSecond(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toSecond(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toStartOfMinute */ + +SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/Paris'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Europe/London'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toStartOfMinute(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toStartOfHour */ + +SELECT toStartOfHour(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toStartOfHour(toDateTime(1412106600), 'Europe/Paris'); +SELECT toStartOfHour(toDateTime(1412106600), 'Europe/London'); +SELECT toStartOfHour(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toStartOfHour(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeYearNum */ + +SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeYearNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeMonthNum */ + +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeMonthNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeWeekNum */ + +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeWeekNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeDayNum */ + +SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeDayNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeHourNum */ + +SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeHourNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeMinuteNum */ + +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toRelativeSecondNum */ + +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Paris'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/London'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toRelativeSecondNum(toDateTime(1412106600), 'Pacific/Pitcairn'); + +/* toDate */ + +SELECT toDate(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toDate(toDateTime(1412106600), 'Europe/Paris'); +SELECT toDate(toDateTime(1412106600), 'Europe/London'); +SELECT toDate(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toDate(toDateTime(1412106600), 'Pacific/Pitcairn'); + +SELECT toDate(1412106600, 'Europe/Moscow'); +SELECT toDate(1412106600, 'Europe/Paris'); +SELECT toDate(1412106600, 'Europe/London'); +SELECT toDate(1412106600, 'Asia/Tokyo'); +SELECT toDate(1412106600, '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'); @@ -55,8 +197,25 @@ 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 y ASC; +SELECT toYear(toDateTime(x), y), toMonth(toDateTime(x), y), toDayOfMonth(toDateTime(x), y), toDayOfWeek(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toHour(toDateTime(x), y), toMinute(toDateTime(x), y), toSecond(toDateTime(x), y), toStartOfMinute(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toStartOfHour(toDateTime(x), y), toRelativeYearNum(toDateTime(x), y), toRelativeMonthNum(toDateTime(x), y), toRelativeWeekNum(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toRelativeDayNum(toDateTime(x), y), toRelativeHourNum(toDateTime(x), y), toRelativeMinuteNum(toDateTime(x), y), toRelativeSecondNum(toDateTime(x), y) FROM foo ORDER BY y ASC; +SELECT toDate(toDateTime(x), y), toDate(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 toYear(toDateTime(x), 'Europe/Paris'), toMonth(toDateTime(x), 'Europe/London'), toDayOfMonth(toDateTime(x), 'Asia/Tokyo'), toDayOfWeek(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toHour(toDateTime(x), 'Europe/Paris'), toMinute(toDateTime(x), 'Europe/London'), toSecond(toDateTime(x), 'Asia/Tokyo'), toStartOfMinute(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toStartOfHour(toDateTime(x), 'Europe/Paris'), toRelativeYearNum(toDateTime(x), 'Europe/London'), toRelativeMonthNum(toDateTime(x), 'Asia/Tokyo'), toRelativeWeekNum(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toRelativeDayNum(toDateTime(x), 'Europe/Paris'), toRelativeHourNum(toDateTime(x), 'Europe/London'), toRelativeMinuteNum(toDateTime(x), 'Asia/Tokyo'), toRelativeSecondNum(toDateTime(x), 'Pacific/Pitcairn') FROM foo ORDER BY y ASC; +SELECT toDate(toDateTime(x), 'Europe/Paris'), toDate(x, 'Europe/Paris') FROM foo ORDER BY y ASC; + SELECT toMonday(toDateTime(1426415400), y), toStartOfMonth(toDateTime(1426415400), y), toStartOfQuarter(toDateTime(1426415400), y), toTime(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toYear(toDateTime(1426415400), y), toMonth(toDateTime(1426415400), y), toDayOfMonth(toDateTime(1426415400), y), toDayOfWeek(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toHour(toDateTime(1426415400), y), toMinute(toDateTime(1426415400), y), toSecond(toDateTime(1426415400), y), toStartOfMinute(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toStartOfHour(toDateTime(1426415400), y), toRelativeYearNum(toDateTime(1426415400), y), toRelativeMonthNum(toDateTime(1426415400), y), toRelativeWeekNum(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toRelativeDayNum(toDateTime(1426415400), y), toRelativeHourNum(toDateTime(1426415400), y), toRelativeMinuteNum(toDateTime(1426415400), y), toRelativeSecondNum(toDateTime(1426415400), y) FROM foo ORDER BY y ASC; +SELECT toDate(toDateTime(1426415400), y), toDate(1426415400, y) FROM foo ORDER BY y ASC; /* toString */