mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
More correct time zones support; removed support for non-constant time zone argument [#CLICKHOUSE-2765].
This commit is contained in:
parent
eec6c54ce1
commit
641faac912
@ -7,24 +7,15 @@
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnTuple.h>
|
||||
#include <DB/Columns/ColumnNullable.h>
|
||||
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/Functions/Conditional/CondException.h>
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
#include <DB/Common/HashTable/ClearableHashMap.h>
|
||||
#include <DB/Functions/DataTypeTraits.h>
|
||||
#include <DB/Common/StringUtils.h>
|
||||
#include <DB/Interpreters/AggregationCommon.h>
|
||||
#include <DB/Functions/FunctionsConditional.h>
|
||||
#include <DB/Functions/FunctionsConversion.h>
|
||||
#include <DB/Functions/Conditional/getArrayType.h>
|
||||
#include <DB/Parsers/ExpressionListParsers.h>
|
||||
#include <DB/Parsers/parseQuery.h>
|
||||
#include <DB/Parsers/ASTExpressionList.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
|
||||
|
@ -28,6 +28,7 @@
|
||||
#include <DB/Interpreters/ExpressionActions.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||
#include <DB/Functions/FunctionsDateTime.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -92,236 +93,33 @@ struct ConvertImpl
|
||||
|
||||
/** Conversion of Date to DateTime: adding 00:00:00 time component.
|
||||
*/
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
||||
struct ToDateTimeImpl
|
||||
{
|
||||
using FromFieldType = DataTypeDate::FieldType;
|
||||
using ToFieldType = DataTypeDateTime::FieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
using FromFieldType = DataTypeDate::FieldType;
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(block.safeGetByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container_t & vec_from = col_from->getData();
|
||||
typename ColumnVector<ToFieldType>::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.fromDayNum(DayNum_t(vec_from[i]));
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst<FromFieldType> * col_from
|
||||
= typeid_cast<const ColumnConst<FromFieldType> *>(block.safeGetByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
block.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToFieldType>>(
|
||||
col_from->size(), date_lut.fromDayNum(DayNum_t(col_from->getData())));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
return time_zone.fromDayNum(DayNum_t(d));
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
||||
: DateTimeTransformImpl<UInt16, UInt32, ToDateTimeImpl, Name> {};
|
||||
|
||||
|
||||
/// Implementation of toDate function.
|
||||
|
||||
namespace details
|
||||
{
|
||||
|
||||
template<typename FromType, typename ToType, template <typename, typename> class Transformation>
|
||||
class Transformer
|
||||
{
|
||||
private:
|
||||
using Op = Transformation<FromType, ToType>;
|
||||
|
||||
public:
|
||||
static void vector_vector(const PaddedPODArray<FromType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PaddedPODArray<ToType> & vec_to)
|
||||
{
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Op::execute(vec_from[i], remote_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_constant(const PaddedPODArray<FromType> & vec_from, const std::string & data,
|
||||
PaddedPODArray<ToType> & 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 PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & 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, PaddedPODArray<ToType> & vec_to)
|
||||
{
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Op::execute(from, remote_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
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 <typename FromType, template <typename, typename> class Transformation, typename Name>
|
||||
class ToDateConverter
|
||||
{
|
||||
private:
|
||||
using FromFieldType = typename FromType::FieldType;
|
||||
using ToFieldType = typename DataTypeDate::FieldType;
|
||||
using Op = Transformer<FromFieldType, ToFieldType, Transformation>;
|
||||
|
||||
public:
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr source_col = block.safeGetByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnVector<FromFieldType> *>(source_col.get());
|
||||
const auto * const_source = typeid_cast<const ColumnConst<FromFieldType> *>(source_col.get());
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(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.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToFieldType>>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnPtr time_zone_col = block.safeGetByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(time_zone_col.get());
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(time_zone_col.get());
|
||||
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(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.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(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.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToFieldType>>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("FunctionsConversion: Internal error", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDateTransform
|
||||
{
|
||||
static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut)
|
||||
{
|
||||
return date_lut.toDayNum(from);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType>
|
||||
struct ToDateTransform32Or64
|
||||
{
|
||||
static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut)
|
||||
static inline ToType execute(const FromType & from, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return (from < 0xFFFF) ? from : date_lut.toDayNum(from);
|
||||
return (from < 0xFFFF) ? from : time_zone.toDayNum(from);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
/** Conversion of DateTime to Date: throw off time component.
|
||||
*/
|
||||
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
: details::ToDateConverter<DataTypeDateTime, details::ToDateTransform, Name> {};
|
||||
: DateTimeTransformImpl<UInt32, UInt16, ToDateImpl, Name> {};
|
||||
|
||||
/** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date.
|
||||
* If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp.
|
||||
@ -331,17 +129,17 @@ template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name
|
||||
* (otherwise such usage would be frequent mistake).
|
||||
*/
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name>
|
||||
: details::ToDateConverter<DataTypeUInt32, details::ToDateTransform32Or64, Name> {};
|
||||
: DateTimeTransformImpl<UInt32, UInt16, ToDateTransform32Or64<UInt32, UInt16>, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name>
|
||||
: details::ToDateConverter<DataTypeUInt64, details::ToDateTransform32Or64, Name> {};
|
||||
: DateTimeTransformImpl<UInt64, UInt16, ToDateTransform32Or64<UInt64, UInt16>, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name>
|
||||
: details::ToDateConverter<DataTypeInt32, details::ToDateTransform32Or64, Name> {};
|
||||
: DateTimeTransformImpl<Int32, UInt16, ToDateTransform32Or64<Int32, UInt16>, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name>
|
||||
: details::ToDateConverter<DataTypeInt64, details::ToDateTransform32Or64, Name> {};
|
||||
: DateTimeTransformImpl<Int64, UInt16, ToDateTransform32Or64<Int64, UInt16>, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name>
|
||||
: details::ToDateConverter<DataTypeUInt32, details::ToDateTransform32Or64, Name> {};
|
||||
: DateTimeTransformImpl<Float32, UInt16, ToDateTransform32Or64<Float32, UInt16>, Name> {};
|
||||
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name>
|
||||
: details::ToDateConverter<DataTypeUInt64, details::ToDateTransform32Or64, Name> {};
|
||||
: DateTimeTransformImpl<Float64, UInt16, ToDateTransform32Or64<Float64, UInt16>, Name> {};
|
||||
|
||||
|
||||
/** Transformation of numbers, dates, datetimes to strings: through formatting.
|
||||
@ -349,7 +147,7 @@ template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name>
|
||||
template <typename DataType>
|
||||
struct FormatImpl
|
||||
{
|
||||
static void execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType & type = DataType{})
|
||||
static void execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType * type, const DateLUTImpl * time_zone)
|
||||
{
|
||||
writeText(x, wb);
|
||||
}
|
||||
@ -358,7 +156,7 @@ struct FormatImpl
|
||||
template <>
|
||||
struct FormatImpl<DataTypeDate>
|
||||
{
|
||||
static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate & type = DataTypeDate{})
|
||||
static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate * type, const DateLUTImpl * time_zone)
|
||||
{
|
||||
writeDateText(DayNum_t(x), wb);
|
||||
}
|
||||
@ -367,18 +165,18 @@ struct FormatImpl<DataTypeDate>
|
||||
template <>
|
||||
struct FormatImpl<DataTypeDateTime>
|
||||
{
|
||||
static void execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime &type = DataTypeDateTime{})
|
||||
static void execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime * type, const DateLUTImpl * time_zone)
|
||||
{
|
||||
writeDateTimeText(x, wb);
|
||||
writeDateTimeText(x, wb, *time_zone);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename FieldType>
|
||||
struct FormatImpl<DataTypeEnum<FieldType>>
|
||||
{
|
||||
static void execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum<FieldType> & type)
|
||||
static void execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum<FieldType> * type, const DateLUTImpl * time_zone)
|
||||
{
|
||||
writeString(type.getNameForValue(x), wb);
|
||||
writeString(type->getNameForValue(x), wb);
|
||||
}
|
||||
};
|
||||
|
||||
@ -394,6 +192,10 @@ struct ConvertImpl<DataTypeEnum<FieldType>, typename DataTypeFromFieldType<Field
|
||||
};
|
||||
|
||||
|
||||
/// For functions toDateTime, toUnixTimestamp and toString from DateTime type, second argument with time zone could be specified.
|
||||
const DateLUTImpl * extractTimeZoneFromFunctionArguments(Block & block, const ColumnNumbers & arguments);
|
||||
|
||||
|
||||
template <typename FromDataType, typename Name>
|
||||
struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
{
|
||||
@ -404,6 +206,12 @@ struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
const auto & col_with_type_and_name = block.safeGetByPosition(arguments[0]);
|
||||
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
|
||||
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
|
||||
/// For argument of DateTime type, second argument with time zone could be specified.
|
||||
if (std::is_same<FromDataType, DataTypeDateTime>::value)
|
||||
time_zone = extractTimeZoneFromFunctionArguments(block, arguments);
|
||||
|
||||
if (const auto col_from = typeid_cast<const ColumnVector<FromFieldType> *>(col_with_type_and_name.column.get()))
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnString>();
|
||||
@ -420,7 +228,7 @@ struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
FormatImpl<FromDataType>::execute(vec_from[i], write_buffer, type);
|
||||
FormatImpl<FromDataType>::execute(vec_from[i], write_buffer, &type, time_zone);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
@ -431,7 +239,7 @@ struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
{
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||
FormatImpl<FromDataType>::execute(col_from->getData(), write_buffer, type);
|
||||
FormatImpl<FromDataType>::execute(col_from->getData(), write_buffer, &type, time_zone);
|
||||
block.safeGetByPosition(result).column = std::make_shared<ColumnConstString>(col_from->size(), std::string(&buf[0], write_buffer.count()));
|
||||
}
|
||||
else
|
||||
@ -491,268 +299,24 @@ struct ConvertImplGenericToString
|
||||
};
|
||||
|
||||
|
||||
namespace details { namespace {
|
||||
|
||||
/** Пусть source_timestamp представляет дату и время в исходном часовом поясе соответствующем
|
||||
* объекту from_date_lut. Эта функция возвращает timestamp представлящий те же дату и время
|
||||
* в часовом поясе соответствующем объекту to_date_lut.
|
||||
*/
|
||||
time_t convertTimestamp(time_t source_timestamp, const DateLUTImpl & from_date_lut, const DateLUTImpl & to_date_lut)
|
||||
{
|
||||
if (&from_date_lut == &to_date_lut)
|
||||
return source_timestamp;
|
||||
else
|
||||
{
|
||||
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.toHour(source_timestamp),
|
||||
from_date_lut.toMinuteInaccurate(source_timestamp),
|
||||
from_date_lut.toSecondInaccurate(source_timestamp));
|
||||
}
|
||||
}
|
||||
|
||||
/** Функции для преобразования даты + времени в строку.
|
||||
*/
|
||||
struct DateTimeToStringConverter
|
||||
{
|
||||
using FromFieldType = typename DataTypeDateTime::FieldType;
|
||||
|
||||
static void vector_vector(const PaddedPODArray<FromFieldType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, ColumnString & vec_to)
|
||||
{
|
||||
const 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<ColumnString::Chars_t> 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<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
||||
FormatImpl<DataTypeDateTime>::execute(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 PaddedPODArray<FromFieldType> & vec_from, const std::string & data,
|
||||
ColumnString & vec_to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
ColumnString::Chars_t & data_to = vec_to.getChars();
|
||||
ColumnString::Offsets_t & offsets_to = vec_to.getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
data_to.resize(size * 2);
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
||||
FormatImpl<DataTypeDateTime>::execute(ti, write_buffer);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
|
||||
static void vector_constant(const PaddedPODArray<FromFieldType> & vec_from, ColumnString & vec_to)
|
||||
{
|
||||
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<ColumnString::Chars_t> write_buffer(data_to);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
FormatImpl<DataTypeDateTime>::execute(vec_from[i], write_buffer);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
|
||||
static void constant_vector(FromFieldType from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets,
|
||||
ColumnString & vec_to)
|
||||
{
|
||||
const 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<ColumnString::Chars_t> 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<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
||||
FormatImpl<DataTypeDateTime>::execute(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(FromFieldType from, const std::string & data, std::string & to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
||||
FormatImpl<DataTypeDateTime>::execute(ti, write_buffer);
|
||||
to = std::string(&buf[0], write_buffer.count());
|
||||
}
|
||||
|
||||
static void constant_constant(FromFieldType from, std::string & to)
|
||||
{
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
||||
FormatImpl<DataTypeDateTime>::execute(from, write_buffer);
|
||||
to = std::string(&buf[0], write_buffer.count());
|
||||
}
|
||||
};
|
||||
|
||||
}}
|
||||
|
||||
template<typename Name>
|
||||
struct ConvertImpl<DataTypeDateTime, DataTypeString, Name>
|
||||
{
|
||||
using Op = details::DateTimeToStringConverter;
|
||||
using FromFieldType = Op::FromFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr source_col = block.safeGetByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnVector<FromFieldType> *>(source_col.get());
|
||||
const auto * const_source = typeid_cast<const ColumnConst<FromFieldType> *>(source_col.get());
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnString>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = *col_to;
|
||||
|
||||
Op::vector_constant(vec_from, vec_to);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
std::string res;
|
||||
Op::constant_constant(const_source->getData(), res);
|
||||
block.safeGetByPosition(result).column = std::make_shared<ColumnConstString>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnPtr time_zone_col = block.safeGetByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(time_zone_col.get());
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(time_zone_col.get());
|
||||
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnString>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getData();
|
||||
auto & vec_to = *col_to;
|
||||
|
||||
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.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnString>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
auto & vec_to = *col_to;
|
||||
|
||||
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
||||
}
|
||||
else if (const_time_zone)
|
||||
{
|
||||
std::string res;
|
||||
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||
block.safeGetByPosition(result).column = std::make_shared<ColumnConstString>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Conversion of strings to numbers, dates, datetimes: through parsing.
|
||||
*/
|
||||
template <typename DataType> void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb) { readText(x,rb); }
|
||||
template <typename DataType> void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
||||
{
|
||||
readText(x, rb);
|
||||
}
|
||||
|
||||
template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb)
|
||||
template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
||||
{
|
||||
DayNum_t tmp(0);
|
||||
readDateText(tmp, rb);
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
template <> inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb)
|
||||
template <> inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
||||
{
|
||||
time_t tmp = 0;
|
||||
readDateTimeText(tmp, rb);
|
||||
readDateTimeText(tmp, rb, *time_zone);
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
@ -770,6 +334,12 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
|
||||
/// For conversion to DateTime type, second argument with time zone could be specified.
|
||||
if (std::is_same<ToDataType, DataTypeDateTime>::value)
|
||||
time_zone = extractTimeZoneFromFunctionArguments(block, arguments);
|
||||
|
||||
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(block.safeGetByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
@ -788,7 +358,7 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
{
|
||||
ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1);
|
||||
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer);
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer, time_zone);
|
||||
|
||||
if (!read_buffer.eof()
|
||||
&& !(std::is_same<ToDataType, DataTypeDate>::value /// Special exception, that allows to parse string with DateTime as Date.
|
||||
@ -803,7 +373,7 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
const String & s = col_from->getData();
|
||||
ReadBufferFromString read_buffer(s);
|
||||
ToFieldType x = 0;
|
||||
parseImpl<ToDataType>(x, read_buffer);
|
||||
parseImpl<ToDataType>(x, read_buffer, time_zone);
|
||||
|
||||
if (!read_buffer.eof()
|
||||
&& !(std::is_same<ToDataType, DataTypeDate>::value /// Special exception, that allows to parse string with DateTime as Date.
|
||||
@ -814,7 +384,7 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
@ -945,226 +515,13 @@ struct ConvertImplGenericFromString
|
||||
};
|
||||
|
||||
|
||||
namespace details { namespace {
|
||||
|
||||
/** Conversion of strings to timestamp. It allows optional second parameter - time zone.
|
||||
*/
|
||||
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, PaddedPODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
ReadBufferFromMemory read_buffer(&vec_from[0], vec_from.size());
|
||||
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
char zero = 0;
|
||||
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||
{
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
auto ti = convertTimestamp(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,
|
||||
PaddedPODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
ReadBufferFromMemory read_buffer(&vec_from[0], vec_from.size());
|
||||
|
||||
char zero = 0;
|
||||
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||
{
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
|
||||
auto ti = convertTimestamp(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 vector_constant(const ColumnString::Chars_t & vec_from, PaddedPODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
ReadBufferFromMemory read_buffer(&vec_from[0], vec_from.size());
|
||||
|
||||
char zero = 0;
|
||||
for (size_t i = 0; i < vec_to.size(); ++i)
|
||||
{
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
vec_to[i] = x;
|
||||
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, PaddedPODArray<ToFieldType> & vec_to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
|
||||
ReadBufferFromString read_buffer(from);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(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<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
|
||||
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, ToFieldType & to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
|
||||
ReadBufferFromString read_buffer(from);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
|
||||
to = convertTimestamp(x, local_date_lut, remote_date_lut);
|
||||
}
|
||||
|
||||
static void constant_constant(const std::string & from, ToFieldType & to)
|
||||
{
|
||||
ReadBufferFromString read_buffer(from);
|
||||
DataTypeDateTime::FieldType x = 0;
|
||||
parseImpl<DataTypeDateTime>(x, read_buffer);
|
||||
to = x;
|
||||
}
|
||||
};
|
||||
|
||||
}}
|
||||
|
||||
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
|
||||
/// Note that time zone argument could be passed only for toUnixTimestamp function.
|
||||
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
||||
|
||||
template <>
|
||||
struct ConvertImpl<DataTypeString, DataTypeInt32, NameToUnixTimestamp>
|
||||
{
|
||||
using Op = details::StringToTimestampConverter;
|
||||
using ToFieldType = Op::ToFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr source_col = block.safeGetByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnString *>(source_col.get());
|
||||
const auto * const_source = typeid_cast<const ColumnConstString *>(source_col.get());
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
|
||||
auto & vec_from = sources->getChars();
|
||||
auto & vec_to = col_to->getData();
|
||||
size_t size = sources->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.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToFieldType>>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnPtr time_zone_col = block.safeGetByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(time_zone_col.get());
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(time_zone_col.get());
|
||||
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(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)
|
||||
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.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(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.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToFieldType>>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + NameToUnixTimestamp::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
};
|
||||
: ConvertImpl<DataTypeString, DataTypeDateTime, NameToUnixTimestamp> {};
|
||||
|
||||
|
||||
/** If types are identical, just take reference to column.
|
||||
@ -1190,6 +547,12 @@ struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
||||
{
|
||||
if (const ColumnFixedString * col_from = typeid_cast<const ColumnFixedString *>(block.safeGetByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
|
||||
/// For conversion to DateTime type, second argument with time zone could be specified.
|
||||
if (std::is_same<ToDataType, DataTypeDateTime>::value)
|
||||
time_zone = extractTimeZoneFromFunctionArguments(block, arguments);
|
||||
|
||||
auto col_to = std::make_shared<ColumnVector<ToFieldType>>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
|
||||
@ -1203,7 +566,7 @@ struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
||||
{
|
||||
ReadBufferFromMemory read_buffer(&data_from[i * n], n);
|
||||
const char * end = read_buffer.buffer().end();
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer);
|
||||
parseImpl<ToDataType>(vec_to[i], read_buffer, time_zone);
|
||||
|
||||
if (!read_buffer.eof())
|
||||
{
|
||||
|
@ -64,17 +64,17 @@ namespace DB
|
||||
/// Это фактор-преобразование будет говорить, что функция монотонна всюду.
|
||||
struct ZeroTransform
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return 0; }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return 0; }
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return 0; }
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return 0; }
|
||||
};
|
||||
|
||||
struct ToDateImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toDate(t);
|
||||
return time_zone.toDate(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return d;
|
||||
}
|
||||
@ -84,11 +84,11 @@ struct ToDateImpl
|
||||
|
||||
struct ToStartOfDayImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toDate(t);
|
||||
return time_zone.toDate(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return d;
|
||||
}
|
||||
@ -98,13 +98,13 @@ struct ToStartOfDayImpl
|
||||
|
||||
struct ToMondayImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t));
|
||||
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfWeek(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -112,13 +112,13 @@ struct ToMondayImpl
|
||||
|
||||
struct ToStartOfMonthImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t));
|
||||
return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfMonth(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -126,13 +126,13 @@ struct ToStartOfMonthImpl
|
||||
|
||||
struct ToStartOfQuarterImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t));
|
||||
return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfQuarter(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -140,13 +140,13 @@ struct ToStartOfQuarterImpl
|
||||
|
||||
struct ToStartOfYearImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t));
|
||||
return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d));
|
||||
return time_zone.toFirstDayNumOfYear(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -156,23 +156,12 @@ struct ToStartOfYearImpl
|
||||
struct ToTimeImpl
|
||||
{
|
||||
/// При переводе во время, дату будем приравнивать к 1970-01-02.
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
time_t remote_ts = remote_date_lut.toTime(t) + 86400;
|
||||
|
||||
if (&remote_date_lut == &local_date_lut)
|
||||
return remote_ts;
|
||||
else
|
||||
{
|
||||
const auto & values = remote_date_lut.getValues(remote_ts);
|
||||
return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||
remote_date_lut.toHour(remote_ts),
|
||||
remote_date_lut.toMinuteInaccurate(remote_ts),
|
||||
remote_date_lut.toSecondInaccurate(remote_ts));
|
||||
}
|
||||
return time_zone.toTime(t) + 86400;
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -182,23 +171,11 @@ struct ToTimeImpl
|
||||
|
||||
struct ToStartOfMinuteImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
if (&remote_date_lut == &local_date_lut)
|
||||
return local_date_lut.toStartOfMinuteInaccurate(t);
|
||||
else
|
||||
{
|
||||
time_t remote_ts = remote_date_lut.toTime(t) + 86400;
|
||||
remote_ts = remote_date_lut.toStartOfMinuteInaccurate(remote_ts);
|
||||
|
||||
const auto & values = remote_date_lut.getValues(t);
|
||||
return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||
remote_date_lut.toHour(remote_ts),
|
||||
remote_date_lut.toMinuteInaccurate(remote_ts),
|
||||
remote_date_lut.toSecondInaccurate(remote_ts));
|
||||
}
|
||||
return time_zone.toStartOfMinuteInaccurate(t);
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -208,23 +185,11 @@ struct ToStartOfMinuteImpl
|
||||
|
||||
struct ToStartOfFiveMinuteImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
if (&remote_date_lut == &local_date_lut)
|
||||
return local_date_lut.toStartOfFiveMinuteInaccurate(t);
|
||||
else
|
||||
{
|
||||
time_t remote_ts = remote_date_lut.toTime(t) + 86400;
|
||||
remote_ts = remote_date_lut.toStartOfFiveMinuteInaccurate(remote_ts);
|
||||
|
||||
const auto & values = remote_date_lut.getValues(t);
|
||||
return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||
remote_date_lut.toHour(remote_ts),
|
||||
remote_date_lut.toMinuteInaccurate(remote_ts),
|
||||
remote_date_lut.toSecondInaccurate(remote_ts));
|
||||
}
|
||||
return time_zone.toStartOfFiveMinuteInaccurate(t);
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -234,23 +199,11 @@ struct ToStartOfFiveMinuteImpl
|
||||
|
||||
struct ToStartOfHourImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
if (&remote_date_lut == &local_date_lut)
|
||||
return local_date_lut.toStartOfHourInaccurate(t);
|
||||
else
|
||||
{
|
||||
time_t remote_ts = remote_date_lut.toTime(t) + 86400;
|
||||
remote_ts = remote_date_lut.toStartOfHourInaccurate(remote_ts);
|
||||
|
||||
const auto & values = remote_date_lut.getValues(t);
|
||||
return local_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
||||
remote_date_lut.toHour(remote_ts),
|
||||
remote_date_lut.toMinuteInaccurate(remote_ts),
|
||||
remote_date_lut.toSecondInaccurate(remote_ts));
|
||||
}
|
||||
return time_zone.toStartOfHourInaccurate(t);
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -260,13 +213,13 @@ struct ToStartOfHourImpl
|
||||
|
||||
struct ToYearImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toYear(t);
|
||||
return time_zone.toYear(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toYear(DayNum_t(d));
|
||||
return time_zone.toYear(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -274,13 +227,13 @@ struct ToYearImpl
|
||||
|
||||
struct ToMonthImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toMonth(t);
|
||||
return time_zone.toMonth(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toMonth(DayNum_t(d));
|
||||
return time_zone.toMonth(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
@ -288,13 +241,13 @@ struct ToMonthImpl
|
||||
|
||||
struct ToDayOfMonthImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toDayOfMonth(t);
|
||||
return time_zone.toDayOfMonth(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toDayOfMonth(DayNum_t(d));
|
||||
return time_zone.toDayOfMonth(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfMonthImpl;
|
||||
@ -302,13 +255,13 @@ struct ToDayOfMonthImpl
|
||||
|
||||
struct ToDayOfWeekImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toDayOfWeek(t);
|
||||
return time_zone.toDayOfWeek(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toDayOfWeek(DayNum_t(d));
|
||||
return time_zone.toDayOfWeek(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ToMondayImpl;
|
||||
@ -316,11 +269,11 @@ struct ToDayOfWeekImpl
|
||||
|
||||
struct ToHourImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toHour(t);
|
||||
return time_zone.toHour(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -330,11 +283,11 @@ struct ToHourImpl
|
||||
|
||||
struct ToMinuteImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toMinuteInaccurate(t);
|
||||
return time_zone.toMinuteInaccurate(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -344,11 +297,11 @@ struct ToMinuteImpl
|
||||
|
||||
struct ToSecondImpl
|
||||
{
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toSecondInaccurate(t);
|
||||
return time_zone.toSecondInaccurate(t);
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -358,13 +311,13 @@ struct ToSecondImpl
|
||||
|
||||
struct ToRelativeYearNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toYear(t);
|
||||
return time_zone.toYear(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toYear(DayNum_t(d));
|
||||
return time_zone.toYear(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -372,13 +325,13 @@ struct ToRelativeYearNumImpl
|
||||
|
||||
struct ToRelativeMonthNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toRelativeMonthNum(t);
|
||||
return time_zone.toRelativeMonthNum(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toRelativeMonthNum(DayNum_t(d));
|
||||
return time_zone.toRelativeMonthNum(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -386,13 +339,13 @@ struct ToRelativeMonthNumImpl
|
||||
|
||||
struct ToRelativeWeekNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toRelativeWeekNum(t);
|
||||
return time_zone.toRelativeWeekNum(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toRelativeWeekNum(DayNum_t(d));
|
||||
return time_zone.toRelativeWeekNum(DayNum_t(d));
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
@ -400,11 +353,11 @@ struct ToRelativeWeekNumImpl
|
||||
|
||||
struct ToRelativeDayNumImpl
|
||||
{
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toDayNum(t);
|
||||
return time_zone.toDayNum(t);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return static_cast<DayNum_t>(d);
|
||||
}
|
||||
@ -415,11 +368,11 @@ struct ToRelativeDayNumImpl
|
||||
|
||||
struct ToRelativeHourNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toRelativeHourNum(t);
|
||||
return time_zone.toRelativeHourNum(t);
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -429,11 +382,11 @@ struct ToRelativeHourNumImpl
|
||||
|
||||
struct ToRelativeMinuteNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return remote_date_lut.toRelativeMinuteNum(t);
|
||||
return time_zone.toRelativeMinuteNum(t);
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -443,11 +396,11 @@ struct ToRelativeMinuteNumImpl
|
||||
|
||||
struct ToRelativeSecondNumImpl
|
||||
{
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return t;
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -459,68 +412,22 @@ struct ToRelativeSecondNumImpl
|
||||
template<typename FromType, typename ToType, typename Transform>
|
||||
struct Transformer
|
||||
{
|
||||
static void vector_vector(const PaddedPODArray<FromType> & vec_from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PaddedPODArray<ToType> & vec_to)
|
||||
static void vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, const DateLUTImpl & time_zone)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
size_t size = vec_from.size();
|
||||
vec_to.resize(size);
|
||||
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], time_zone);
|
||||
}
|
||||
|
||||
static void vector_constant(const PaddedPODArray<FromType> & vec_from, const std::string & data,
|
||||
PaddedPODArray<ToType> & vec_to)
|
||||
static void constant(const FromType & from, ToType & to, const DateLUTImpl & time_zone)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
for (size_t i = 0; i < vec_from.size(); ++i)
|
||||
vec_to[i] = Transform::execute(vec_from[i], remote_date_lut, local_date_lut);
|
||||
}
|
||||
|
||||
static void vector_constant(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to)
|
||||
{
|
||||
const 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_vector(const FromType & from, const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets_t & offsets, PaddedPODArray<ToType> & vec_to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
ColumnString::Offset_t prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
ColumnString::Offset_t cur_offset = offsets[i];
|
||||
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
||||
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
||||
vec_to[i] = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
static void constant_constant(const FromType & from, const std::string & data, ToType & to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
const auto & remote_date_lut = DateLUT::instance(data);
|
||||
to = Transform::execute(from, remote_date_lut, local_date_lut);
|
||||
}
|
||||
|
||||
static void constant_constant(const FromType & from, ToType & to)
|
||||
{
|
||||
const auto & local_date_lut = DateLUT::instance();
|
||||
to = Transform::execute(from, local_date_lut, local_date_lut);
|
||||
to = Transform::execute(from, time_zone);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform, typename Name>
|
||||
struct DateTimeTransformImpl
|
||||
{
|
||||
@ -529,93 +436,47 @@ struct DateTimeTransformImpl
|
||||
using Op = Transformer<FromType, ToType, Transform>;
|
||||
|
||||
const ColumnPtr source_col = block.safeGetByPosition(arguments[0]).column;
|
||||
const auto * sources = typeid_cast<const ColumnVector<FromType> *>(&*source_col);
|
||||
const auto * const_source = typeid_cast<const ColumnConst<FromType> *>(&*source_col);
|
||||
const auto * sources = typeid_cast<const ColumnVector<FromType> *>(source_col.get());
|
||||
const auto * const_source = typeid_cast<const ColumnConst<FromType> *>(source_col.get());
|
||||
|
||||
if (arguments.size() == 1)
|
||||
const ColumnConstString * time_zone_column = nullptr;
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToType>>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
time_zone_column = typeid_cast<const ColumnConstString *>(block.safeGetByPosition(arguments[1]).column.get());
|
||||
|
||||
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)
|
||||
{
|
||||
ToType res;
|
||||
Op::constant_constant(const_source->getData(), res);
|
||||
block.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToType>>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
if (!time_zone_column)
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second (time zone) argument of function " + Name::name + ", must be constant string",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
|
||||
const DateLUTImpl & time_zone = time_zone_column
|
||||
? DateLUT::instance(time_zone_column->getData())
|
||||
: DateLUT::instance();
|
||||
|
||||
if (sources)
|
||||
{
|
||||
const ColumnPtr time_zone_col = block.safeGetByPosition(arguments[1]).column;
|
||||
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
||||
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
||||
|
||||
if (sources)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToType>>();
|
||||
block.safeGetByPosition(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.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
if (time_zones)
|
||||
{
|
||||
auto col_to = std::make_shared<ColumnVector<ToType>>();
|
||||
block.safeGetByPosition(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)
|
||||
{
|
||||
ToType res;
|
||||
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
||||
block.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToType>>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
auto col_to = std::make_shared<ColumnVector<ToType>>();
|
||||
block.safeGetByPosition(result).column = col_to;
|
||||
Op::vector(sources->getData(), col_to->getData(), time_zone);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
ToType res;
|
||||
Op::constant(const_source->getData(), res, time_zone);
|
||||
block.safeGetByPosition(result).column = std::make_shared<ColumnConst<ToType>>(const_source->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
||||
{
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename ToDataType, typename Transform, typename Name>
|
||||
class FunctionDateOrDateTimeToSomething : public IFunction
|
||||
{
|
||||
@ -637,25 +498,23 @@ public:
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if ((typeid_cast<const DataTypeDate *>(&*arguments[0]) == nullptr) &&
|
||||
(typeid_cast<const DataTypeDateTime *>(&*arguments[0]) == nullptr))
|
||||
if (!typeid_cast<const DataTypeDate *>(arguments[0].get())
|
||||
&& !typeid_cast<const DataTypeDateTime *>(arguments[0].get()))
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName() +
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if ((typeid_cast<const DataTypeDate *>(&*arguments[0]) != nullptr)
|
||||
|| (typeid_cast<const DataTypeDateTime *>(&*arguments[0]) == nullptr)
|
||||
|| (typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr))
|
||||
if (typeid_cast<const DataTypeDate *>(arguments[0].get())
|
||||
|| typeid_cast<const DataTypeDateTime *>(arguments[0].get())
|
||||
|| typeid_cast<const DataTypeString *>(arguments[1].get()))
|
||||
throw Exception{
|
||||
"Function " + getName() + " supports 1 or 2 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with timezone name. The timezone argument is allowed "
|
||||
"only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
else
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -703,14 +562,14 @@ public:
|
||||
|
||||
if (typeid_cast<const DataTypeDate *>(&type))
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), date_lut, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
}
|
||||
else
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), date_lut, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
}
|
||||
}
|
||||
@ -825,7 +684,7 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!typeid_cast<const DataTypeDateTime *>(&*arguments[0]))
|
||||
if (!typeid_cast<const DataTypeDateTime *>(arguments[0].get()))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -958,11 +817,11 @@ public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!typeid_cast<const DataTypeDateTime *>(&*arguments[0]))
|
||||
if (!typeid_cast<const DataTypeDateTime *>(arguments[0].get()))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!typeid_cast<const DataTypeUInt32 *>(&*arguments[1]))
|
||||
if (!typeid_cast<const DataTypeUInt32 *>(arguments[1].get()))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be UInt32.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -585,12 +585,12 @@ template <typename T>
|
||||
inline T parse(const char * data, size_t size);
|
||||
|
||||
|
||||
void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf);
|
||||
void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut);
|
||||
|
||||
/** In YYYY-MM-DD hh:mm:ss format, according to current time zone.
|
||||
* As an exception, also supported parsing of unix timestamp in form of decimal number.
|
||||
*/
|
||||
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf)
|
||||
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
|
||||
{
|
||||
/** Read 10 characters, that could represent unix timestamp.
|
||||
* Only unix timestamp of 5-10 characters is supported.
|
||||
@ -615,7 +615,7 @@ inline void readDateTimeText(time_t & datetime, ReadBuffer & buf)
|
||||
if (unlikely(year == 0))
|
||||
datetime = 0;
|
||||
else
|
||||
datetime = DateLUT::instance().makeDateTime(year, month, day, hour, minute, second);
|
||||
datetime = date_lut.makeDateTime(year, month, day, hour, minute, second);
|
||||
|
||||
buf.position() += 19;
|
||||
}
|
||||
@ -624,7 +624,7 @@ inline void readDateTimeText(time_t & datetime, ReadBuffer & buf)
|
||||
readIntText(datetime, buf);
|
||||
}
|
||||
else
|
||||
readDateTimeTextFallback(datetime, buf);
|
||||
readDateTimeTextFallback(datetime, buf, date_lut);
|
||||
}
|
||||
|
||||
inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf)
|
||||
|
@ -518,7 +518,7 @@ inline void writeDateText(LocalDate date, WriteBuffer & buf)
|
||||
|
||||
/// в формате YYYY-MM-DD HH:MM:SS, согласно текущему часовому поясу
|
||||
template <char date_delimeter = '-', char time_delimeter = ':'>
|
||||
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf)
|
||||
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
|
||||
{
|
||||
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'};
|
||||
|
||||
@ -528,7 +528,6 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf)
|
||||
return;
|
||||
}
|
||||
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
const auto & values = date_lut.getValues(datetime);
|
||||
|
||||
s[0] += values.year / 1000;
|
||||
|
@ -3,10 +3,26 @@
|
||||
|
||||
#include <DB/Functions/FunctionFactory.h>
|
||||
#include <DB/Functions/FunctionsArray.h>
|
||||
#include <DB/Functions/FunctionsConversion.h>
|
||||
#include <DB/Functions/Conditional/getArrayType.h>
|
||||
#include <DB/Functions/Conditional/CondException.h>
|
||||
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
#include <DB/Common/HashTable/ClearableHashMap.h>
|
||||
|
||||
#include <DB/Parsers/ExpressionListParsers.h>
|
||||
#include <DB/Parsers/parseQuery.h>
|
||||
#include <DB/Parsers/ASTExpressionList.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
|
||||
#include <DB/Interpreters/AggregationCommon.h>
|
||||
|
||||
#include <DB/Columns/ColumnTuple.h>
|
||||
|
||||
#include <tuple>
|
||||
#include <array>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -4,6 +4,24 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
const DateLUTImpl * extractTimeZoneFromFunctionArguments(Block & block, const ColumnNumbers & arguments)
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConstString * time_zone_column = typeid_cast<const ColumnConstString *>(block.safeGetByPosition(arguments[1]).column.get());
|
||||
|
||||
if (!time_zone_column)
|
||||
throw Exception("Illegal column " + block.safeGetByPosition(arguments[1]).column->getName()
|
||||
+ " of second (time zone) argument of function, must be constant string",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return &DateLUT::instance(time_zone_column->getData());
|
||||
}
|
||||
else
|
||||
return &DateLUT::instance();
|
||||
}
|
||||
|
||||
|
||||
void throwExceptionForIncompletelyParsedValue(
|
||||
ReadBuffer & read_buffer, Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
|
@ -562,7 +562,7 @@ template void readJSONStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> &
|
||||
template void readJSONStringInto<NullSink>(NullSink & s, ReadBuffer & buf);
|
||||
|
||||
|
||||
void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf)
|
||||
void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut)
|
||||
{
|
||||
static constexpr auto DATE_TIME_BROKEN_DOWN_LENGTH = 19;
|
||||
static constexpr auto UNIX_TIMESTAMP_MAX_LENGTH = 10;
|
||||
@ -600,7 +600,7 @@ void readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf)
|
||||
if (unlikely(year == 0))
|
||||
datetime = 0;
|
||||
else
|
||||
datetime = DateLUT::instance().makeDateTime(year, month, day, hour, minute, second);
|
||||
datetime = date_lut.makeDateTime(year, month, day, hour, minute, second);
|
||||
}
|
||||
else
|
||||
datetime = parse<time_t>(s, s_pos - s);
|
||||
|
@ -113,116 +113,11 @@
|
||||
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 19:30:00
|
||||
21 0 0 2014-12-28 21:00:00
|
||||
12 0 0 2015-01-01 12:00:00
|
||||
21 50 0 2014-09-30 21:50:00
|
||||
2 30 0 2015-03-15 02:00:00
|
||||
2015-07-15 19:00:00 2015 24187 2375
|
||||
2014-12-28 21:00:00 2014 24180 2346
|
||||
2015-01-01 12:00:00 2015 24181 2347
|
||||
2014-09-30 21:00:00 2014 24177 2334
|
||||
2015-03-15 02: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 02:00:00
|
||||
22 0 0 2014-12-28 13:30:00
|
||||
10 0 0 2015-01-01 01:30:00
|
||||
21 50 0 2014-09-30 11:20:00
|
||||
11 30 0 2015-03-15 02:00:00
|
||||
2015-07-15 12:00:00 2015 24187 2375
|
||||
2014-12-28 22:00:00 2014 24180 2346
|
||||
2015-01-01 10:00:00 2015 24181 2347
|
||||
2014-09-30 21:00:00 2014 24178 2334
|
||||
2015-03-15 11: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 19:30:00
|
||||
10 30 0 2015-03-15 10:30:00
|
||||
13 30 0 2015-03-15 13:30:00
|
||||
11 30 0 2015-03-15 11:30:00
|
||||
2 30 0 2015-03-15 02:00:00
|
||||
2015-03-15 19:00:00 2015 24183 2357
|
||||
2015-03-15 10:00:00 2015 24183 2357
|
||||
2015-03-15 13:00:00 2015 24183 2357
|
||||
2015-03-15 11:00:00 2015 24183 2357
|
||||
2015-03-15 02: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
|
||||
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
|
||||
@ -233,18 +128,3 @@
|
||||
1426415400
|
||||
1426415400
|
||||
1426415400
|
||||
1436934600
|
||||
1419811200
|
||||
1420102800
|
||||
1412113800
|
||||
1426455000
|
||||
1426393800
|
||||
1426426200
|
||||
1426415400
|
||||
1426422600
|
||||
1426455000
|
||||
1412113800
|
||||
1419807600
|
||||
1420110000
|
||||
1426422600
|
||||
1436959800
|
||||
|
@ -188,35 +188,6 @@ 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');
|
||||
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 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 */
|
||||
|
||||
SELECT toString(toDateTime(1436956200), 'Europe/Moscow');
|
||||
@ -225,10 +196,6 @@ 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'), 'Europe/Moscow');
|
||||
@ -242,7 +209,3 @@ 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), 'Europe/Moscow'), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Moscow'), y) FROM foo ORDER BY y ASC;
|
||||
SELECT toUnixTimestamp(toString(toDateTime(x), 'Europe/Moscow'), 'Europe/Paris') FROM foo ORDER BY x ASC;
|
||||
|
Loading…
Reference in New Issue
Block a user