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