2011-10-15 23:40:56 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
2016-01-13 00:32:59 +00:00
|
|
|
|
#include <ext/enumerate.hpp>
|
|
|
|
|
#include <ext/collection_cast.hpp>
|
|
|
|
|
#include <ext/range.hpp>
|
|
|
|
|
#include <type_traits>
|
|
|
|
|
|
2011-10-16 01:57:10 +00:00
|
|
|
|
#include <DB/IO/WriteBufferFromVector.h>
|
2011-11-29 18:53:44 +00:00
|
|
|
|
#include <DB/IO/ReadBufferFromString.h>
|
2015-12-23 08:11:11 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeFactory.h>
|
2011-10-15 23:40:56 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeString.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeFixedString.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeDate.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
2015-12-22 13:07:40 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeEnum.h>
|
2016-01-13 00:32:59 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeArray.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeTuple.h>
|
2011-10-15 23:40:56 +00:00
|
|
|
|
#include <DB/Columns/ColumnString.h>
|
|
|
|
|
#include <DB/Columns/ColumnFixedString.h>
|
|
|
|
|
#include <DB/Columns/ColumnConst.h>
|
2016-01-13 00:32:59 +00:00
|
|
|
|
#include <DB/Columns/ColumnArray.h>
|
2015-11-29 08:06:29 +00:00
|
|
|
|
#include <DB/Core/FieldVisitors.h>
|
2015-12-23 08:11:11 +00:00
|
|
|
|
#include <DB/Interpreters/ExpressionActions.h>
|
2016-01-13 00:32:59 +00:00
|
|
|
|
#include <DB/Functions/IFunction.h>
|
2015-12-29 12:57:11 +00:00
|
|
|
|
#include <DB/Functions/FunctionsMiscellaneous.h>
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2016-01-12 02:21:15 +00:00
|
|
|
|
namespace ErrorCodes
|
|
|
|
|
{
|
|
|
|
|
extern const int CANNOT_PARSE_NUMBER;
|
|
|
|
|
}
|
|
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
|
/** Функции преобразования типов.
|
2014-04-13 13:58:54 +00:00
|
|
|
|
* toType - преобразование "естественным образом";
|
2011-10-15 23:40:56 +00:00
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
|
2015-12-22 13:07:40 +00:00
|
|
|
|
/** Преобразование чисел друг в друга, перечислений в числа, дат/дат-с-временем в числа и наоборот: делается обычным присваиванием.
|
2011-10-15 23:40:56 +00:00
|
|
|
|
* (дата внутри хранится как количество дней с какого-то, дата-с-временем - как unix timestamp)
|
|
|
|
|
*/
|
2011-10-16 01:57:10 +00:00
|
|
|
|
template <typename FromDataType, typename ToDataType, typename Name>
|
|
|
|
|
struct ConvertImpl
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2011-10-16 01:57:10 +00:00
|
|
|
|
typedef typename FromDataType::FieldType FromFieldType;
|
2011-10-15 23:40:56 +00:00
|
|
|
|
typedef typename ToDataType::FieldType ToFieldType;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2011-10-16 01:57:10 +00:00
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2016-01-13 00:32:59 +00:00
|
|
|
|
if (const ColumnVector<FromFieldType> * col_from
|
|
|
|
|
= typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
|
|
|
|
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] = vec_from[i];
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const ColumnConst<FromFieldType> * col_from = typeid_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
|
|
|
|
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), col_from->getData());
|
|
|
|
|
}
|
|
|
|
|
else
|
2011-10-16 01:57:10 +00:00
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2014-11-12 17:23:26 +00:00
|
|
|
|
+ " of first argument of function " + Name::name,
|
2011-10-16 01:57:10 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|
2011-10-16 01:57:10 +00:00
|
|
|
|
};
|
|
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
2011-10-16 01:57:10 +00:00
|
|
|
|
/** Преобразование даты в дату-с-временем: добавление нулевого времени.
|
|
|
|
|
*/
|
|
|
|
|
template <typename Name>
|
|
|
|
|
struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
|
|
|
|
{
|
|
|
|
|
typedef DataTypeDate::FieldType FromFieldType;
|
|
|
|
|
typedef DataTypeDateTime::FieldType ToFieldType;
|
|
|
|
|
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
|
|
|
|
typedef DataTypeDate::FieldType FromFieldType;
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & date_lut = DateLUT::instance();
|
2011-10-16 01:57:10 +00:00
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const ColumnVector<FromFieldType> * col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
|
|
|
|
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)
|
2013-01-10 09:24:30 +00:00
|
|
|
|
{
|
2013-08-11 03:40:14 +00:00
|
|
|
|
vec_to[i] = date_lut.fromDayNum(DayNum_t(vec_from[i]));
|
2013-01-10 09:24:30 +00:00
|
|
|
|
}
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const ColumnConst<FromFieldType> * col_from = typeid_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2013-08-11 03:40:14 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), date_lut.fromDayNum(DayNum_t(col_from->getData())));
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2014-11-12 17:23:26 +00:00
|
|
|
|
+ " of first argument of function " + Name::name,
|
2011-10-15 23:40:56 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
2011-10-16 01:57:10 +00:00
|
|
|
|
};
|
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
/// Реализация функции toDate.
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
2015-12-23 08:11:11 +00:00
|
|
|
|
namespace details {
|
2015-10-22 15:31:42 +00:00
|
|
|
|
|
|
|
|
|
template<typename FromType, typename ToType, template <typename, typename> class Transformation>
|
|
|
|
|
class Transformer
|
2011-10-16 01:57:10 +00:00
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
private:
|
|
|
|
|
using Op = Transformation<FromType, ToType>;
|
2011-10-16 01:57:10 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
public:
|
|
|
|
|
static void vector_vector(const PODArray<FromType> & vec_from, const ColumnString::Chars_t & data,
|
|
|
|
|
const ColumnString::Offsets_t & offsets, PODArray<ToType> & vec_to)
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
ColumnString::Offset_t prev_offset = 0;
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
for (size_t i = 0; i < vec_from.size(); ++i)
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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;
|
|
|
|
|
}
|
|
|
|
|
}
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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);
|
|
|
|
|
}
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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)
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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;
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2011-10-16 01:57:10 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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);
|
|
|
|
|
}
|
2015-03-16 04:44:53 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
static void constant_constant(const FromType & from, ToType & to)
|
2015-03-16 04:44:53 +00:00
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
|
|
|
|
to = Op::execute(from, local_date_lut);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
}
|
2015-10-22 15:31:42 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
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>;
|
2015-03-16 04:44:53 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
public:
|
2015-03-16 04:44:53 +00:00
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
if (arguments.size() == 1)
|
2015-03-16 04:44:53 +00:00
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
if (sources)
|
|
|
|
|
{
|
|
|
|
|
auto * col_to = new ColumnVector<ToFieldType>;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
2015-03-16 04:44:53 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
const auto & vec_from = sources->getData();
|
|
|
|
|
auto & vec_to = col_to->getData();
|
|
|
|
|
size_t size = vec_from.size();
|
|
|
|
|
vec_to.resize(size);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
}
|
2015-10-22 15:31:42 +00:00
|
|
|
|
else if (arguments.size() == 2)
|
2015-03-16 04:44:53 +00:00
|
|
|
|
{
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
2015-10-22 15:31:42 +00:00
|
|
|
|
throw Exception("FunctionsConversion: Internal error", ErrorCodes::LOGICAL_ERROR);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
template <typename FromType, typename ToType>
|
|
|
|
|
struct ToDateTransform
|
|
|
|
|
{
|
|
|
|
|
static inline ToType execute(const FromType & from, const DateLUTImpl & date_lut)
|
|
|
|
|
{
|
|
|
|
|
return date_lut.toDayNum(from);
|
|
|
|
|
}
|
|
|
|
|
};
|
2015-03-16 04:44:53 +00:00
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-12-23 08:11:11 +00:00
|
|
|
|
}
|
2015-10-22 15:31:42 +00:00
|
|
|
|
|
|
|
|
|
/** Преобразование даты-с-временем в дату: отбрасывание времени.
|
|
|
|
|
*/
|
|
|
|
|
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name> : details::ToDateConverter<DataTypeDateTime, details::ToDateTransform, Name> {};
|
|
|
|
|
|
|
|
|
|
/** Отдельный случай для преобразования (U)Int32 или (U)Int64 в Date.
|
|
|
|
|
* Если число меньше 65536, то оно понимается, как DayNum, а если больше или равно - как unix timestamp.
|
|
|
|
|
* Немного нелогично, что мы, по сути, помещаем две разные функции в одну.
|
|
|
|
|
* Но зато это позволяет поддержать распространённый случай,
|
|
|
|
|
* когда пользователь пишет toDate(UInt32), ожидая, что это - перевод unix timestamp в дату
|
|
|
|
|
* (иначе такое использование было бы распространённой ошибкой).
|
|
|
|
|
*/
|
|
|
|
|
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> {};
|
2015-03-16 04:44:53 +00:00
|
|
|
|
|
2011-10-16 03:05:15 +00:00
|
|
|
|
/** Преобразование чисел, дат, дат-с-временем в строки: через форматирование.
|
2011-10-16 01:57:10 +00:00
|
|
|
|
*/
|
2015-12-22 13:07:40 +00:00
|
|
|
|
template <typename DataType> struct FormatImpl
|
|
|
|
|
{
|
|
|
|
|
static void execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType & type = DataType{})
|
|
|
|
|
{
|
|
|
|
|
writeText(x, wb);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
template <> struct FormatImpl<DataTypeDate>
|
|
|
|
|
{
|
|
|
|
|
static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate & type = DataTypeDate{})
|
|
|
|
|
{
|
|
|
|
|
writeDateText(DayNum_t(x), wb);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
template <> struct FormatImpl<DataTypeDateTime>
|
|
|
|
|
{
|
|
|
|
|
static void execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime &type = DataTypeDateTime{})
|
|
|
|
|
{
|
|
|
|
|
writeDateTimeText(x, wb);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
template <typename FieldType> struct FormatImpl<DataTypeEnum<FieldType>>
|
|
|
|
|
{
|
|
|
|
|
static void execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum<FieldType> & type)
|
|
|
|
|
{
|
2015-12-30 11:53:12 +00:00
|
|
|
|
writeString(type.getNameForValue(x), wb);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
2011-10-16 03:05:15 +00:00
|
|
|
|
|
2015-12-30 11:53:12 +00:00
|
|
|
|
|
|
|
|
|
/// DataTypeEnum<T> to DataType<T> free conversion
|
|
|
|
|
template <typename FieldType, typename Name>
|
|
|
|
|
struct ConvertImpl<DataTypeEnum<FieldType>, typename DataTypeFromFieldType<FieldType>::Type, Name>
|
|
|
|
|
{
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
|
|
|
|
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2011-10-16 01:57:10 +00:00
|
|
|
|
template <typename FromDataType, typename Name>
|
|
|
|
|
struct ConvertImpl<FromDataType, DataTypeString, Name>
|
|
|
|
|
{
|
|
|
|
|
typedef typename FromDataType::FieldType FromFieldType;
|
|
|
|
|
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2015-12-22 13:07:40 +00:00
|
|
|
|
const auto & col_with_name_and_type = block.getByPosition(arguments[0]);
|
|
|
|
|
const auto & type = static_cast<const FromDataType &>(*col_with_name_and_type.type);
|
|
|
|
|
|
|
|
|
|
if (const auto col_from = typeid_cast<const ColumnVector<FromFieldType> *>(&*col_with_name_and_type.column))
|
2011-10-16 01:57:10 +00:00
|
|
|
|
{
|
|
|
|
|
ColumnString * col_to = new ColumnString;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
const typename ColumnVector<FromFieldType>::Container_t & vec_from = col_from->getData();
|
2013-05-05 15:25:25 +00:00
|
|
|
|
ColumnString::Chars_t & data_to = col_to->getChars();
|
2011-10-16 01:57:10 +00:00
|
|
|
|
ColumnString::Offsets_t & offsets_to = col_to->getOffsets();
|
|
|
|
|
size_t size = vec_from.size();
|
|
|
|
|
data_to.resize(size * 2);
|
|
|
|
|
offsets_to.resize(size);
|
|
|
|
|
|
2013-09-15 05:51:43 +00:00
|
|
|
|
WriteBufferFromVector<ColumnString::Chars_t> write_buffer(data_to);
|
2011-10-16 01:57:10 +00:00
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<FromDataType>::execute(vec_from[i], write_buffer, type);
|
2011-10-16 01:57:10 +00:00
|
|
|
|
writeChar(0, write_buffer);
|
|
|
|
|
offsets_to[i] = write_buffer.count();
|
|
|
|
|
}
|
|
|
|
|
data_to.resize(write_buffer.count());
|
|
|
|
|
}
|
2015-12-22 13:07:40 +00:00
|
|
|
|
else if (const auto col_from = typeid_cast<const ColumnConst<FromFieldType> *>(&*col_with_name_and_type.column))
|
2011-10-16 01:57:10 +00:00
|
|
|
|
{
|
|
|
|
|
std::vector<char> buf;
|
2013-09-15 05:51:43 +00:00
|
|
|
|
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<FromDataType>::execute(col_from->getData(), write_buffer, type);
|
2011-10-16 01:57:10 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConstString(col_from->size(), std::string(&buf[0], write_buffer.count()));
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2014-11-12 17:23:26 +00:00
|
|
|
|
+ " of first argument of function " + Name::name,
|
2011-10-16 01:57:10 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
namespace details { namespace {
|
|
|
|
|
|
|
|
|
|
/** Пусть source_timestamp представляет дату и время в исходном часовом поясе соответствующем
|
|
|
|
|
* объекту from_date_lut. Эта функция возвращает timestamp представлящий те же дату и время
|
|
|
|
|
* в часовом поясе соответствующем объекту to_date_lut.
|
|
|
|
|
*/
|
2015-07-08 14:25:11 +00:00
|
|
|
|
time_t convertTimestamp(time_t source_timestamp, const DateLUTImpl & from_date_lut, const DateLUTImpl & to_date_lut)
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
if (&from_date_lut == &to_date_lut)
|
|
|
|
|
return source_timestamp;
|
2015-07-01 16:07:29 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
const auto & values = from_date_lut.getValues(source_timestamp);
|
|
|
|
|
return to_date_lut.makeDateTime(values.year, values.month, values.day_of_month,
|
|
|
|
|
from_date_lut.toHourInaccurate(source_timestamp),
|
|
|
|
|
from_date_lut.toMinuteInaccurate(source_timestamp),
|
|
|
|
|
from_date_lut.toSecondInaccurate(source_timestamp));
|
2015-07-01 16:07:29 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
/** Функции для преобразования даты + времени в строку.
|
|
|
|
|
*/
|
|
|
|
|
struct DateTimeToStringConverter
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
using FromFieldType = typename DataTypeDateTime::FieldType;
|
|
|
|
|
|
|
|
|
|
static void vector_vector(const PODArray<FromFieldType> & vec_from, const ColumnString::Chars_t & data,
|
|
|
|
|
const ColumnString::Offsets_t & offsets, ColumnString & vec_to)
|
2015-07-03 00:09:57 +00:00
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
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);
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<DataTypeDateTime>::execute(ti, write_buffer);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
writeChar(0, write_buffer);
|
|
|
|
|
offsets_to[i] = write_buffer.count();
|
|
|
|
|
|
|
|
|
|
prev_offset = cur_offset;
|
|
|
|
|
}
|
|
|
|
|
data_to.resize(write_buffer.count());
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
static void vector_constant(const PODArray<FromFieldType> & vec_from, const std::string & data,
|
2015-07-03 00:09:57 +00:00
|
|
|
|
ColumnString & vec_to)
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(data);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
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);
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto ti = convertTimestamp(vec_from[i], remote_date_lut, local_date_lut);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<DataTypeDateTime>::execute(ti, write_buffer);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
writeChar(0, write_buffer);
|
|
|
|
|
offsets_to[i] = write_buffer.count();
|
2015-07-01 16:07:29 +00:00
|
|
|
|
}
|
2015-07-03 00:09:57 +00:00
|
|
|
|
data_to.resize(write_buffer.count());
|
|
|
|
|
}
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-07-08 17:30:42 +00:00
|
|
|
|
static void vector_constant(const PODArray<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)
|
|
|
|
|
{
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<DataTypeDateTime>::execute(vec_from[i], write_buffer);
|
2015-07-08 17:30:42 +00:00
|
|
|
|
writeChar(0, write_buffer);
|
|
|
|
|
offsets_to[i] = write_buffer.count();
|
|
|
|
|
}
|
|
|
|
|
data_to.resize(write_buffer.count());
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
static void constant_vector(FromFieldType from, const ColumnString::Chars_t & data,
|
2015-07-03 00:09:57 +00:00
|
|
|
|
const ColumnString::Offsets_t & offsets,
|
|
|
|
|
ColumnString & vec_to)
|
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
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)
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
2015-07-03 00:09:57 +00:00
|
|
|
|
ColumnString::Offset_t cur_offset = offsets[i];
|
|
|
|
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<DataTypeDateTime>::execute(ti, write_buffer);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
writeChar(0, write_buffer);
|
|
|
|
|
offsets_to[i] = write_buffer.count();
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
prev_offset = cur_offset;
|
|
|
|
|
}
|
|
|
|
|
data_to.resize(write_buffer.count());
|
|
|
|
|
}
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
static void constant_constant(FromFieldType from, const std::string & data, std::string & to)
|
2015-07-03 00:09:57 +00:00
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(data);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
std::vector<char> buf;
|
|
|
|
|
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto ti = convertTimestamp(from, remote_date_lut, local_date_lut);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<DataTypeDateTime>::execute(ti, write_buffer);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
to = std::string(&buf[0], write_buffer.count());
|
|
|
|
|
}
|
2015-07-08 17:30:42 +00:00
|
|
|
|
|
|
|
|
|
static void constant_constant(FromFieldType from, std::string & to)
|
|
|
|
|
{
|
|
|
|
|
std::vector<char> buf;
|
|
|
|
|
WriteBufferFromVector<std::vector<char> > write_buffer(buf);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
FormatImpl<DataTypeDateTime>::execute(from, write_buffer);
|
2015-07-08 17:30:42 +00:00
|
|
|
|
to = std::string(&buf[0], write_buffer.count());
|
|
|
|
|
}
|
2015-07-03 00:09:57 +00:00
|
|
|
|
};
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
}}
|
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
template<typename Name>
|
|
|
|
|
struct ConvertImpl<DataTypeDateTime, DataTypeString, Name>
|
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
using Op = details::DateTimeToStringConverter;
|
|
|
|
|
using FromFieldType = Op::FromFieldType;
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
|
|
|
|
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
2015-07-03 11:55:51 +00:00
|
|
|
|
const auto * sources = typeid_cast<const ColumnVector<FromFieldType> *>(&*source_col);
|
|
|
|
|
const auto * const_source = typeid_cast<const ColumnConst<FromFieldType> *>(&*source_col);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
if (arguments.size() == 1)
|
|
|
|
|
{
|
|
|
|
|
if (sources)
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
2015-07-03 00:09:57 +00:00
|
|
|
|
ColumnString * col_to = new ColumnString;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
auto & vec_from = sources->getData();
|
|
|
|
|
auto & vec_to = *col_to;
|
|
|
|
|
|
2015-07-08 17:30:42 +00:00
|
|
|
|
Op::vector_constant(vec_from, vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
|
|
|
|
else if (const_source)
|
|
|
|
|
{
|
|
|
|
|
std::string res;
|
2015-07-08 17:30:42 +00:00
|
|
|
|
Op::constant_constant(const_source->getData(), res);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConstString(const_source->size(), res);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
|
+ " of first argument of function " + Name::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2015-07-01 16:07:29 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2015-07-03 00:09:57 +00:00
|
|
|
|
else if (arguments.size() == 2)
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
2015-07-03 00:09:57 +00:00
|
|
|
|
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
2015-07-03 11:55:51 +00:00
|
|
|
|
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
|
|
|
|
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
if (sources)
|
|
|
|
|
{
|
|
|
|
|
ColumnString * col_to = new ColumnString;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
auto & vec_from = sources->getData();
|
|
|
|
|
auto & vec_to = *col_to;
|
|
|
|
|
|
|
|
|
|
if (time_zones)
|
2015-07-03 11:55:51 +00:00
|
|
|
|
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
else if (const_time_zone)
|
2015-07-03 11:55:51 +00:00
|
|
|
|
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
|
|
|
|
+ " of second argument of function " + Name::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
else if (const_source)
|
|
|
|
|
{
|
|
|
|
|
if (time_zones)
|
|
|
|
|
{
|
|
|
|
|
ColumnString * col_to = new ColumnString;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
auto & vec_to = *col_to;
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
|
|
|
|
else if (const_time_zone)
|
|
|
|
|
{
|
|
|
|
|
std::string res;
|
2015-07-03 11:55:51 +00:00
|
|
|
|
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConstString(const_source->size(), res);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
|
|
|
|
+ " of second argument of function " + Name::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
|
+ " of first argument of function " + Name::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2015-07-01 16:07:29 +00:00
|
|
|
|
}
|
2015-07-03 11:55:51 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
2015-07-01 16:07:29 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2011-10-16 03:05:15 +00:00
|
|
|
|
/** Преобразование строк в числа, даты, даты-с-временем: через парсинг.
|
2011-10-16 01:57:10 +00:00
|
|
|
|
*/
|
2012-07-21 03:45:48 +00:00
|
|
|
|
template <typename DataType> void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb) { readText(x,rb); }
|
2011-10-16 03:05:15 +00:00
|
|
|
|
|
2012-07-21 03:45:48 +00:00
|
|
|
|
template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb)
|
2011-10-16 03:05:15 +00:00
|
|
|
|
{
|
2013-08-11 03:40:14 +00:00
|
|
|
|
DayNum_t tmp(0);
|
2012-07-21 03:45:48 +00:00
|
|
|
|
readDateText(tmp, rb);
|
2011-10-16 03:05:15 +00:00
|
|
|
|
x = tmp;
|
|
|
|
|
}
|
|
|
|
|
|
2012-07-21 03:45:48 +00:00
|
|
|
|
template <> inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb)
|
2011-10-16 03:05:15 +00:00
|
|
|
|
{
|
|
|
|
|
time_t tmp = 0;
|
2012-07-21 03:45:48 +00:00
|
|
|
|
readDateTimeText(tmp, rb);
|
2011-10-16 03:05:15 +00:00
|
|
|
|
x = tmp;
|
|
|
|
|
}
|
|
|
|
|
|
2011-10-16 01:57:10 +00:00
|
|
|
|
template <typename ToDataType, typename Name>
|
|
|
|
|
struct ConvertImpl<DataTypeString, ToDataType, Name>
|
|
|
|
|
{
|
|
|
|
|
typedef typename ToDataType::FieldType ToFieldType;
|
|
|
|
|
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-16 03:05:15 +00:00
|
|
|
|
{
|
|
|
|
|
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
2013-05-05 15:25:25 +00:00
|
|
|
|
const ColumnString::Chars_t & data_from = col_from->getChars();
|
2011-10-16 03:05:15 +00:00
|
|
|
|
typename ColumnVector<ToFieldType>::Container_t & vec_to = col_to->getData();
|
|
|
|
|
size_t size = col_from->size();
|
|
|
|
|
vec_to.resize(size);
|
|
|
|
|
|
|
|
|
|
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&data_from[0])), data_from.size(), 0);
|
|
|
|
|
|
|
|
|
|
char zero = 0;
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
parseImpl<ToDataType>(vec_to[i], read_buffer);
|
|
|
|
|
readChar(zero, read_buffer);
|
|
|
|
|
if (zero != 0)
|
2014-04-13 13:58:54 +00:00
|
|
|
|
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
2011-10-16 03:05:15 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const ColumnConstString * col_from = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column))
|
2011-10-16 03:05:15 +00:00
|
|
|
|
{
|
|
|
|
|
const String & s = col_from->getData();
|
2011-11-29 18:53:44 +00:00
|
|
|
|
ReadBufferFromString read_buffer(s);
|
2011-10-16 03:05:15 +00:00
|
|
|
|
ToFieldType x = 0;
|
|
|
|
|
parseImpl<ToDataType>(x, read_buffer);
|
|
|
|
|
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), x);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2014-11-12 17:23:26 +00:00
|
|
|
|
+ " of first argument of function " + Name::name,
|
2011-10-16 03:05:15 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2011-10-16 01:57:10 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
namespace details { namespace {
|
|
|
|
|
|
|
|
|
|
/** Функции для преобразования строк в timestamp.
|
|
|
|
|
*/
|
|
|
|
|
struct StringToTimestampConverter
|
2015-07-01 17:32:04 +00:00
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
using ToFieldType = typename DataTypeInt32::FieldType;
|
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
static void vector_vector(const ColumnString::Chars_t & vec_from, const ColumnString::Chars_t & data,
|
2015-07-03 11:55:51 +00:00
|
|
|
|
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
2015-07-03 00:09:57 +00:00
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
2015-07-03 00:09:57 +00:00
|
|
|
|
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
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);
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
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,
|
2015-07-03 11:55:51 +00:00
|
|
|
|
PODArray<ToFieldType> & vec_to)
|
2015-07-01 17:32:04 +00:00
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(data);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
char zero = 0;
|
|
|
|
|
for (size_t i = 0; i < vec_to.size(); ++i)
|
2015-07-01 17:32:04 +00:00
|
|
|
|
{
|
2015-07-03 00:09:57 +00:00
|
|
|
|
DataTypeDateTime::FieldType x = 0;
|
|
|
|
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
vec_to[i] = ti;
|
|
|
|
|
readChar(zero, read_buffer);
|
|
|
|
|
if (zero != 0)
|
|
|
|
|
throw Exception("Cannot parse from string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
}
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
2015-07-08 17:30:42 +00:00
|
|
|
|
static void vector_constant(const ColumnString::Chars_t & vec_from, PODArray<ToFieldType> & vec_to)
|
|
|
|
|
{
|
|
|
|
|
ReadBuffer read_buffer(const_cast<char *>(reinterpret_cast<const char *>(&vec_from[0])), vec_from.size(), 0);
|
|
|
|
|
|
|
|
|
|
char zero = 0;
|
|
|
|
|
for (size_t i = 0; i < vec_to.size(); ++i)
|
|
|
|
|
{
|
|
|
|
|
DataTypeDateTime::FieldType x = 0;
|
|
|
|
|
parseImpl<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);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
static void constant_vector(const std::string & from, const ColumnString::Chars_t & data,
|
2015-07-03 11:55:51 +00:00
|
|
|
|
const ColumnString::Offsets_t & offsets, PODArray<ToFieldType> & vec_to)
|
2015-07-03 00:09:57 +00:00
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
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)
|
2015-07-01 17:32:04 +00:00
|
|
|
|
{
|
2015-07-03 00:09:57 +00:00
|
|
|
|
ColumnString::Offset_t cur_offset = offsets[i];
|
|
|
|
|
const std::string time_zone(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1);
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(time_zone);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto ti = convertTimestamp(x, local_date_lut, remote_date_lut);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
vec_to[i] = ti;
|
|
|
|
|
prev_offset = cur_offset;
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
static void constant_constant(const std::string & from, const std::string & data, ToFieldType & to)
|
2015-07-03 00:09:57 +00:00
|
|
|
|
{
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & local_date_lut = DateLUT::instance();
|
|
|
|
|
const auto & remote_date_lut = DateLUT::instance(data);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
ReadBufferFromString read_buffer(from);
|
|
|
|
|
DataTypeDateTime::FieldType x = 0;
|
|
|
|
|
parseImpl<DataTypeDateTime>(x, read_buffer);
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
to = convertTimestamp(x, local_date_lut, remote_date_lut);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
2015-07-08 17:30:42 +00:00
|
|
|
|
|
|
|
|
|
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;
|
|
|
|
|
}
|
2015-07-03 00:09:57 +00:00
|
|
|
|
};
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
}}
|
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
|
|
|
|
|
|
|
|
|
template<>
|
|
|
|
|
struct ConvertImpl<DataTypeString, DataTypeInt32, NameToUnixTimestamp>
|
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
using Op = details::StringToTimestampConverter;
|
|
|
|
|
using ToFieldType = Op::ToFieldType;
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
|
|
|
|
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
2015-07-03 11:55:51 +00:00
|
|
|
|
const auto * sources = typeid_cast<const ColumnString *>(&*source_col);
|
|
|
|
|
const auto * const_source = typeid_cast<const ColumnConstString *>(&*source_col);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
if (arguments.size() == 1)
|
|
|
|
|
{
|
|
|
|
|
if (sources)
|
2015-07-01 17:32:04 +00:00
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto * col_to = new ColumnVector<ToFieldType>;
|
2015-07-03 00:09:57 +00:00
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
auto & vec_from = sources->getChars();
|
|
|
|
|
auto & vec_to = col_to->getData();
|
|
|
|
|
size_t size = sources->size();
|
|
|
|
|
vec_to.resize(size);
|
|
|
|
|
|
2015-07-08 17:30:42 +00:00
|
|
|
|
Op::vector_constant(vec_from, vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
|
|
|
|
else if (const_source)
|
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
ToFieldType res;
|
2015-07-08 17:30:42 +00:00
|
|
|
|
Op::constant_constant(const_source->getData(), res);
|
2015-07-03 11:55:51 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
|
+ " of first argument of function " + NameToUnixTimestamp::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2015-07-03 00:09:57 +00:00
|
|
|
|
else if (arguments.size() == 2)
|
2015-07-01 17:32:04 +00:00
|
|
|
|
{
|
2015-07-03 00:09:57 +00:00
|
|
|
|
const ColumnPtr time_zone_col = block.getByPosition(arguments[1]).column;
|
2015-07-03 11:55:51 +00:00
|
|
|
|
const auto * time_zones = typeid_cast<const ColumnString *>(&*time_zone_col);
|
|
|
|
|
const auto * const_time_zone = typeid_cast<const ColumnConstString *>(&*time_zone_col);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
|
|
|
|
if (sources)
|
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto * col_to = new ColumnVector<ToFieldType>;
|
2015-07-03 00:09:57 +00:00
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
auto & vec_from = sources->getChars();
|
|
|
|
|
auto & vec_to = col_to->getData();
|
|
|
|
|
size_t size = sources->size();
|
|
|
|
|
vec_to.resize(size);
|
|
|
|
|
|
|
|
|
|
if (time_zones)
|
2015-07-03 11:55:51 +00:00
|
|
|
|
Op::vector_vector(vec_from, time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
else if (const_time_zone)
|
2015-07-03 11:55:51 +00:00
|
|
|
|
Op::vector_constant(vec_from, const_time_zone->getData(), vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
|
|
|
|
+ " of second argument of function " + NameToUnixTimestamp::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
else if (const_source)
|
|
|
|
|
{
|
|
|
|
|
if (time_zones)
|
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
auto * col_to = new ColumnVector<ToFieldType>;
|
2015-07-03 00:09:57 +00:00
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
auto & vec_to = col_to->getData();
|
|
|
|
|
vec_to.resize(time_zones->getOffsets().size());
|
|
|
|
|
|
2015-07-03 11:55:51 +00:00
|
|
|
|
Op::constant_vector(const_source->getData(), time_zones->getChars(), time_zones->getOffsets(), vec_to);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
|
|
|
|
else if (const_time_zone)
|
|
|
|
|
{
|
2015-07-03 11:55:51 +00:00
|
|
|
|
ToFieldType res;
|
|
|
|
|
Op::constant_constant(const_source->getData(), const_time_zone->getData(), res);
|
|
|
|
|
block.getByPosition(result).column = new ColumnConst<ToFieldType>(const_source->size(), res);
|
2015-07-03 00:09:57 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
|
|
|
|
+ " of second argument of function " + NameToUnixTimestamp::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
|
+ " of first argument of function " + NameToUnixTimestamp::name,
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
}
|
2015-07-03 11:55:51 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Internal error.", ErrorCodes::LOGICAL_ERROR);
|
2015-07-01 17:32:04 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
|
2011-10-16 03:05:15 +00:00
|
|
|
|
/** Если типы совпадают - просто скопируем ссылку на столбец.
|
|
|
|
|
*/
|
2011-10-16 01:57:10 +00:00
|
|
|
|
template <typename Name>
|
|
|
|
|
struct ConvertImpl<DataTypeString, DataTypeString, Name>
|
|
|
|
|
{
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2011-10-16 03:05:15 +00:00
|
|
|
|
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
|
2011-10-16 01:57:10 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2014-04-13 13:58:54 +00:00
|
|
|
|
/** Преобразование из FixedString.
|
|
|
|
|
*/
|
|
|
|
|
template <typename ToDataType, typename Name>
|
|
|
|
|
struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
|
|
|
|
{
|
|
|
|
|
typedef typename ToDataType::FieldType ToFieldType;
|
|
|
|
|
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const ColumnFixedString * col_from = typeid_cast<const ColumnFixedString *>(&*block.getByPosition(arguments[0]).column))
|
2014-04-13 13:58:54 +00:00
|
|
|
|
{
|
|
|
|
|
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
const ColumnFixedString::Chars_t & data_from = col_from->getChars();
|
|
|
|
|
size_t n = col_from->getN();
|
|
|
|
|
typename ColumnVector<ToFieldType>::Container_t & vec_to = col_to->getData();
|
|
|
|
|
size_t size = col_from->size();
|
|
|
|
|
vec_to.resize(size);
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
char * begin = const_cast<char *>(reinterpret_cast<const char *>(&data_from[i * n]));
|
|
|
|
|
char * end = begin + n;
|
|
|
|
|
ReadBuffer read_buffer(begin, n, 0);
|
|
|
|
|
parseImpl<ToDataType>(vec_to[i], read_buffer);
|
|
|
|
|
|
|
|
|
|
if (!read_buffer.eof())
|
|
|
|
|
{
|
|
|
|
|
while (read_buffer.position() < end && *read_buffer.position() == 0)
|
|
|
|
|
++read_buffer.position();
|
|
|
|
|
|
|
|
|
|
if (read_buffer.position() < end)
|
|
|
|
|
throw Exception("Cannot parse from fixed string.", ErrorCodes::CANNOT_PARSE_NUMBER);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column))
|
2014-04-13 13:58:54 +00:00
|
|
|
|
{
|
|
|
|
|
ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2014-11-12 17:23:26 +00:00
|
|
|
|
+ " of first argument of function " + Name::name,
|
2014-04-13 13:58:54 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
/** Преобразование из FixedString в String.
|
|
|
|
|
* При этом, вырезаются последовательности нулевых байт с конца строк.
|
|
|
|
|
*/
|
|
|
|
|
template <typename Name>
|
|
|
|
|
struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
|
|
|
|
{
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const ColumnFixedString * col_from = typeid_cast<const ColumnFixedString *>(&*block.getByPosition(arguments[0]).column))
|
2014-04-13 13:58:54 +00:00
|
|
|
|
{
|
|
|
|
|
ColumnString * col_to = new ColumnString;
|
|
|
|
|
block.getByPosition(result).column = col_to;
|
|
|
|
|
|
|
|
|
|
const ColumnFixedString::Chars_t & data_from = col_from->getChars();
|
|
|
|
|
ColumnString::Chars_t & data_to = col_to->getChars();
|
|
|
|
|
ColumnString::Offsets_t & offsets_to = col_to->getOffsets();
|
|
|
|
|
size_t size = col_from->size();
|
|
|
|
|
size_t n = col_from->getN();
|
|
|
|
|
data_to.resize(size * (n + 1)); /// + 1 - нулевой байт
|
|
|
|
|
offsets_to.resize(size);
|
|
|
|
|
|
|
|
|
|
size_t offset_from = 0;
|
|
|
|
|
size_t offset_to = 0;
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
|
{
|
|
|
|
|
size_t bytes_to_copy = n;
|
|
|
|
|
while (bytes_to_copy > 0 && data_from[offset_from + bytes_to_copy - 1] == 0)
|
|
|
|
|
--bytes_to_copy;
|
|
|
|
|
|
|
|
|
|
memcpy(&data_to[offset_to], &data_from[offset_from], bytes_to_copy);
|
|
|
|
|
offset_from += n;
|
|
|
|
|
offset_to += bytes_to_copy;
|
|
|
|
|
data_to[offset_to] = 0;
|
|
|
|
|
++offset_to;
|
|
|
|
|
offsets_to[i] = offset_to;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
data_to.resize(offset_to);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const ColumnConstString * col_from = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column))
|
2014-04-13 13:58:54 +00:00
|
|
|
|
{
|
|
|
|
|
const String & s = col_from->getData();
|
|
|
|
|
|
|
|
|
|
size_t bytes_to_copy = s.size();
|
|
|
|
|
while (bytes_to_copy > 0 && s[bytes_to_copy - 1] == 0)
|
|
|
|
|
--bytes_to_copy;
|
|
|
|
|
|
|
|
|
|
block.getByPosition(result).column = new ColumnConstString(col_from->size(), s.substr(0, bytes_to_copy));
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2014-11-12 17:23:26 +00:00
|
|
|
|
+ " of first argument of function " + Name::name,
|
2014-04-13 13:58:54 +00:00
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
|
/// Предварительное объявление.
|
|
|
|
|
struct NameToDate { static constexpr auto name = "toDate"; };
|
2011-10-16 01:57:10 +00:00
|
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
|
template <typename ToDataType, typename Name, typename MonotonicityImpl>
|
2011-10-16 01:57:10 +00:00
|
|
|
|
class FunctionConvert : public IFunction
|
|
|
|
|
{
|
2011-10-15 23:40:56 +00:00
|
|
|
|
public:
|
2015-12-29 13:51:12 +00:00
|
|
|
|
using Monotonic = MonotonicityImpl;
|
|
|
|
|
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = Name::name;
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionConvert; }
|
|
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
|
/// Получить имя функции.
|
2015-10-11 23:36:45 +00:00
|
|
|
|
String getName() const override
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
2015-10-11 23:36:45 +00:00
|
|
|
|
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2015-07-01 16:07:29 +00:00
|
|
|
|
return getReturnTypeImpl(arguments);
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
2015-10-11 23:36:45 +00:00
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
|
2011-10-15 23:40:56 +00:00
|
|
|
|
{
|
2011-10-16 01:57:10 +00:00
|
|
|
|
IDataType * from_type = &*block.getByPosition(arguments[0]).type;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
|
|
|
|
if (typeid_cast<const DataTypeUInt8 * >(from_type)) ConvertImpl<DataTypeUInt8, ToDataType, Name>::execute(block, arguments, result);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
else if (typeid_cast<const DataTypeUInt16 * >(from_type)) ConvertImpl<DataTypeUInt16, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeUInt32 * >(from_type)) ConvertImpl<DataTypeUInt32, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeUInt64 * >(from_type)) ConvertImpl<DataTypeUInt64, ToDataType, Name>::execute(block, arguments, result);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (typeid_cast<const DataTypeInt8 * >(from_type)) ConvertImpl<DataTypeInt8, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeInt16 * >(from_type)) ConvertImpl<DataTypeInt16, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeInt32 * >(from_type)) ConvertImpl<DataTypeInt32, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeInt64 * >(from_type)) ConvertImpl<DataTypeInt64, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeFloat32 * >(from_type)) ConvertImpl<DataTypeFloat32, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeFloat64 * >(from_type)) ConvertImpl<DataTypeFloat64, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeDate * >(from_type)) ConvertImpl<DataTypeDate, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeDateTime * >(from_type)) ConvertImpl<DataTypeDateTime, ToDataType, Name>::execute(block, arguments, result);
|
2015-03-16 04:44:53 +00:00
|
|
|
|
else if (typeid_cast<const DataTypeString * >(from_type)) ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (typeid_cast<const DataTypeFixedString *>(from_type)) ConvertImpl<DataTypeFixedString, ToDataType, Name>::execute(block, arguments, result);
|
2015-12-22 13:07:40 +00:00
|
|
|
|
else if (typeid_cast<const DataTypeEnum8 *>(from_type)) ConvertImpl<DataTypeEnum8, ToDataType, Name>::execute(block, arguments, result);
|
|
|
|
|
else if (typeid_cast<const DataTypeEnum16 *>(from_type)) ConvertImpl<DataTypeEnum16, ToDataType, Name>::execute(block, arguments, result);
|
2011-10-16 01:57:10 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|
2015-07-01 16:07:29 +00:00
|
|
|
|
|
2015-11-29 08:06:29 +00:00
|
|
|
|
bool hasInformationAboutMonotonicity() const override
|
|
|
|
|
{
|
|
|
|
|
return Monotonic::has();
|
|
|
|
|
}
|
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
2015-11-29 08:06:29 +00:00
|
|
|
|
{
|
2015-11-29 11:58:44 +00:00
|
|
|
|
return Monotonic::get(type, left, right);
|
2015-11-29 08:06:29 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-07-01 16:07:29 +00:00
|
|
|
|
private:
|
2015-07-01 17:32:04 +00:00
|
|
|
|
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
2015-07-01 16:07:29 +00:00
|
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
2015-10-22 15:31:42 +00:00
|
|
|
|
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
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
|
|
|
|
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;
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-01 17:32:04 +00:00
|
|
|
|
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
2015-07-01 16:07:29 +00:00
|
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
2015-07-01 17:32:04 +00:00
|
|
|
|
typename std::enable_if<std::is_same<ToDataType2, DataTypeString>::value>::type * = nullptr) const
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
|
|
|
|
if ((arguments.size() < 1) || (arguments.size() > 2))
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
|
+ toString(arguments.size()) + ", should be 1 or 2.",
|
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
if (typeid_cast<const DataTypeDateTime *>(&*arguments[0]) == nullptr)
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 1)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
|
+ toString(arguments.size()) + ", should be 1.",
|
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
}
|
2015-10-22 15:31:42 +00:00
|
|
|
|
else if ((arguments.size() == 2) && (typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr))
|
2015-07-01 16:07:29 +00:00
|
|
|
|
{
|
|
|
|
|
throw Exception{
|
|
|
|
|
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return new ToDataType2;
|
|
|
|
|
}
|
2015-07-01 17:32:04 +00:00
|
|
|
|
|
|
|
|
|
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
|
|
|
|
|
typename std::enable_if<std::is_same<Name2, NameToUnixTimestamp>::value, void>::type * = nullptr) const
|
|
|
|
|
{
|
|
|
|
|
if ((arguments.size() < 1) || (arguments.size() > 2))
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
|
+ toString(arguments.size()) + ", should be 1 or 2.",
|
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
if (typeid_cast<const DataTypeString *>(&*arguments[0]) == nullptr)
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 1)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
|
+ toString(arguments.size()) + ", should be 1.",
|
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
}
|
2015-10-22 15:31:42 +00:00
|
|
|
|
else if ((arguments.size() == 2) && (typeid_cast<const DataTypeString *>(&*arguments[1]) == nullptr))
|
2015-12-28 13:38:45 +00:00
|
|
|
|
{
|
2015-07-01 17:32:04 +00:00
|
|
|
|
throw Exception{
|
|
|
|
|
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return new ToDataType2;
|
|
|
|
|
}
|
2015-10-22 15:31:42 +00:00
|
|
|
|
|
|
|
|
|
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;
|
|
|
|
|
}
|
2011-10-15 23:40:56 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2013-12-03 10:19:57 +00:00
|
|
|
|
/** Преобразование в строку фиксированной длины реализовано только из строк.
|
|
|
|
|
*/
|
|
|
|
|
class FunctionToFixedString : public IFunction
|
|
|
|
|
{
|
|
|
|
|
public:
|
2014-11-12 17:23:26 +00:00
|
|
|
|
static constexpr auto name = "toFixedString";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionToFixedString; };
|
|
|
|
|
|
2013-12-03 10:19:57 +00:00
|
|
|
|
/// Получить имя функции.
|
2015-10-11 23:36:45 +00:00
|
|
|
|
String getName() const override
|
2013-12-03 10:19:57 +00:00
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
|
return name;
|
2013-12-03 10:19:57 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/** Получить тип результата по типам аргументов и значениям константных аргументов.
|
|
|
|
|
* Если функция неприменима для данных аргументов - кинуть исключение.
|
2014-04-08 07:58:53 +00:00
|
|
|
|
* Для неконстантных столбцов arguments[i].column = nullptr.
|
2013-12-03 10:19:57 +00:00
|
|
|
|
*/
|
2015-07-17 01:27:35 +00:00
|
|
|
|
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
|
2015-05-04 17:52:19 +00:00
|
|
|
|
DataTypePtr & out_return_type,
|
2015-10-11 23:36:45 +00:00
|
|
|
|
std::vector<ExpressionAction> & out_prerequisites) override
|
2013-12-03 10:19:57 +00:00
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 2)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
2013-12-05 13:15:04 +00:00
|
|
|
|
+ toString(arguments.size()) + ", should be 2.",
|
2013-12-03 10:19:57 +00:00
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
if (!arguments[1].column)
|
|
|
|
|
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
2015-03-31 15:54:36 +00:00
|
|
|
|
if (!typeid_cast<const DataTypeString *>(arguments[0].type.get()) &&
|
|
|
|
|
!typeid_cast<const DataTypeFixedString *>(arguments[0].type.get()))
|
|
|
|
|
throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED);
|
2013-12-03 10:19:57 +00:00
|
|
|
|
|
2015-03-31 15:54:36 +00:00
|
|
|
|
const size_t n = getSize(arguments[1]);
|
2013-12-03 10:19:57 +00:00
|
|
|
|
|
|
|
|
|
out_return_type = new DataTypeFixedString(n);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выполнить функцию над блоком.
|
2015-12-24 17:14:10 +00:00
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
|
|
|
|
{
|
|
|
|
|
const auto n = getSize(block.getByPosition(arguments[1]));
|
|
|
|
|
return execute(block, arguments, result, n);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
|
2013-12-03 10:19:57 +00:00
|
|
|
|
{
|
2015-12-24 17:14:10 +00:00
|
|
|
|
const auto & column = block.getByPosition(arguments[0]).column;
|
2013-12-03 10:19:57 +00:00
|
|
|
|
|
2015-12-24 17:14:10 +00:00
|
|
|
|
if (const auto column_const = typeid_cast<const ColumnConstString *>(&*column))
|
2013-12-03 10:19:57 +00:00
|
|
|
|
{
|
|
|
|
|
if (column_const->getData().size() > n)
|
2013-12-04 10:02:43 +00:00
|
|
|
|
throw Exception("String too long for type FixedString(" + toString(n) + ")",
|
2013-12-03 10:19:57 +00:00
|
|
|
|
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
2014-10-31 13:48:30 +00:00
|
|
|
|
|
|
|
|
|
auto resized_string = column_const->getData();
|
|
|
|
|
resized_string.resize(n);
|
|
|
|
|
|
2015-12-24 17:14:10 +00:00
|
|
|
|
block.getByPosition(result).column = new ColumnConst<String>{
|
|
|
|
|
column_const->size(), std::move(resized_string), new DataTypeFixedString(n)
|
|
|
|
|
};
|
2013-12-03 10:19:57 +00:00
|
|
|
|
}
|
2015-12-24 17:14:10 +00:00
|
|
|
|
else if (const auto column_string = typeid_cast<const ColumnString *>(&*column))
|
2013-12-03 10:19:57 +00:00
|
|
|
|
{
|
2015-12-24 17:14:10 +00:00
|
|
|
|
const auto column_fixed = new ColumnFixedString(n);
|
2013-12-03 10:19:57 +00:00
|
|
|
|
ColumnPtr result_ptr = column_fixed;
|
2015-12-24 17:14:10 +00:00
|
|
|
|
|
|
|
|
|
auto & out_chars = column_fixed->getChars();
|
|
|
|
|
const auto & in_chars = column_string->getChars();
|
|
|
|
|
const auto & in_offsets = column_string->getOffsets();
|
|
|
|
|
|
2013-12-03 10:19:57 +00:00
|
|
|
|
out_chars.resize_fill(in_offsets.size() * n);
|
2015-12-24 17:14:10 +00:00
|
|
|
|
|
2013-12-03 10:19:57 +00:00
|
|
|
|
for (size_t i = 0; i < in_offsets.size(); ++i)
|
|
|
|
|
{
|
2015-12-24 17:14:10 +00:00
|
|
|
|
const size_t off = i ? in_offsets[i - 1] : 0;
|
|
|
|
|
const size_t len = in_offsets[i] - off - 1;
|
2013-12-03 10:19:57 +00:00
|
|
|
|
if (len > n)
|
2013-12-04 10:02:43 +00:00
|
|
|
|
throw Exception("String too long for type FixedString(" + toString(n) + ")",
|
2013-12-03 10:19:57 +00:00
|
|
|
|
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
|
|
|
|
memcpy(&out_chars[i * n], &in_chars[off], len);
|
|
|
|
|
}
|
2015-12-24 17:14:10 +00:00
|
|
|
|
|
2013-12-03 10:19:57 +00:00
|
|
|
|
block.getByPosition(result).column = result_ptr;
|
|
|
|
|
}
|
2015-03-31 15:54:36 +00:00
|
|
|
|
else if (const auto column_fixed_string = typeid_cast<const ColumnFixedString *>(column.get()))
|
|
|
|
|
{
|
|
|
|
|
const auto src_n = column_fixed_string->getN();
|
|
|
|
|
if (src_n > n)
|
|
|
|
|
throw Exception{
|
|
|
|
|
"String too long for type FixedString(" + toString(n) + ")",
|
|
|
|
|
ErrorCodes::TOO_LARGE_STRING_SIZE
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
const auto column_fixed = new ColumnFixedString{n};
|
|
|
|
|
block.getByPosition(result).column = column_fixed;
|
|
|
|
|
|
|
|
|
|
auto & out_chars = column_fixed->getChars();
|
|
|
|
|
const auto & in_chars = column_fixed_string->getChars();
|
|
|
|
|
const auto size = column_fixed_string->size();
|
|
|
|
|
out_chars.resize_fill(size * n);
|
|
|
|
|
|
|
|
|
|
for (const auto i : ext::range(0, size))
|
|
|
|
|
memcpy(&out_chars[i * n], &in_chars[i * src_n], src_n);
|
|
|
|
|
}
|
2013-12-03 10:19:57 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private:
|
|
|
|
|
template <typename T>
|
2015-07-17 01:27:35 +00:00
|
|
|
|
bool getSizeTyped(const ColumnWithTypeAndName & column, size_t & out_size)
|
2013-12-03 10:19:57 +00:00
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (!typeid_cast<const typename DataTypeFromFieldType<T>::Type *>(&*column.type))
|
2013-12-03 10:19:57 +00:00
|
|
|
|
return false;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
const ColumnConst<T> * column_const = typeid_cast<const ColumnConst<T> *>(&*column.column);
|
2013-12-03 10:19:57 +00:00
|
|
|
|
if (!column_const)
|
|
|
|
|
throw Exception("Unexpected type of column for FixedString length: " + column.column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
2013-12-04 10:02:43 +00:00
|
|
|
|
T s = column_const->getData();
|
|
|
|
|
if (s <= 0)
|
|
|
|
|
throw Exception("FixedString length must be positive (unlike " + toString(s) + ")", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
out_size = static_cast<size_t>(s);
|
2013-12-03 10:19:57 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-17 01:27:35 +00:00
|
|
|
|
size_t getSize(const ColumnWithTypeAndName & column)
|
2013-12-03 10:19:57 +00:00
|
|
|
|
{
|
|
|
|
|
size_t res;
|
|
|
|
|
if (getSizeTyped<UInt8>(column, res) ||
|
|
|
|
|
getSizeTyped<UInt16>(column, res) ||
|
|
|
|
|
getSizeTyped<UInt32>(column, res) ||
|
|
|
|
|
getSizeTyped<UInt64>(column, res) ||
|
|
|
|
|
getSizeTyped< Int8 >(column, res) ||
|
|
|
|
|
getSizeTyped< Int16>(column, res) ||
|
|
|
|
|
getSizeTyped< Int32>(column, res) ||
|
|
|
|
|
getSizeTyped< Int64>(column, res))
|
|
|
|
|
return res;
|
|
|
|
|
throw Exception("Length of FixedString must be integer; got " + column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2015-11-29 08:06:29 +00:00
|
|
|
|
/// Монотонность.
|
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
struct PositiveMonotonicity
|
2015-11-29 08:06:29 +00:00
|
|
|
|
{
|
|
|
|
|
static bool has() { return true; }
|
2015-11-29 11:58:44 +00:00
|
|
|
|
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
2015-11-29 08:06:29 +00:00
|
|
|
|
{
|
2015-11-29 11:58:44 +00:00
|
|
|
|
return { true };
|
2015-11-29 08:06:29 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
struct ToIntMonotonicity
|
2015-11-29 08:06:29 +00:00
|
|
|
|
{
|
2015-11-29 11:58:44 +00:00
|
|
|
|
static bool has() { return true; }
|
|
|
|
|
|
|
|
|
|
template <typename T2 = T>
|
|
|
|
|
static UInt64 divideByRangeOfType(typename std::enable_if_t<sizeof(T2) != sizeof(UInt64), UInt64> x) { return x >> (sizeof(T) * 8); };
|
|
|
|
|
|
|
|
|
|
template <typename T2 = T>
|
|
|
|
|
static UInt64 divideByRangeOfType(typename std::enable_if_t<sizeof(T2) == sizeof(UInt64), UInt64> x) { return 0; };
|
|
|
|
|
|
|
|
|
|
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
2015-11-29 08:06:29 +00:00
|
|
|
|
{
|
2015-11-29 11:58:44 +00:00
|
|
|
|
size_t size_of_type = type.getSizeOfField();
|
|
|
|
|
|
|
|
|
|
/// Если тип расширяется, то функция монотонна.
|
|
|
|
|
if (sizeof(T) > size_of_type)
|
|
|
|
|
return { true };
|
|
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
|
/// Если тип совпадает - тоже. (Enum обрабатываем отдельно, так как он имеет другой тип)
|
|
|
|
|
if (typeid_cast<const typename DataTypeFromFieldType<T>::Type *>(&type) ||
|
|
|
|
|
typeid_cast<const DataTypeEnum<T> *>(&type))
|
2015-11-29 11:58:44 +00:00
|
|
|
|
return { true };
|
|
|
|
|
|
|
|
|
|
/// В других случаях, для неограниченного диапазона не знаем, будет ли функция монотонной.
|
|
|
|
|
if (left.isNull() || right.isNull())
|
|
|
|
|
return {};
|
|
|
|
|
|
|
|
|
|
/// Если преобразуем из float, то аргументы должны помещаться в тип результата.
|
|
|
|
|
if (typeid_cast<const DataTypeFloat32 *>(&type)
|
|
|
|
|
|| typeid_cast<const DataTypeFloat64 *>(&type))
|
|
|
|
|
{
|
|
|
|
|
Float64 left_float = left.get<Float64>();
|
|
|
|
|
Float64 right_float = right.get<Float64>();
|
|
|
|
|
|
|
|
|
|
if (left_float >= std::numeric_limits<T>::min() && left_float <= std::numeric_limits<T>::max()
|
|
|
|
|
&& right_float >= std::numeric_limits<T>::min() && right_float <= std::numeric_limits<T>::max())
|
|
|
|
|
return { true };
|
|
|
|
|
|
|
|
|
|
return {};
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Если меняем знаковость типа или преобразуем из даты, даты-времени, то аргумент должен быть из одной половинки.
|
|
|
|
|
/// На всякий случай, в остальных случаях тоже будем этого требовать.
|
|
|
|
|
if ((left.get<Int64>() >= 0) != (right.get<Int64>() >= 0))
|
|
|
|
|
return {};
|
|
|
|
|
|
|
|
|
|
/// Если уменьшаем тип, то все биты кроме тех, которые в него помещаются, должны совпадать.
|
|
|
|
|
if (divideByRangeOfType(left.get<UInt64>()) != divideByRangeOfType(right.get<UInt64>()))
|
|
|
|
|
return {};
|
|
|
|
|
|
|
|
|
|
return { true };
|
2015-11-29 08:06:29 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
/** Монотонность для функции toString определяем, в основном, для тестовых целей.
|
|
|
|
|
* Всерьёз вряд ли кто-нибудь рассчитывает на оптимизацию запросов с условиями toString(CounterID) = 34.
|
|
|
|
|
*/
|
|
|
|
|
struct ToStringMonotonicity
|
2015-11-29 08:06:29 +00:00
|
|
|
|
{
|
|
|
|
|
static bool has() { return true; }
|
2015-11-29 11:58:44 +00:00
|
|
|
|
|
|
|
|
|
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
2015-11-29 08:06:29 +00:00
|
|
|
|
{
|
2016-03-07 05:15:42 +00:00
|
|
|
|
IFunction::Monotonicity positive(true, true);
|
2015-11-29 11:58:44 +00:00
|
|
|
|
IFunction::Monotonicity not_monotonic;
|
2015-11-29 08:06:29 +00:00
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
/// Функция toString монотонна, если аргумент - Date или DateTime, или неотрицательные числа с одинаковым количеством знаков.
|
2015-11-29 08:06:29 +00:00
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
if (typeid_cast<const DataTypeDate *>(&type)
|
|
|
|
|
|| typeid_cast<const DataTypeDateTime *>(&type))
|
|
|
|
|
return positive;
|
|
|
|
|
|
|
|
|
|
if (left.isNull() || right.isNull())
|
2015-11-29 08:06:29 +00:00
|
|
|
|
return {};
|
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
if (left.getType() == Field::Types::UInt64
|
|
|
|
|
&& right.getType() == Field::Types::UInt64)
|
|
|
|
|
{
|
|
|
|
|
return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
|
|
|
|
|
|| (floor(log10(left.get<UInt64>())) == floor(log10(right.get<UInt64>())))
|
|
|
|
|
? positive : not_monotonic;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (left.getType() == Field::Types::Int64
|
|
|
|
|
&& right.getType() == Field::Types::Int64)
|
|
|
|
|
{
|
|
|
|
|
return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
|
|
|
|
|
|| (left.get<Int64>() > 0 && right.get<Int64>() > 0 && floor(log10(left.get<Int64>())) == floor(log10(right.get<Int64>())))
|
|
|
|
|
? positive : not_monotonic;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return not_monotonic;
|
2015-11-29 08:06:29 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2014-11-12 17:23:26 +00:00
|
|
|
|
struct NameToUInt8 { static constexpr auto name = "toUInt8"; };
|
|
|
|
|
struct NameToUInt16 { static constexpr auto name = "toUInt16"; };
|
|
|
|
|
struct NameToUInt32 { static constexpr auto name = "toUInt32"; };
|
|
|
|
|
struct NameToUInt64 { static constexpr auto name = "toUInt64"; };
|
|
|
|
|
struct NameToInt8 { static constexpr auto name = "toInt8"; };
|
|
|
|
|
struct NameToInt16 { static constexpr auto name = "toInt16"; };
|
|
|
|
|
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 NameToDateTime { static constexpr auto name = "toDateTime"; };
|
|
|
|
|
struct NameToString { static constexpr auto name = "toString"; };
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
|
typedef FunctionConvert<DataTypeUInt8, NameToUInt8, ToIntMonotonicity<UInt8>> FunctionToUInt8;
|
|
|
|
|
typedef FunctionConvert<DataTypeUInt16, NameToUInt16, ToIntMonotonicity<UInt16>> FunctionToUInt16;
|
|
|
|
|
typedef FunctionConvert<DataTypeUInt32, NameToUInt32, ToIntMonotonicity<UInt32>> FunctionToUInt32;
|
|
|
|
|
typedef FunctionConvert<DataTypeUInt64, NameToUInt64, ToIntMonotonicity<UInt64>> FunctionToUInt64;
|
|
|
|
|
typedef FunctionConvert<DataTypeInt8, NameToInt8, ToIntMonotonicity<Int8>> FunctionToInt8;
|
|
|
|
|
typedef FunctionConvert<DataTypeInt16, NameToInt16, ToIntMonotonicity<Int16>> FunctionToInt16;
|
|
|
|
|
typedef FunctionConvert<DataTypeInt32, NameToInt32, ToIntMonotonicity<Int32>> FunctionToInt32;
|
|
|
|
|
typedef FunctionConvert<DataTypeInt64, NameToInt64, ToIntMonotonicity<Int64>> FunctionToInt64;
|
|
|
|
|
typedef FunctionConvert<DataTypeFloat32, NameToFloat32, PositiveMonotonicity> FunctionToFloat32;
|
|
|
|
|
typedef FunctionConvert<DataTypeFloat64, NameToFloat64, PositiveMonotonicity> FunctionToFloat64;
|
|
|
|
|
typedef FunctionConvert<DataTypeDate, NameToDate, ToIntMonotonicity<UInt16>> FunctionToDate;
|
|
|
|
|
typedef FunctionConvert<DataTypeDateTime, NameToDateTime, ToIntMonotonicity<UInt32>> FunctionToDateTime;
|
|
|
|
|
typedef FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity> FunctionToString;
|
|
|
|
|
typedef FunctionConvert<DataTypeInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>> FunctionToUnixTimestamp;
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
|
template <typename DataType> struct FunctionTo;
|
|
|
|
|
template <> struct FunctionTo<DataTypeUInt8> { using Type = FunctionToUInt8; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeUInt16> { using Type = FunctionToUInt16; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeUInt32> { using Type = FunctionToUInt32; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeUInt64> { using Type = FunctionToUInt64; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeInt8> { using Type = FunctionToInt8; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeInt16> { using Type = FunctionToInt16; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeInt32> { using Type = FunctionToInt32; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeInt64> { using Type = FunctionToInt64; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeFloat32> { using Type = FunctionToFloat32; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeFloat64> { using Type = FunctionToFloat64; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeDate> { using Type = FunctionToDate; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeDateTime> { using Type = FunctionToDateTime; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeString> { using Type = FunctionToString; };
|
|
|
|
|
template <> struct FunctionTo<DataTypeFixedString> { using Type = FunctionToFixedString; };
|
|
|
|
|
template <typename FieldType> struct FunctionTo<DataTypeEnum<FieldType>>
|
|
|
|
|
: FunctionTo<typename DataTypeFromFieldType<FieldType>::Type>
|
|
|
|
|
{
|
|
|
|
|
};
|
2015-12-25 15:43:38 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
class FunctionCast final : public IFunction
|
2015-12-23 08:11:11 +00:00
|
|
|
|
{
|
2015-12-23 09:52:21 +00:00
|
|
|
|
using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t)>;
|
2015-12-23 08:11:11 +00:00
|
|
|
|
const Context & context;
|
2015-12-23 09:52:21 +00:00
|
|
|
|
WrapperType wrapper_function;
|
2015-12-29 13:51:12 +00:00
|
|
|
|
std::function<Monotonicity(const IDataType &, const Field &, const Field &)> monotonicity_for_range;
|
2015-12-23 08:11:11 +00:00
|
|
|
|
|
|
|
|
|
FunctionCast(const Context & context) : context(context) {}
|
|
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
|
template <typename DataType> auto createWrapper(const DataTypePtr & from_type, const DataType * const)
|
2015-12-23 08:11:11 +00:00
|
|
|
|
{
|
2015-12-29 13:51:12 +00:00
|
|
|
|
using FunctionType = typename FunctionTo<DataType>::Type;
|
|
|
|
|
|
2015-12-23 08:11:11 +00:00
|
|
|
|
std::shared_ptr<FunctionType> function{static_cast<FunctionType *>(FunctionType::create(context))};
|
|
|
|
|
|
|
|
|
|
/// Check conversion using underlying function
|
2015-12-23 09:52:21 +00:00
|
|
|
|
(void) function->getReturnType({ from_type });
|
2015-12-23 08:11:11 +00:00
|
|
|
|
|
|
|
|
|
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result) {
|
2015-12-25 15:43:38 +00:00
|
|
|
|
function->execute(block, arguments, result);
|
2015-12-23 08:11:11 +00:00
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-23 09:52:21 +00:00
|
|
|
|
static auto createFixedStringWrapper(const DataTypePtr & from_type, const size_t N)
|
2015-12-23 08:11:11 +00:00
|
|
|
|
{
|
2015-12-23 09:52:21 +00:00
|
|
|
|
if (!typeid_cast<const DataTypeString *>(from_type.get()) &&
|
|
|
|
|
!typeid_cast<const DataTypeFixedString *>(from_type.get()))
|
2015-12-23 08:11:11 +00:00
|
|
|
|
throw Exception{
|
|
|
|
|
"CAST AS FixedString is only implemented for types String and FixedString",
|
|
|
|
|
ErrorCodes::NOT_IMPLEMENTED
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result)
|
|
|
|
|
{
|
2015-12-24 17:14:10 +00:00
|
|
|
|
FunctionToFixedString::execute(block, arguments, result, N);
|
2015-12-23 08:11:11 +00:00
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-25 14:45:58 +00:00
|
|
|
|
auto createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray * to_type)
|
2015-12-23 08:11:11 +00:00
|
|
|
|
{
|
|
|
|
|
DataTypePtr from_nested_type, to_nested_type;
|
2015-12-25 14:45:58 +00:00
|
|
|
|
auto from_type = typeid_cast<const DataTypeArray *>(from_type_untyped.get());
|
2015-12-23 08:11:11 +00:00
|
|
|
|
|
|
|
|
|
/// get the most nested type
|
|
|
|
|
while (from_type && to_type)
|
|
|
|
|
{
|
|
|
|
|
from_nested_type = from_type->getNestedType();
|
|
|
|
|
to_nested_type = to_type->getNestedType();
|
|
|
|
|
|
|
|
|
|
from_type = typeid_cast<const DataTypeArray *>(from_nested_type.get());
|
|
|
|
|
to_type = typeid_cast<const DataTypeArray *>(to_nested_type.get());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// both from_type and to_type should be nullptr now is array types had same dimensions
|
|
|
|
|
if (from_type || to_type)
|
|
|
|
|
throw Exception{
|
2015-12-23 09:52:21 +00:00
|
|
|
|
"CAST AS Array can only be performed between same-dimensional array types",
|
2015-12-23 08:11:11 +00:00
|
|
|
|
ErrorCodes::TYPE_MISMATCH
|
|
|
|
|
};
|
|
|
|
|
|
2015-12-23 09:52:21 +00:00
|
|
|
|
/// Prepare nested type conversion
|
|
|
|
|
const auto nested_function = prepare(from_nested_type, to_nested_type.get());
|
2015-12-23 08:11:11 +00:00
|
|
|
|
|
|
|
|
|
return [nested_function, from_nested_type, to_nested_type] (
|
|
|
|
|
Block & block, const ColumnNumbers & arguments, const size_t result)
|
|
|
|
|
{
|
2015-12-25 15:43:38 +00:00
|
|
|
|
auto array_arg = block.getByPosition(arguments.front());
|
2015-12-24 17:14:10 +00:00
|
|
|
|
|
2015-12-23 08:11:11 +00:00
|
|
|
|
/// @todo add const variant which retains array constness
|
2015-12-24 17:14:10 +00:00
|
|
|
|
if (const auto col_const_array = typeid_cast<const ColumnConstArray *>(array_arg.column.get()))
|
|
|
|
|
array_arg.column = col_const_array->convertToFullColumn();
|
2015-12-23 08:11:11 +00:00
|
|
|
|
|
|
|
|
|
if (auto col_array = typeid_cast<const ColumnArray *>(array_arg.column.get()))
|
|
|
|
|
{
|
|
|
|
|
auto res = new ColumnArray{nullptr, col_array->getOffsetsColumn()};
|
|
|
|
|
block.getByPosition(result).column = res;
|
|
|
|
|
|
|
|
|
|
/// get the most nested column
|
|
|
|
|
while (const auto nested_col_array = typeid_cast<const ColumnArray *>(col_array->getDataPtr().get()))
|
|
|
|
|
{
|
|
|
|
|
/// create new level of array, copy offsets
|
|
|
|
|
res->getDataPtr() = new ColumnArray{nullptr, nested_col_array->getOffsetsColumn()};
|
|
|
|
|
|
|
|
|
|
res = static_cast<ColumnArray *>(res->getDataPtr().get());
|
|
|
|
|
col_array = nested_col_array;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// create block for converting nested column containing original and result columns
|
|
|
|
|
Block nested_block{
|
|
|
|
|
{ col_array->getDataPtr(), from_nested_type, "" },
|
|
|
|
|
{ nullptr, to_nested_type, "" }
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
const auto nested_result = 1;
|
|
|
|
|
/// convert nested column
|
|
|
|
|
nested_function(nested_block, {0 }, nested_result);
|
|
|
|
|
|
|
|
|
|
/// set converted nested column to result
|
|
|
|
|
res->getDataPtr() = nested_block.getByPosition(nested_result).column;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception{
|
2015-12-23 09:52:21 +00:00
|
|
|
|
"Illegal column " + array_arg.column->getName() + " for function CAST AS Array",
|
2015-12-23 08:11:11 +00:00
|
|
|
|
ErrorCodes::LOGICAL_ERROR
|
|
|
|
|
};
|
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-25 14:45:58 +00:00
|
|
|
|
auto createTupleWrapper(const DataTypePtr & from_type_untyped, const DataTypeTuple * to_type)
|
2015-12-23 09:52:21 +00:00
|
|
|
|
{
|
2015-12-25 14:45:58 +00:00
|
|
|
|
const auto from_type = typeid_cast<const DataTypeTuple *>(from_type_untyped.get());
|
2015-12-23 09:52:21 +00:00
|
|
|
|
if (!from_type)
|
|
|
|
|
throw Exception{
|
|
|
|
|
"CAST AS Tuple can only be performed between tuple types.\nLeft type: " + from_type_untyped->getName() +
|
|
|
|
|
", right type: " + to_type->getName(),
|
|
|
|
|
ErrorCodes::TYPE_MISMATCH
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
if (from_type->getElements().size() != to_type->getElements().size())
|
|
|
|
|
throw Exception{
|
|
|
|
|
"CAST AS Tuple can only be performed between tuple types with the same number of elements.\n"
|
|
|
|
|
"Left type: " + from_type->getName() + ", right type: " + to_type->getName(),
|
|
|
|
|
ErrorCodes::TYPE_MISMATCH
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
const auto & from_element_types = from_type->getElements();
|
|
|
|
|
const auto & to_element_types = to_type->getElements();
|
|
|
|
|
std::vector<WrapperType> element_wrappers;
|
|
|
|
|
element_wrappers.reserve(from_element_types.size());
|
|
|
|
|
|
|
|
|
|
/// Create conversion wrapper for each element in tuple
|
|
|
|
|
for (const auto & idx_type : ext::enumerate(from_type->getElements()))
|
|
|
|
|
element_wrappers.push_back(prepare(idx_type.second, to_element_types[idx_type.first].get()));
|
|
|
|
|
|
|
|
|
|
std::shared_ptr<FunctionTuple> function_tuple{static_cast<FunctionTuple *>(FunctionTuple::create(context))};
|
|
|
|
|
return [element_wrappers, function_tuple, from_element_types, to_element_types]
|
|
|
|
|
(Block & block, const ColumnNumbers & arguments, const size_t result)
|
|
|
|
|
{
|
2015-12-24 14:31:41 +00:00
|
|
|
|
const auto col = block.getByPosition(arguments.front()).column.get();
|
|
|
|
|
|
2015-12-23 09:52:21 +00:00
|
|
|
|
/// copy tuple elements to a separate block
|
2015-12-24 14:31:41 +00:00
|
|
|
|
Block element_block;
|
|
|
|
|
|
|
|
|
|
/// @todo retain constness
|
|
|
|
|
if (const auto column_tuple = typeid_cast<const ColumnTuple *>(col))
|
|
|
|
|
element_block = column_tuple->getData();
|
|
|
|
|
else if (const auto column_const_tuple = typeid_cast<const ColumnConstTuple *>(col))
|
|
|
|
|
element_block = static_cast<const ColumnTuple &>(*column_const_tuple->convertToFullColumn()).getData();
|
2015-12-23 09:52:21 +00:00
|
|
|
|
|
|
|
|
|
/// create columns for converted elements
|
|
|
|
|
for (const auto & to_element_type : to_element_types)
|
|
|
|
|
element_block.insert({ nullptr, to_element_type, "" });
|
|
|
|
|
|
|
|
|
|
/// store position for converted tuple
|
|
|
|
|
const auto converted_tuple_pos = element_block.columns();
|
|
|
|
|
|
|
|
|
|
/// insert column for converted tuple
|
|
|
|
|
element_block.insert({ nullptr, new DataTypeTuple{to_element_types}, "" });
|
|
|
|
|
|
|
|
|
|
const auto converted_element_offset = from_element_types.size();
|
|
|
|
|
|
|
|
|
|
/// invoke conversion for each element
|
|
|
|
|
for (const auto & idx_element_wrapper : ext::enumerate(element_wrappers))
|
|
|
|
|
idx_element_wrapper.second(element_block, { idx_element_wrapper.first },
|
2015-12-24 14:31:41 +00:00
|
|
|
|
converted_element_offset + idx_element_wrapper.first);
|
2015-12-23 09:52:21 +00:00
|
|
|
|
|
|
|
|
|
/// form tuple from converted elements using FunctionTuple
|
|
|
|
|
function_tuple->execute(element_block,
|
|
|
|
|
ext::collection_cast<ColumnNumbers>(ext::range(converted_element_offset, 2 * converted_element_offset)),
|
|
|
|
|
converted_tuple_pos);
|
|
|
|
|
|
|
|
|
|
/// copy FunctionTuple's result from element_block to resulting block
|
|
|
|
|
block.getByPosition(result).column = element_block.getByPosition(converted_tuple_pos).column;
|
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
|
template <typename FieldType>
|
2015-12-25 14:45:58 +00:00
|
|
|
|
WrapperType createEnumWrapper(const DataTypePtr & from_type, const DataTypeEnum<FieldType> * to_type)
|
|
|
|
|
{
|
|
|
|
|
using EnumType = DataTypeEnum<FieldType>;
|
2015-12-29 13:51:12 +00:00
|
|
|
|
using Function = typename FunctionTo<EnumType>::Type;
|
2015-12-25 14:45:58 +00:00
|
|
|
|
|
|
|
|
|
if (const auto from_enum8 = typeid_cast<const DataTypeEnum8 *>(from_type.get()))
|
2015-12-25 15:43:38 +00:00
|
|
|
|
checkEnumToEnumConversion(from_enum8, to_type);
|
2015-12-25 14:45:58 +00:00
|
|
|
|
else if (const auto from_enum16 = typeid_cast<const DataTypeEnum16 *>(from_type.get()))
|
2015-12-25 15:43:38 +00:00
|
|
|
|
checkEnumToEnumConversion(from_enum16, to_type);
|
2015-12-25 14:45:58 +00:00
|
|
|
|
|
2015-12-25 15:43:38 +00:00
|
|
|
|
if (typeid_cast<const DataTypeString *>(from_type.get()))
|
|
|
|
|
return createStringToEnumWrapper<ColumnString, EnumType>();
|
|
|
|
|
else if (typeid_cast<const DataTypeFixedString *>(from_type.get()))
|
|
|
|
|
return createStringToEnumWrapper<ColumnFixedString, EnumType>();
|
2015-12-25 14:45:58 +00:00
|
|
|
|
else if (from_type->behavesAsNumber())
|
|
|
|
|
{
|
|
|
|
|
std::shared_ptr<Function> function{static_cast<Function *>(Function::create(context))};
|
|
|
|
|
|
|
|
|
|
/// Check conversion using underlying function
|
|
|
|
|
(void) function->getReturnType({ from_type });
|
|
|
|
|
|
|
|
|
|
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result) {
|
2015-12-25 15:43:38 +00:00
|
|
|
|
function->execute(block, arguments, result);
|
2015-12-25 14:45:58 +00:00
|
|
|
|
};
|
|
|
|
|
}
|
2015-12-25 15:43:38 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception{
|
|
|
|
|
"Conversion from " + from_type->getName() + " to " + to_type->getName() +
|
|
|
|
|
" is not supported",
|
|
|
|
|
ErrorCodes::CANNOT_CONVERT_TYPE
|
|
|
|
|
};
|
|
|
|
|
}
|
2015-12-25 14:45:58 +00:00
|
|
|
|
|
2015-12-25 15:43:38 +00:00
|
|
|
|
template <typename EnumTypeFrom, typename EnumTypeTo>
|
|
|
|
|
void checkEnumToEnumConversion(const EnumTypeFrom * const from_type, const EnumTypeTo * const to_type)
|
|
|
|
|
{
|
|
|
|
|
const auto & from_values = from_type->getValues();
|
|
|
|
|
const auto & to_values = to_type->getValues();
|
|
|
|
|
|
|
|
|
|
using ValueType = std::common_type_t<typename EnumTypeFrom::FieldType, typename EnumTypeTo::FieldType>;
|
|
|
|
|
using NameValuePair = std::pair<std::string, ValueType>;
|
2015-12-29 12:57:11 +00:00
|
|
|
|
using EnumValues = std::vector<NameValuePair>;
|
|
|
|
|
|
|
|
|
|
EnumValues name_intersection;
|
2015-12-25 15:43:38 +00:00
|
|
|
|
std::set_intersection(std::begin(from_values), std::end(from_values),
|
2015-12-29 12:57:11 +00:00
|
|
|
|
std::begin(to_values), std::end(to_values), std::back_inserter(name_intersection),
|
|
|
|
|
[] (auto && from, auto && to) { return from.first < to.first; });
|
2015-12-25 15:43:38 +00:00
|
|
|
|
|
2015-12-29 12:57:11 +00:00
|
|
|
|
for (const auto & name_value : name_intersection)
|
|
|
|
|
{
|
|
|
|
|
const auto & old_value = name_value.second;
|
|
|
|
|
const auto & new_value = to_type->getValue(name_value.first);
|
|
|
|
|
if (old_value != new_value)
|
|
|
|
|
throw Exception{
|
|
|
|
|
"Enum conversion changes value for element '" + name_value.first +
|
|
|
|
|
"' from " + toString(old_value) + " to " + toString(new_value),
|
|
|
|
|
ErrorCodes::CANNOT_CONVERT_TYPE
|
|
|
|
|
};
|
|
|
|
|
}
|
2015-12-25 15:43:38 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
template <typename ColumnStringType, typename EnumType>
|
|
|
|
|
auto createStringToEnumWrapper()
|
|
|
|
|
{
|
2016-02-01 18:32:34 +00:00
|
|
|
|
return [] (Block & block, const ColumnNumbers & arguments, const size_t result)
|
|
|
|
|
{
|
2015-12-25 15:43:38 +00:00
|
|
|
|
const auto first_col = block.getByPosition(arguments.front()).column.get();
|
|
|
|
|
|
|
|
|
|
auto & col_with_type_and_name = block.getByPosition(result);
|
|
|
|
|
auto & result_col = col_with_type_and_name.column;
|
|
|
|
|
const auto & result_type = typeid_cast<EnumType &>(*col_with_type_and_name.type);
|
|
|
|
|
|
|
|
|
|
if (const auto col = typeid_cast<const ColumnStringType *>(first_col))
|
|
|
|
|
{
|
|
|
|
|
const auto size = col->size();
|
|
|
|
|
|
2016-02-01 18:32:34 +00:00
|
|
|
|
auto res = result_type.createColumn();
|
|
|
|
|
auto & out_data = static_cast<typename EnumType::ColumnType &>(*res).getData();
|
2015-12-25 15:43:38 +00:00
|
|
|
|
out_data.resize(size);
|
|
|
|
|
|
|
|
|
|
for (const auto i : ext::range(0, size))
|
2016-02-01 18:32:34 +00:00
|
|
|
|
out_data[i] = result_type.getValue(col->getDataAt(i));
|
2015-12-25 15:43:38 +00:00
|
|
|
|
|
|
|
|
|
result_col = res;
|
|
|
|
|
}
|
|
|
|
|
else if (const auto const_col = typeid_cast<const ColumnConstString *>(first_col))
|
|
|
|
|
{
|
|
|
|
|
result_col = result_type.createConstColumn(const_col->size(),
|
|
|
|
|
nearestFieldType(result_type.getValue(const_col->getData())));
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception{
|
|
|
|
|
"Unexpected column " + first_col->getName() + " as first argument of function " +
|
|
|
|
|
name,
|
|
|
|
|
ErrorCodes::LOGICAL_ERROR
|
|
|
|
|
};
|
2015-12-25 14:45:58 +00:00
|
|
|
|
};
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
|
WrapperType prepare(const DataTypePtr & from_type, const IDataType * const to_type)
|
|
|
|
|
{
|
|
|
|
|
if (const auto to_actual_type = typeid_cast<const DataTypeUInt8 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeUInt16 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeUInt32 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeUInt64 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeInt8 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeInt16 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeInt32 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeInt64 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeFloat32 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeFloat64 *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeDate *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeDateTime *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
|
|
|
|
else if (const auto to_actual_type = typeid_cast<const DataTypeString *>(to_type))
|
|
|
|
|
return createWrapper(from_type, to_actual_type);
|
2015-12-23 09:52:21 +00:00
|
|
|
|
else if (const auto type_fixed_string = typeid_cast<const DataTypeFixedString *>(to_type))
|
|
|
|
|
return createFixedStringWrapper(from_type, type_fixed_string->getN());
|
|
|
|
|
else if (const auto type_array = typeid_cast<const DataTypeArray *>(to_type))
|
|
|
|
|
return createArrayWrapper(from_type, type_array);
|
|
|
|
|
else if (const auto type_tuple = typeid_cast<const DataTypeTuple *>(to_type))
|
|
|
|
|
return createTupleWrapper(from_type, type_tuple);
|
2015-12-25 14:45:58 +00:00
|
|
|
|
else if (const auto type_enum = typeid_cast<const DataTypeEnum8 *>(to_type))
|
2015-12-29 13:51:12 +00:00
|
|
|
|
return createEnumWrapper(from_type, type_enum);
|
2015-12-25 14:45:58 +00:00
|
|
|
|
else if (const auto type_enum = typeid_cast<const DataTypeEnum16 *>(to_type))
|
2015-12-29 13:51:12 +00:00
|
|
|
|
return createEnumWrapper(from_type, type_enum);
|
2015-12-25 14:45:58 +00:00
|
|
|
|
|
|
|
|
|
throw Exception{
|
|
|
|
|
"Conversion from " + from_type->getName() + " to " + to_type->getName() +
|
|
|
|
|
" is not supported",
|
|
|
|
|
ErrorCodes::CANNOT_CONVERT_TYPE
|
|
|
|
|
};
|
2015-12-23 09:52:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
|
template <typename DataType> static auto monotonicityForType(const DataType * const)
|
|
|
|
|
{
|
|
|
|
|
return FunctionTo<DataType>::Type::Monotonic::get;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void prepareMonotonicityInformation(const DataTypePtr & from_type, const IDataType * to_type)
|
|
|
|
|
{
|
|
|
|
|
if (const auto type = typeid_cast<const DataTypeUInt8 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeUInt16 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeUInt32 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeUInt64 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeInt8 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeInt16 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeInt32 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeInt64 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeFloat32 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeFloat64 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeDate *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeDateTime *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeString *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (from_type->isNumeric())
|
|
|
|
|
{
|
|
|
|
|
if (const auto type = typeid_cast<const DataTypeEnum8 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
else if (const auto type = typeid_cast<const DataTypeEnum16 *>(to_type))
|
|
|
|
|
monotonicity_for_range = monotonicityForType(type);
|
|
|
|
|
}
|
|
|
|
|
/// other types like FixedString, Array and Tuple have no monotonicity defined
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-23 08:11:11 +00:00
|
|
|
|
public:
|
|
|
|
|
static constexpr auto name = "CAST";
|
|
|
|
|
static IFunction * create(const Context & context) { return new FunctionCast{context}; }
|
|
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
|
|
void getReturnTypeAndPrerequisites(
|
|
|
|
|
const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type,
|
|
|
|
|
std::vector<ExpressionAction> & out_prerequisites) override
|
|
|
|
|
{
|
|
|
|
|
if (arguments.size() != 2)
|
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
|
+ toString(arguments.size()) + ", should be 2.",
|
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
const auto type_col = typeid_cast<const ColumnConstString *>(arguments.back().column.get());
|
|
|
|
|
if (!type_col)
|
|
|
|
|
throw Exception("Second argument to " + getName() + " must be a constant string describing type",
|
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
2015-12-23 09:52:21 +00:00
|
|
|
|
out_return_type = DataTypeFactory::instance().get(type_col->getData());
|
|
|
|
|
|
|
|
|
|
wrapper_function = prepare(arguments.front().type, out_return_type.get());
|
2015-12-29 13:51:12 +00:00
|
|
|
|
|
|
|
|
|
prepareMonotonicityInformation(arguments.front().type, out_return_type.get());
|
2015-12-23 08:11:11 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-12-25 15:43:38 +00:00
|
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
2015-12-23 08:11:11 +00:00
|
|
|
|
{
|
2015-12-25 15:43:38 +00:00
|
|
|
|
/// drop second argument, pass others
|
|
|
|
|
ColumnNumbers new_arguments{arguments.front()};
|
|
|
|
|
if (arguments.size() > 2)
|
|
|
|
|
new_arguments.insert(std::end(new_arguments), std::next(std::begin(arguments), 2), std::end(arguments));
|
|
|
|
|
|
2015-12-28 12:43:08 +00:00
|
|
|
|
wrapper_function(block, new_arguments, result);
|
2015-12-23 08:11:11 +00:00
|
|
|
|
}
|
2015-12-29 13:51:12 +00:00
|
|
|
|
|
|
|
|
|
bool hasInformationAboutMonotonicity() const override
|
|
|
|
|
{
|
|
|
|
|
return static_cast<bool>(monotonicity_for_range);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
|
|
|
|
{
|
|
|
|
|
return monotonicity_for_range(type, left, right);
|
|
|
|
|
}
|
2015-12-23 08:11:11 +00:00
|
|
|
|
};
|
|
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
|
}
|