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;