2011-10-15 23:40:56 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-06-06 17:18:32 +00:00
|
|
|
#include <ext/enumerate.h>
|
|
|
|
#include <ext/collection_cast.h>
|
|
|
|
#include <ext/range.h>
|
2016-01-13 00:32:59 +00:00
|
|
|
#include <type_traits>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/WriteBufferFromVector.h>
|
|
|
|
#include <IO/ReadBufferFromMemory.h>
|
2017-07-21 06:35:58 +00:00
|
|
|
#include <IO/Operators.h>
|
2018-02-11 23:57:07 +00:00
|
|
|
#include <IO/parseDateTimeBestEffort.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-08-21 18:25:38 +00:00
|
|
|
#include <DataTypes/DataTypesDecimal.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypeFixedString.h>
|
|
|
|
#include <DataTypes/DataTypeDate.h>
|
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
|
|
|
#include <DataTypes/DataTypeEnum.h>
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2017-12-09 20:56:53 +00:00
|
|
|
#include <DataTypes/DataTypeNothing.h>
|
2017-07-04 10:42:53 +00:00
|
|
|
#include <DataTypes/DataTypeUUID.h>
|
2017-10-30 02:18:06 +00:00
|
|
|
#include <DataTypes/DataTypeInterval.h>
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <Formats/FormatSettings.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnFixedString.h>
|
|
|
|
#include <Columns/ColumnConst.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
#include <Columns/ColumnTuple.h>
|
2018-02-22 21:20:02 +00:00
|
|
|
#include <Columns/ColumnsCommon.h>
|
2017-11-24 13:55:31 +00:00
|
|
|
#include <Common/FieldVisitors.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
|
|
|
#include <Functions/IFunction.h>
|
|
|
|
#include <Functions/FunctionsMiscellaneous.h>
|
2017-07-21 06:35:58 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2018-09-26 00:31:40 +00:00
|
|
|
#include <Functions/DateTimeTransforms.h>
|
2018-09-27 15:55:22 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
|
|
|
#include <Columns/ColumnLowCardinality.h>
|
2011-10-15 23:40:56 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-12 02:21:15 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
|
|
|
extern const int CANNOT_PARSE_NUMBER;
|
|
|
|
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
|
|
|
|
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
|
|
|
extern const int CANNOT_PARSE_QUOTED_STRING;
|
|
|
|
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
|
|
|
extern const int CANNOT_PARSE_DATE;
|
|
|
|
extern const int CANNOT_PARSE_DATETIME;
|
|
|
|
extern const int CANNOT_PARSE_TEXT;
|
2017-06-15 09:12:32 +00:00
|
|
|
extern const int CANNOT_PARSE_UUID;
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int TOO_LARGE_STRING_SIZE;
|
2018-12-07 03:20:27 +00:00
|
|
|
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
2018-02-18 02:22:32 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int TYPE_MISMATCH;
|
|
|
|
extern const int CANNOT_CONVERT_TYPE;
|
|
|
|
extern const int ILLEGAL_COLUMN;
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
extern const int NOT_IMPLEMENTED;
|
2018-02-22 21:20:02 +00:00
|
|
|
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
|
2016-01-12 02:21:15 +00:00
|
|
|
}
|
|
|
|
|
2016-08-30 16:14:05 +00:00
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
/** Type conversion functions.
|
|
|
|
* toType - conversion in "natural way";
|
2011-10-15 23:40:56 +00:00
|
|
|
*/
|
|
|
|
|
2018-08-21 18:25:38 +00:00
|
|
|
inline UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column)
|
|
|
|
{
|
|
|
|
const auto * arg_type = named_column.type.get();
|
|
|
|
bool ok = checkAndGetDataType<DataTypeUInt64>(arg_type)
|
|
|
|
|| checkAndGetDataType<DataTypeUInt32>(arg_type)
|
|
|
|
|| checkAndGetDataType<DataTypeUInt16>(arg_type)
|
|
|
|
|| checkAndGetDataType<DataTypeUInt8>(arg_type);
|
|
|
|
if (!ok)
|
|
|
|
throw Exception("Illegal type of toDecimal() scale " + named_column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
Field field;
|
|
|
|
named_column.column->get(0, field);
|
|
|
|
return field.get<UInt32>();
|
|
|
|
}
|
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
/** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment.
|
|
|
|
* (Date is represented internally as number of days from some day; DateTime - as unix timestamp)
|
2011-10-15 23:40:56 +00:00
|
|
|
*/
|
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
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using FromFieldType = typename FromDataType::FieldType;
|
|
|
|
using ToFieldType = typename ToDataType::FieldType;
|
|
|
|
|
2018-08-31 08:59:21 +00:00
|
|
|
template <typename Additions = void *>
|
2018-12-27 00:25:47 +00:00
|
|
|
static void NO_SANITIZE_UNDEFINED execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/,
|
2018-08-31 08:59:21 +00:00
|
|
|
Additions additions [[maybe_unused]] = Additions())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-08-21 18:25:38 +00:00
|
|
|
const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]);
|
|
|
|
|
2018-08-27 16:16:16 +00:00
|
|
|
using ColVecFrom = std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>, ColumnVector<FromFieldType>>;
|
|
|
|
using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
|
|
|
|
|
2018-09-25 16:03:50 +00:00
|
|
|
if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
|
|
|
|
{
|
|
|
|
if constexpr (!IsDataTypeDecimalOrNumber<FromDataType> || !IsDataTypeDecimalOrNumber<ToDataType>)
|
|
|
|
throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
|
2018-08-27 16:16:16 +00:00
|
|
|
if (const ColVecFrom * col_from = checkAndGetColumn<ColVecFrom>(named_from.column.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-08-27 16:16:16 +00:00
|
|
|
typename ColVecTo::MutablePtr col_to = nullptr;
|
2018-09-10 13:52:18 +00:00
|
|
|
if constexpr (IsDataTypeDecimal<ToDataType>)
|
2018-08-21 18:25:38 +00:00
|
|
|
{
|
2018-08-31 08:59:21 +00:00
|
|
|
UInt32 scale = additions;
|
2018-08-27 16:16:16 +00:00
|
|
|
col_to = ColVecTo::create(0, scale);
|
2018-08-21 18:25:38 +00:00
|
|
|
}
|
2018-08-27 16:16:16 +00:00
|
|
|
else
|
|
|
|
col_to = ColVecTo::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-08-27 16:16:16 +00:00
|
|
|
const auto & vec_from = col_from->getData();
|
|
|
|
auto & vec_to = col_to->getData();
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = vec_from.size();
|
|
|
|
vec_to.resize(size);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2018-08-21 18:25:38 +00:00
|
|
|
{
|
2018-09-25 16:03:50 +00:00
|
|
|
if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
|
|
|
|
{
|
|
|
|
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
|
|
|
vec_to[i] = convertDecimals<FromDataType, ToDataType>(vec_from[i], vec_from.getScale(), vec_to.getScale());
|
|
|
|
else if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeNumber<ToDataType>)
|
|
|
|
vec_to[i] = convertFromDecimal<FromDataType, ToDataType>(vec_from[i], vec_from.getScale());
|
|
|
|
else if constexpr (IsDataTypeNumber<FromDataType> && IsDataTypeDecimal<ToDataType>)
|
|
|
|
vec_to[i] = convertToDecimal<FromDataType, ToDataType>(vec_from[i], vec_to.getScale());
|
|
|
|
}
|
2018-08-21 18:25:38 +00:00
|
|
|
else
|
|
|
|
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
|
|
|
|
}
|
2017-12-16 04:59:32 +00:00
|
|
|
|
|
|
|
block.getByPosition(result).column = std::move(col_to);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2018-08-21 18:25:38 +00:00
|
|
|
throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
2011-10-16 01:57:10 +00:00
|
|
|
};
|
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
/** Conversion of Date to DateTime: adding 00:00:00 time component.
|
2011-10-16 01:57:10 +00:00
|
|
|
*/
|
2017-01-22 08:33:16 +00:00
|
|
|
struct ToDateTimeImpl
|
2011-10-16 01:57:10 +00:00
|
|
|
{
|
2017-10-29 04:18:48 +00:00
|
|
|
static constexpr auto name = "toDateTime";
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
|
|
|
{
|
2018-05-25 13:29:15 +00:00
|
|
|
return time_zone.fromDayNum(DayNum(d));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2011-10-16 01:57:10 +00:00
|
|
|
};
|
|
|
|
|
2017-01-22 08:33:16 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<UInt16, UInt32, ToDateTimeImpl> {};
|
2015-10-22 15:31:42 +00:00
|
|
|
|
2015-03-16 04:44:53 +00:00
|
|
|
|
2017-01-22 08:33:16 +00:00
|
|
|
/// Implementation of toDate function.
|
2015-03-16 04:44:53 +00:00
|
|
|
|
2015-10-22 15:31:42 +00:00
|
|
|
template <typename FromType, typename ToType>
|
|
|
|
struct ToDateTransform32Or64
|
|
|
|
{
|
2017-10-29 04:18:48 +00:00
|
|
|
static constexpr auto name = "toDate";
|
|
|
|
|
2018-12-27 00:40:10 +00:00
|
|
|
static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return (from < 0xFFFF) ? from : time_zone.toDayNum(from);
|
|
|
|
}
|
2015-10-22 15:31:42 +00:00
|
|
|
};
|
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
/** Conversion of DateTime to Date: throw off time component.
|
2015-10-22 15:31:42 +00:00
|
|
|
*/
|
2016-07-30 04:39:51 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<UInt32, UInt16, ToDateImpl> {};
|
2016-07-30 04:39:51 +00:00
|
|
|
|
2016-09-04 00:42:24 +00:00
|
|
|
/** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date.
|
2016-07-30 04:39:51 +00:00
|
|
|
* If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp.
|
|
|
|
* It's a bit illogical, as we actually have two functions in one.
|
|
|
|
* But allows to support frequent case,
|
|
|
|
* when user write toDate(UInt32), expecting conversion of unix timestamp to Date.
|
|
|
|
* (otherwise such usage would be frequent mistake).
|
2015-10-22 15:31:42 +00:00
|
|
|
*/
|
2016-07-30 04:39:51 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<UInt32, UInt16, ToDateTransform32Or64<UInt32, UInt16>> {};
|
2016-07-30 04:39:51 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<UInt64, UInt16, ToDateTransform32Or64<UInt64, UInt16>> {};
|
2016-07-30 04:39:51 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<Int32, UInt16, ToDateTransform32Or64<Int32, UInt16>> {};
|
2016-07-30 04:39:51 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<Int64, UInt16, ToDateTransform32Or64<Int64, UInt16>> {};
|
2016-09-04 00:42:24 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<Float32, UInt16, ToDateTransform32Or64<Float32, UInt16>> {};
|
2016-09-04 00:42:24 +00:00
|
|
|
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name>
|
2017-10-29 04:18:48 +00:00
|
|
|
: DateTimeTransformImpl<Float64, UInt16, ToDateTransform32Or64<Float64, UInt16>> {};
|
2015-03-16 04:44:53 +00:00
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
|
|
|
|
/** Transformation of numbers, dates, datetimes to strings: through formatting.
|
2011-10-16 01:57:10 +00:00
|
|
|
*/
|
2016-07-30 04:39:51 +00:00
|
|
|
template <typename DataType>
|
|
|
|
struct FormatImpl
|
2015-12-22 13:07:40 +00:00
|
|
|
{
|
2017-12-02 02:47:12 +00:00
|
|
|
static void execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType *, const DateLUTImpl *)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
writeText(x, wb);
|
|
|
|
}
|
2015-12-22 13:07:40 +00:00
|
|
|
};
|
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
template <>
|
|
|
|
struct FormatImpl<DataTypeDate>
|
2015-12-22 13:07:40 +00:00
|
|
|
{
|
2017-12-02 02:47:12 +00:00
|
|
|
static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-05-25 13:29:15 +00:00
|
|
|
writeDateText(DayNum(x), wb);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-12-22 13:07:40 +00:00
|
|
|
};
|
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
template <>
|
|
|
|
struct FormatImpl<DataTypeDateTime>
|
2015-12-22 13:07:40 +00:00
|
|
|
{
|
2017-12-02 02:47:12 +00:00
|
|
|
static void execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime *, const DateLUTImpl * time_zone)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
writeDateTimeText(x, wb, *time_zone);
|
|
|
|
}
|
2015-12-22 13:07:40 +00:00
|
|
|
};
|
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
template <typename FieldType>
|
|
|
|
struct FormatImpl<DataTypeEnum<FieldType>>
|
2015-12-22 13:07:40 +00:00
|
|
|
{
|
2017-12-02 02:47:12 +00:00
|
|
|
static void execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum<FieldType> * type, const DateLUTImpl *)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
writeString(type->getNameForValue(x), wb);
|
|
|
|
}
|
2015-12-22 13:07:40 +00:00
|
|
|
};
|
2011-10-16 03:05:15 +00:00
|
|
|
|
2018-08-21 18:25:38 +00:00
|
|
|
template <typename FieldType>
|
|
|
|
struct FormatImpl<DataTypeDecimal<FieldType>>
|
|
|
|
{
|
|
|
|
static void execute(const FieldType x, WriteBuffer & wb, const DataTypeDecimal<FieldType> * type, const DateLUTImpl *)
|
|
|
|
{
|
2018-08-23 19:11:31 +00:00
|
|
|
writeText(x, type->getScale(), wb);
|
2018-08-21 18:25:38 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2015-12-30 11:53:12 +00:00
|
|
|
|
|
|
|
/// DataTypeEnum<T> to DataType<T> free conversion
|
|
|
|
template <typename FieldType, typename Name>
|
2017-03-12 10:13:45 +00:00
|
|
|
struct ConvertImpl<DataTypeEnum<FieldType>, DataTypeNumber<FieldType>, Name>
|
2015-12-30 11:53:12 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-12-30 11:53:12 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2011-10-16 01:57:10 +00:00
|
|
|
template <typename FromDataType, typename Name>
|
2017-12-25 04:01:46 +00:00
|
|
|
struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType, DataTypeString>, DataTypeString>, Name>
|
2011-10-16 01:57:10 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using FromFieldType = typename FromDataType::FieldType;
|
2018-09-12 17:50:51 +00:00
|
|
|
using ColVecType = std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>, ColumnVector<FromFieldType>>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto & col_with_type_and_name = block.getByPosition(arguments[0]);
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
|
|
|
|
|
|
|
|
const DateLUTImpl * time_zone = nullptr;
|
|
|
|
|
|
|
|
/// For argument of DateTime type, second argument with time zone could be specified.
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
2017-12-22 01:54:29 +00:00
|
|
|
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-12 17:50:51 +00:00
|
|
|
if (const auto col_from = checkAndGetColumn<ColVecType>(col_with_type_and_name.column.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-14 01:43:19 +00:00
|
|
|
auto col_to = ColumnString::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-12 17:50:51 +00:00
|
|
|
const typename ColVecType::Container & vec_from = col_from->getData();
|
2018-11-25 00:08:50 +00:00
|
|
|
ColumnString::Chars & data_to = col_to->getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = vec_from.size();
|
2017-11-17 23:45:49 +00:00
|
|
|
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (std::is_same_v<FromDataType, DataTypeDate>)
|
2017-11-17 23:45:49 +00:00
|
|
|
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
2017-12-25 04:01:46 +00:00
|
|
|
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
|
2017-11-17 23:45:49 +00:00
|
|
|
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
|
|
|
|
else
|
|
|
|
data_to.resize(size * 3); /// Arbitary
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
offsets_to.resize(size);
|
|
|
|
|
2018-11-25 00:08:50 +00:00
|
|
|
WriteBufferFromVector<ColumnString::Chars> write_buffer(data_to);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
FormatImpl<FromDataType>::execute(vec_from[i], write_buffer, &type, time_zone);
|
|
|
|
writeChar(0, write_buffer);
|
|
|
|
offsets_to[i] = write_buffer.count();
|
|
|
|
}
|
|
|
|
|
2018-11-30 15:36:41 +00:00
|
|
|
write_buffer.finish();
|
2017-12-16 04:59:32 +00:00
|
|
|
block.getByPosition(result).column = std::move(col_to);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2017-07-21 06:35:58 +00:00
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2017-04-01 07:20:54 +00:00
|
|
|
+ " of first argument of function " + Name::name,
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
2011-10-16 01:57:10 +00:00
|
|
|
};
|
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
|
|
|
|
/// Generic conversion of any type to String.
|
|
|
|
struct ConvertImplGenericToString
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto & col_with_type_and_name = block.getByPosition(arguments[0]);
|
2017-04-01 07:20:54 +00:00
|
|
|
const IDataType & type = *col_with_type_and_name.type;
|
|
|
|
const IColumn & col_from = *col_with_type_and_name.column;
|
|
|
|
|
|
|
|
size_t size = col_from.size();
|
|
|
|
|
2017-12-14 01:43:19 +00:00
|
|
|
auto col_to = ColumnString::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-25 00:08:50 +00:00
|
|
|
ColumnString::Chars & data_to = col_to->getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-13 14:33:43 +00:00
|
|
|
data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitrary.
|
2017-12-09 10:14:45 +00:00
|
|
|
offsets_to.resize(size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-25 00:08:50 +00:00
|
|
|
WriteBufferFromVector<ColumnString::Chars> write_buffer(data_to);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-08 01:51:55 +00:00
|
|
|
FormatSettings format_settings;
|
2017-12-09 10:14:45 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-12-13 13:41:47 +00:00
|
|
|
type.serializeAsText(col_from, i, write_buffer, format_settings);
|
2017-12-09 10:14:45 +00:00
|
|
|
writeChar(0, write_buffer);
|
|
|
|
offsets_to[i] = write_buffer.count();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-12-09 10:14:45 +00:00
|
|
|
|
2018-11-30 15:36:41 +00:00
|
|
|
write_buffer.finish();
|
2017-12-16 04:59:32 +00:00
|
|
|
block.getByPosition(result).column = std::move(col_to);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-07-30 04:39:51 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2017-01-22 08:33:16 +00:00
|
|
|
/** Conversion of strings to numbers, dates, datetimes: through parsing.
|
2015-07-03 11:55:51 +00:00
|
|
|
*/
|
2018-07-11 21:43:09 +00:00
|
|
|
template <typename DataType>
|
|
|
|
void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
2015-07-01 16:07:29 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
readText(x, rb);
|
2015-07-01 16:07:29 +00:00
|
|
|
}
|
|
|
|
|
2018-07-11 21:43:09 +00:00
|
|
|
template <>
|
|
|
|
inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
2011-10-16 03:05:15 +00:00
|
|
|
{
|
2018-05-25 13:29:15 +00:00
|
|
|
DayNum tmp(0);
|
2017-04-01 07:20:54 +00:00
|
|
|
readDateText(tmp, rb);
|
|
|
|
x = tmp;
|
2011-10-16 03:05:15 +00:00
|
|
|
}
|
|
|
|
|
2018-07-11 21:43:09 +00:00
|
|
|
template <>
|
|
|
|
inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
2011-10-16 03:05:15 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
time_t tmp = 0;
|
|
|
|
readDateTimeText(tmp, rb, *time_zone);
|
|
|
|
x = tmp;
|
2011-10-16 03:05:15 +00:00
|
|
|
}
|
|
|
|
|
2018-07-16 00:28:44 +00:00
|
|
|
template <>
|
|
|
|
inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
2017-07-06 14:42:27 +00:00
|
|
|
{
|
|
|
|
UUID tmp;
|
|
|
|
readText(tmp, rb);
|
|
|
|
x = tmp;
|
|
|
|
}
|
2016-08-02 05:32:09 +00:00
|
|
|
|
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
template <typename DataType>
|
2018-07-11 21:43:09 +00:00
|
|
|
bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
2017-12-25 02:57:54 +00:00
|
|
|
{
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (std::is_integral_v<typename DataType::FieldType>)
|
2017-12-25 02:57:54 +00:00
|
|
|
return tryReadIntText(x, rb);
|
2017-12-25 04:01:46 +00:00
|
|
|
else if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
|
2017-12-25 02:57:54 +00:00
|
|
|
return tryReadFloatText(x, rb);
|
2018-07-11 21:43:09 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <>
|
|
|
|
inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
|
|
|
{
|
|
|
|
DayNum tmp(0);
|
2018-07-14 23:39:00 +00:00
|
|
|
if (!tryReadDateText(tmp, rb))
|
|
|
|
return false;
|
2018-07-11 21:43:09 +00:00
|
|
|
x = tmp;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
template <>
|
|
|
|
inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
|
|
|
{
|
|
|
|
time_t tmp = 0;
|
2018-07-14 23:39:00 +00:00
|
|
|
if (!tryReadDateTimeText(tmp, rb, *time_zone))
|
|
|
|
return false;
|
2018-07-11 21:43:09 +00:00
|
|
|
x = tmp;
|
|
|
|
return true;
|
2017-12-25 02:57:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-08-17 07:40:06 +00:00
|
|
|
/** Throw exception with verbose message when string value is not parsed completely.
|
|
|
|
*/
|
2017-12-02 02:47:12 +00:00
|
|
|
void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result);
|
2016-08-17 07:40:06 +00:00
|
|
|
|
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
enum class ConvertFromStringExceptionMode
|
|
|
|
{
|
|
|
|
Throw, /// Throw exception if value cannot be parsed.
|
|
|
|
Zero, /// Fill with zero or default if value cannot be parsed.
|
|
|
|
Null /// Return ColumnNullable with NULLs when value cannot be parsed.
|
|
|
|
};
|
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
enum class ConvertFromStringParsingMode
|
|
|
|
{
|
|
|
|
Normal,
|
|
|
|
BestEffort /// Only applicable for DateTime. Will use sophisticated method, that is slower.
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename FromDataType, typename ToDataType, typename Name,
|
|
|
|
ConvertFromStringExceptionMode exception_mode, ConvertFromStringParsingMode parsing_mode>
|
2017-12-25 02:57:54 +00:00
|
|
|
struct ConvertThroughParsing
|
2011-10-16 01:57:10 +00:00
|
|
|
{
|
2017-12-25 02:57:54 +00:00
|
|
|
static_assert(std::is_same_v<FromDataType, DataTypeString> || std::is_same_v<FromDataType, DataTypeFixedString>,
|
2017-12-25 04:10:43 +00:00
|
|
|
"ConvertThroughParsing is only applicable for String or FixedString data types");
|
2017-12-25 02:57:54 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using ToFieldType = typename ToDataType::FieldType;
|
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
static bool isAllRead(ReadBuffer & in)
|
|
|
|
{
|
|
|
|
/// In case of FixedString, skip zero bytes at end.
|
|
|
|
if constexpr (std::is_same_v<FromDataType, DataTypeFixedString>)
|
|
|
|
while (!in.eof() && *in.position() == 0)
|
|
|
|
++in.position();
|
|
|
|
|
|
|
|
if (in.eof())
|
|
|
|
return true;
|
|
|
|
|
|
|
|
/// Special case, that allows to parse string with DateTime as Date.
|
2017-12-25 04:01:46 +00:00
|
|
|
if (std::is_same_v<ToDataType, DataTypeDate> && (in.buffer().size()) == strlen("YYYY-MM-DD hh:mm:ss"))
|
2017-12-25 02:57:54 +00:00
|
|
|
return true;
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2018-08-31 08:59:21 +00:00
|
|
|
template <typename Additions = void *>
|
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count,
|
|
|
|
Additions additions [[maybe_unused]] = Additions())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-08-27 16:16:16 +00:00
|
|
|
using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
|
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr;
|
|
|
|
const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// For conversion to DateTime type, second argument with time zone could be specified.
|
2018-02-11 23:57:07 +00:00
|
|
|
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
|
|
|
|
{
|
|
|
|
local_time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
|
|
|
|
|
|
|
|
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
|
|
|
|
utc_time_zone = &DateLUT::instance("UTC");
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
const IColumn * col_from = block.getByPosition(arguments[0]).column.get();
|
|
|
|
const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(col_from);
|
|
|
|
const ColumnFixedString * col_from_fixed_string = checkAndGetColumn<ColumnFixedString>(col_from);
|
|
|
|
|
|
|
|
if (std::is_same_v<FromDataType, DataTypeString> && !col_from_string)
|
|
|
|
throw Exception("Illegal column " + col_from->getName()
|
|
|
|
+ " of first argument of function " + Name::name,
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
|
|
|
if (std::is_same_v<FromDataType, DataTypeFixedString> && !col_from_fixed_string)
|
|
|
|
throw Exception("Illegal column " + col_from->getName()
|
|
|
|
+ " of first argument of function " + Name::name,
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
size_t size = input_rows_count;
|
2018-08-27 16:16:16 +00:00
|
|
|
typename ColVecTo::MutablePtr col_to = nullptr;
|
2017-12-25 02:57:54 +00:00
|
|
|
|
2018-09-10 13:52:18 +00:00
|
|
|
if constexpr (IsDataTypeDecimal<ToDataType>)
|
2018-08-21 18:25:38 +00:00
|
|
|
{
|
2018-08-31 08:59:21 +00:00
|
|
|
UInt32 scale = additions;
|
2018-08-21 18:25:38 +00:00
|
|
|
ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
|
2018-08-31 08:59:21 +00:00
|
|
|
col_to = ColVecTo::create(size, scale);
|
2018-08-21 18:25:38 +00:00
|
|
|
}
|
2018-08-27 16:16:16 +00:00
|
|
|
else
|
|
|
|
col_to = ColVecTo::create(size);
|
|
|
|
|
|
|
|
typename ColVecTo::Container & vec_to = col_to->getData();
|
2017-12-25 02:57:54 +00:00
|
|
|
|
|
|
|
ColumnUInt8::MutablePtr col_null_map_to;
|
2017-12-25 07:18:27 +00:00
|
|
|
ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr;
|
2018-02-11 23:57:07 +00:00
|
|
|
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-25 02:57:54 +00:00
|
|
|
col_null_map_to = ColumnUInt8::create(size);
|
|
|
|
vec_null_map_to = &col_null_map_to->getData();
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-25 00:08:50 +00:00
|
|
|
const ColumnString::Chars * chars = nullptr;
|
2017-12-25 02:57:54 +00:00
|
|
|
const IColumn::Offsets * offsets = nullptr;
|
|
|
|
size_t fixed_string_size = 0;
|
|
|
|
|
|
|
|
if constexpr (std::is_same_v<FromDataType, DataTypeString>)
|
|
|
|
{
|
|
|
|
chars = &col_from_string->getChars();
|
|
|
|
offsets = &col_from_string->getOffsets();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
chars = &col_from_fixed_string->getChars();
|
|
|
|
fixed_string_size = col_from_fixed_string->getN();
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
size_t current_offset = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
size_t next_offset = std::is_same_v<FromDataType, DataTypeString> ? (*offsets)[i] : (current_offset + fixed_string_size);
|
|
|
|
size_t string_size = std::is_same_v<FromDataType, DataTypeString> ? next_offset - current_offset - 1 : fixed_string_size;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
ReadBufferFromMemory read_buffer(&(*chars)[current_offset], string_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-10 13:52:18 +00:00
|
|
|
if constexpr (IsDataTypeDecimal<ToDataType>)
|
2018-08-21 18:25:38 +00:00
|
|
|
{
|
|
|
|
ToDataType::readText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
|
|
|
|
}
|
|
|
|
else if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw)
|
2017-12-25 02:57:54 +00:00
|
|
|
{
|
2018-02-11 23:57:07 +00:00
|
|
|
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
|
|
|
|
{
|
|
|
|
time_t res;
|
|
|
|
parseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
|
|
|
|
vec_to[i] = res;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
if (!isAllRead(read_buffer))
|
2017-12-02 02:47:12 +00:00
|
|
|
throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
|
2017-12-25 02:57:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-02-11 23:57:07 +00:00
|
|
|
bool parsed;
|
|
|
|
|
|
|
|
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
|
|
|
|
{
|
|
|
|
time_t res;
|
|
|
|
parsed = tryParseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
|
|
|
|
vec_to[i] = res;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-07-11 21:43:09 +00:00
|
|
|
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone) && isAllRead(read_buffer);
|
2018-02-11 23:57:07 +00:00
|
|
|
}
|
2017-12-25 05:22:33 +00:00
|
|
|
|
|
|
|
if (!parsed)
|
2017-12-25 02:57:54 +00:00
|
|
|
vec_to[i] = 0;
|
2017-12-25 05:22:33 +00:00
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
|
2017-12-25 05:22:33 +00:00
|
|
|
(*vec_null_map_to)[i] = !parsed;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-12-16 04:59:32 +00:00
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
current_offset = next_offset;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-12-25 02:57:54 +00:00
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
|
2017-12-25 02:57:54 +00:00
|
|
|
block.getByPosition(result).column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2017-12-25 02:57:54 +00:00
|
|
|
block.getByPosition(result).column = std::move(col_to);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-08-02 05:32:09 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2017-12-25 02:57:54 +00:00
|
|
|
template <typename ToDataType, typename Name>
|
|
|
|
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeString>, DataTypeString>, ToDataType, Name>
|
2018-02-11 23:57:07 +00:00
|
|
|
: ConvertThroughParsing<DataTypeString, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal> {};
|
2017-12-25 02:57:54 +00:00
|
|
|
|
|
|
|
template <typename ToDataType, typename Name>
|
|
|
|
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeFixedString>, DataTypeFixedString>, ToDataType, Name>
|
2018-02-11 23:57:07 +00:00
|
|
|
: ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal> {};
|
2011-10-16 01:57:10 +00:00
|
|
|
|
2016-08-02 03:30:34 +00:00
|
|
|
/// Generic conversion of any type from String. Used for complex types: Array and Tuple.
|
|
|
|
struct ConvertImplGenericFromString
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
const IColumn & col_from = *block.getByPosition(arguments[0]).column;
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = col_from.size();
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2017-12-16 04:59:32 +00:00
|
|
|
const IDataType & data_type_to = *block.getByPosition(result).type;
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
if (const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(&col_from))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-16 04:59:32 +00:00
|
|
|
auto res = data_type_to.createColumn();
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2017-12-16 04:59:32 +00:00
|
|
|
IColumn & column_to = *res;
|
2017-04-01 07:20:54 +00:00
|
|
|
column_to.reserve(size);
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2018-11-25 00:08:50 +00:00
|
|
|
const ColumnString::Chars & chars = col_from_string->getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
const IColumn::Offsets & offsets = col_from_string->getOffsets();
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t current_offset = 0;
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2018-06-08 01:51:55 +00:00
|
|
|
FormatSettings format_settings;
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1);
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2018-12-13 13:41:47 +00:00
|
|
|
data_type_to.deserializeAsTextEscaped(column_to, read_buffer, format_settings);
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!read_buffer.eof())
|
2017-12-02 02:47:12 +00:00
|
|
|
throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
|
2016-08-02 03:30:34 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
current_offset = offsets[i];
|
|
|
|
}
|
2017-12-16 04:59:32 +00:00
|
|
|
|
|
|
|
block.getByPosition(result).column = std::move(res);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2017-07-21 06:35:58 +00:00
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2017-04-01 07:20:54 +00:00
|
|
|
+ " of first argument of conversion function from string",
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
2016-08-02 03:30:34 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2017-01-22 08:33:16 +00:00
|
|
|
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
|
2017-10-30 02:18:06 +00:00
|
|
|
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
|
2015-07-03 00:09:57 +00:00
|
|
|
|
2016-07-31 03:53:16 +00:00
|
|
|
template <>
|
2017-01-22 11:45:41 +00:00
|
|
|
struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp>
|
2017-04-01 07:20:54 +00:00
|
|
|
: ConvertImpl<DataTypeString, DataTypeDateTime, NameToUnixTimestamp> {};
|
2015-07-01 17:32:04 +00:00
|
|
|
|
2015-07-03 00:09:57 +00:00
|
|
|
|
2016-07-31 03:53:16 +00:00
|
|
|
/** If types are identical, just take reference to column.
|
2011-10-16 03:05:15 +00:00
|
|
|
*/
|
2017-10-30 02:18:06 +00:00
|
|
|
template <typename T, typename Name>
|
2017-12-25 04:01:46 +00:00
|
|
|
struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
|
2011-10-16 01:57:10 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2011-10-16 01:57:10 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2016-07-31 03:53:16 +00:00
|
|
|
/** Conversion from FixedString to String.
|
|
|
|
* Cutting sequences of zero bytes from end of strings.
|
2014-04-13 13:58:54 +00:00
|
|
|
*/
|
|
|
|
template <typename Name>
|
|
|
|
struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(block.getByPosition(arguments[0]).column.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-14 01:43:19 +00:00
|
|
|
auto col_to = ColumnString::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-25 00:08:50 +00:00
|
|
|
const ColumnFixedString::Chars & data_from = col_from->getChars();
|
|
|
|
ColumnString::Chars & data_to = col_to->getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = col_from->size();
|
|
|
|
size_t n = col_from->getN();
|
2017-10-30 02:18:06 +00:00
|
|
|
data_to.resize(size * (n + 1)); /// + 1 - zero terminator
|
2017-04-01 07:20:54 +00:00
|
|
|
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);
|
2017-12-16 04:59:32 +00:00
|
|
|
block.getByPosition(result).column = std::move(col_to);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2017-07-21 06:35:58 +00:00
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
2017-04-01 07:20:54 +00:00
|
|
|
+ " of first argument of function " + Name::name,
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
2014-04-13 13:58:54 +00:00
|
|
|
};
|
|
|
|
|
2016-07-31 03:53:16 +00:00
|
|
|
|
|
|
|
/// Declared early because used below.
|
2017-10-30 02:18:06 +00:00
|
|
|
struct NameToDate { static constexpr auto name = "toDate"; };
|
2017-11-05 05:32:22 +00:00
|
|
|
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
2017-10-30 02:18:06 +00:00
|
|
|
struct NameToString { static constexpr auto name = "toString"; };
|
2018-08-22 16:16:39 +00:00
|
|
|
struct NameToDecimal32 { static constexpr auto name = "toDecimal32"; };
|
|
|
|
struct NameToDecimal64 { static constexpr auto name = "toDecimal64"; };
|
|
|
|
struct NameToDecimal128 { static constexpr auto name = "toDecimal128"; };
|
2017-10-30 02:18:06 +00:00
|
|
|
|
|
|
|
|
|
|
|
#define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \
|
|
|
|
struct NameToInterval ## INTERVAL_KIND \
|
|
|
|
{ \
|
|
|
|
static constexpr auto name = "toInterval" #INTERVAL_KIND; \
|
2017-10-30 14:00:47 +00:00
|
|
|
static constexpr int kind = DataTypeInterval::INTERVAL_KIND; \
|
2017-10-30 02:18:06 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
DEFINE_NAME_TO_INTERVAL(Second)
|
|
|
|
DEFINE_NAME_TO_INTERVAL(Minute)
|
|
|
|
DEFINE_NAME_TO_INTERVAL(Hour)
|
|
|
|
DEFINE_NAME_TO_INTERVAL(Day)
|
|
|
|
DEFINE_NAME_TO_INTERVAL(Week)
|
|
|
|
DEFINE_NAME_TO_INTERVAL(Month)
|
2018-12-18 13:16:48 +00:00
|
|
|
DEFINE_NAME_TO_INTERVAL(Quarter)
|
2017-10-30 02:18:06 +00:00
|
|
|
DEFINE_NAME_TO_INTERVAL(Year)
|
|
|
|
|
|
|
|
#undef DEFINE_NAME_TO_INTERVAL
|
2011-10-16 01:57:10 +00:00
|
|
|
|
2016-07-30 04:39:51 +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:
|
2017-04-01 07:20:54 +00:00
|
|
|
using Monotonic = MonotonicityImpl;
|
|
|
|
|
|
|
|
static constexpr auto name = Name::name;
|
2018-08-21 18:25:38 +00:00
|
|
|
static constexpr bool to_decimal =
|
2018-08-22 16:16:39 +00:00
|
|
|
std::is_same_v<Name, NameToDecimal32> || std::is_same_v<Name, NameToDecimal64> || std::is_same_v<Name, NameToDecimal128>;
|
2018-08-21 18:25:38 +00:00
|
|
|
|
2017-12-02 02:47:12 +00:00
|
|
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvert>(); }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return name;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool isVariadic() const override { return true; }
|
|
|
|
size_t getNumberOfArguments() const override { return 0; }
|
2017-12-25 04:01:46 +00:00
|
|
|
bool isInjective(const Block &) override { return std::is_same_v<Name, NameToString>; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-08-21 18:25:38 +00:00
|
|
|
if (to_decimal && 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);
|
|
|
|
}
|
|
|
|
else if (arguments.size() != 1 && arguments.size() != 2)
|
2018-02-11 23:57:07 +00:00
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
+ toString(arguments.size()) + ", should be 1 or 2. Second argument (time zone) is optional only make sense for DateTime.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
2017-12-25 04:01:46 +00:00
|
|
|
if constexpr (std::is_same_v<ToDataType, DataTypeInterval>)
|
2017-11-05 05:32:22 +00:00
|
|
|
{
|
2018-02-06 19:34:53 +00:00
|
|
|
return std::make_shared<DataTypeInterval>(DataTypeInterval::Kind(Name::kind));
|
2017-12-02 02:47:12 +00:00
|
|
|
}
|
2018-08-21 18:25:38 +00:00
|
|
|
else if constexpr (to_decimal)
|
|
|
|
{
|
2018-10-25 13:53:25 +00:00
|
|
|
if (!arguments[1].column)
|
|
|
|
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
2018-08-21 18:25:38 +00:00
|
|
|
UInt64 scale = extractToDecimalScale(arguments[1]);
|
|
|
|
|
2018-08-22 16:16:39 +00:00
|
|
|
if constexpr (std::is_same_v<Name, NameToDecimal32>)
|
2018-08-21 18:25:38 +00:00
|
|
|
return createDecimal(9, scale);
|
2018-08-22 16:16:39 +00:00
|
|
|
else if constexpr (std::is_same_v<Name, NameToDecimal64>)
|
2018-08-21 18:25:38 +00:00
|
|
|
return createDecimal(18, scale);
|
2018-11-24 01:48:06 +00:00
|
|
|
else if constexpr (std::is_same_v<Name, NameToDecimal128>)
|
2018-08-21 18:25:38 +00:00
|
|
|
return createDecimal(38, scale);
|
|
|
|
|
|
|
|
throw Exception("Someting wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
2017-12-02 02:47:12 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
/** Optional second argument with time zone is supported:
|
|
|
|
* - for functions toDateTime, toUnixTimestamp, toDate;
|
|
|
|
* - for function toString of DateTime argument.
|
|
|
|
*/
|
2017-11-05 05:32:22 +00:00
|
|
|
|
2017-12-02 02:47:12 +00:00
|
|
|
if (arguments.size() == 2)
|
2017-11-05 05:32:22 +00:00
|
|
|
{
|
2017-12-02 02:47:12 +00:00
|
|
|
if (!checkAndGetDataType<DataTypeString>(arguments[1].type.get()))
|
|
|
|
throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
2018-08-21 18:25:38 +00:00
|
|
|
static constexpr bool to_date_or_time = std::is_same_v<Name, NameToDateTime>
|
2017-12-25 04:01:46 +00:00
|
|
|
|| std::is_same_v<Name, NameToDate>
|
2018-08-21 18:25:38 +00:00
|
|
|
|| std::is_same_v<Name, NameToUnixTimestamp>;
|
|
|
|
|
|
|
|
if (!(to_date_or_time
|
2018-09-07 14:37:26 +00:00
|
|
|
|| (std::is_same_v<Name, NameToString> && WhichDataType(arguments[0].type).isDateTime())))
|
2017-12-02 02:47:12 +00:00
|
|
|
{
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
+ toString(arguments.size()) + ", should be 1.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
}
|
2017-11-05 05:32:22 +00:00
|
|
|
}
|
|
|
|
|
2017-12-25 04:01:46 +00:00
|
|
|
if (std::is_same_v<ToDataType, DataTypeDateTime>)
|
2018-02-02 08:33:36 +00:00
|
|
|
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
|
2017-12-02 02:47:12 +00:00
|
|
|
else
|
2018-02-02 08:33:36 +00:00
|
|
|
return std::make_shared<ToDataType>();
|
2017-12-02 02:47:12 +00:00
|
|
|
}
|
2017-11-05 05:32:22 +00:00
|
|
|
}
|
|
|
|
|
2017-07-23 08:40:43 +00:00
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
2018-08-08 16:05:50 +00:00
|
|
|
bool canBeExecutedOnDefaultArguments() const override { return false; }
|
2017-07-23 08:40:43 +00:00
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
executeInternal(block, arguments, result, input_rows_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
|
|
|
/// More convenient error message.
|
|
|
|
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
|
|
|
{
|
|
|
|
e.addMessage("Cannot parse "
|
|
|
|
+ block.getByPosition(result).type->getName() + " from "
|
|
|
|
+ block.getByPosition(arguments[0]).type->getName()
|
|
|
|
+ ", because value is too short");
|
|
|
|
}
|
|
|
|
else if (e.code() == ErrorCodes::CANNOT_PARSE_NUMBER
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_QUOTED_STRING
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_DATE
|
2017-06-15 09:12:32 +00:00
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_DATETIME
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_UUID)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
e.addMessage("Cannot parse "
|
|
|
|
+ block.getByPosition(result).type->getName() + " from "
|
|
|
|
+ block.getByPosition(arguments[0]).type->getName());
|
|
|
|
}
|
|
|
|
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasInformationAboutMonotonicity() const override
|
|
|
|
{
|
|
|
|
return Monotonic::has();
|
|
|
|
}
|
|
|
|
|
|
|
|
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
|
|
|
{
|
|
|
|
return Monotonic::get(type, left, right);
|
|
|
|
}
|
2016-08-17 06:55:47 +00:00
|
|
|
|
|
|
|
private:
|
2018-04-24 07:16:39 +00:00
|
|
|
void executeInternal(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-13 21:09:31 +00:00
|
|
|
if (!arguments.size())
|
|
|
|
throw Exception{"Function " + getName() + " expects at least 1 arguments",
|
2018-12-07 03:20:27 +00:00
|
|
|
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
|
2017-12-13 21:09:31 +00:00
|
|
|
|
2017-12-18 01:11:48 +00:00
|
|
|
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
2017-07-21 06:35:58 +00:00
|
|
|
|
2018-08-20 15:17:55 +00:00
|
|
|
auto call = [&](const auto & types) -> bool
|
|
|
|
{
|
|
|
|
using Types = std::decay_t<decltype(types)>;
|
|
|
|
using LeftDataType = typename Types::LeftType;
|
|
|
|
using RightDataType = typename Types::RightType;
|
|
|
|
|
2018-09-10 13:52:18 +00:00
|
|
|
if constexpr (IsDataTypeDecimal<RightDataType>)
|
2018-08-31 08:59:21 +00:00
|
|
|
{
|
|
|
|
if (arguments.size() != 2)
|
|
|
|
throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.",
|
2018-12-07 03:20:27 +00:00
|
|
|
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
|
2018-08-31 08:59:21 +00:00
|
|
|
|
|
|
|
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
|
|
|
|
UInt32 scale = extractToDecimalScale(scale_column);
|
|
|
|
|
|
|
|
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, scale);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
|
2018-08-20 15:17:55 +00:00
|
|
|
return true;
|
|
|
|
};
|
|
|
|
|
|
|
|
bool done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call);
|
|
|
|
if (!done)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// Generic conversion of any type to String.
|
2017-12-25 04:01:46 +00:00
|
|
|
if (std::is_same_v<ToDataType, DataTypeString>)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
ConvertImplGenericToString::execute(block, arguments, result);
|
|
|
|
}
|
|
|
|
else
|
2017-07-21 06:35:58 +00:00
|
|
|
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
|
|
|
}
|
2011-10-15 23:40:56 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
/** Function toTOrZero (where T is number of date or datetime type):
|
2016-08-02 05:32:09 +00:00
|
|
|
* try to convert from String to type T through parsing,
|
|
|
|
* if cannot parse, return default value instead of throwing exception.
|
2018-02-11 23:57:07 +00:00
|
|
|
* Function toTOrNull will return Nullable type with NULL when cannot parse.
|
|
|
|
* NOTE Also need to implement tryToUnixTimestamp with timezone.
|
2016-08-02 05:32:09 +00:00
|
|
|
*/
|
2018-02-11 23:57:07 +00:00
|
|
|
template <typename ToDataType, typename Name,
|
|
|
|
ConvertFromStringExceptionMode exception_mode,
|
|
|
|
ConvertFromStringParsingMode parsing_mode = ConvertFromStringParsingMode::Normal>
|
|
|
|
class FunctionConvertFromString : public IFunction
|
2016-08-02 05:32:09 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = Name::name;
|
2018-08-31 08:59:21 +00:00
|
|
|
static constexpr bool to_decimal =
|
|
|
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>;
|
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvertFromString>(); }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return name;
|
|
|
|
}
|
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
bool isVariadic() const override { return true; }
|
|
|
|
size_t getNumberOfArguments() const override { return 0; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-23 08:40:43 +00:00
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
2018-02-11 23:57:07 +00:00
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
2017-07-23 08:40:43 +00:00
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-11 23:57:07 +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. Second argument (time zone) is optional only make sense for DateTime.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
if (!isStringOrFixedString(arguments[0].type))
|
2018-02-11 23:57:07 +00:00
|
|
|
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),
|
2017-12-25 05:22:33 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
/// Optional second argument with time zone is supported.
|
2017-12-25 05:22:33 +00:00
|
|
|
|
2018-02-11 23:57:07 +00:00
|
|
|
if (arguments.size() == 2)
|
|
|
|
{
|
|
|
|
if constexpr (!std::is_same_v<ToDataType, DataTypeDateTime>)
|
|
|
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
|
|
|
+ toString(arguments.size()) + ", should be 1. Second argument makes sense only when converting to DateTime.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2017-12-25 05:22:33 +00:00
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
if (!isString(arguments[1].type))
|
2018-02-11 23:57:07 +00:00
|
|
|
throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
2017-12-25 05:22:33 +00:00
|
|
|
|
2018-02-12 00:55:46 +00:00
|
|
|
DataTypePtr res;
|
|
|
|
|
2018-08-31 08:59:21 +00:00
|
|
|
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
|
2018-02-12 00:55:46 +00:00
|
|
|
res = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
|
2018-08-31 08:59:21 +00:00
|
|
|
else if constexpr (to_decimal)
|
|
|
|
throw Exception(getName() + " is only implemented for types String and Decimal", ErrorCodes::NOT_IMPLEMENTED);
|
2018-02-11 23:57:07 +00:00
|
|
|
else
|
2018-02-12 00:55:46 +00:00
|
|
|
res = std::make_shared<ToDataType>();
|
|
|
|
|
|
|
|
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
|
|
|
|
res = std::make_shared<DataTypeNullable>(res);
|
|
|
|
|
|
|
|
return res;
|
2017-12-25 05:22:33 +00:00
|
|
|
}
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
2017-12-25 05:22:33 +00:00
|
|
|
{
|
|
|
|
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
|
|
|
|
|
|
|
if (checkAndGetDataType<DataTypeString>(from_type))
|
2018-07-11 19:51:18 +00:00
|
|
|
ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(
|
|
|
|
block, arguments, result, input_rows_count);
|
2018-02-11 23:57:07 +00:00
|
|
|
else if (checkAndGetDataType<DataTypeFixedString>(from_type))
|
2018-07-11 19:51:18 +00:00
|
|
|
ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(
|
|
|
|
block, arguments, result, input_rows_count);
|
2017-12-25 05:22:33 +00:00
|
|
|
else
|
|
|
|
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
|
2018-08-31 08:59:21 +00:00
|
|
|
+ ". Only String or FixedString argument is accepted for try-conversion function."
|
|
|
|
+ " For other arguments, use function without 'orZero' or 'orNull'.",
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
2016-08-02 05:32:09 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2016-07-30 04:39:51 +00:00
|
|
|
/** Conversion to fixed string is implemented only for strings.
|
2013-12-03 10:19:57 +00:00
|
|
|
*/
|
|
|
|
class FunctionToFixedString : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "toFixedString";
|
2018-08-26 00:50:48 +00:00
|
|
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToFixedString>(); }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return name;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t getNumberOfArguments() const override { return 2; }
|
|
|
|
bool isInjective(const Block &) override { return true; }
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-09-07 14:37:26 +00:00
|
|
|
if (!isUnsignedInteger(arguments[1].type))
|
2017-12-25 05:22:33 +00:00
|
|
|
throw Exception("Second argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_COLUMN);
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!arguments[1].column)
|
|
|
|
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
2018-09-07 14:37:26 +00:00
|
|
|
if (!isStringOrFixedString(arguments[0].type))
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
|
2017-12-25 05:22:33 +00:00
|
|
|
const size_t n = arguments[1].column->getUInt(0);
|
2018-02-02 08:33:36 +00:00
|
|
|
return std::make_shared<DataTypeFixedString>(n);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-23 08:40:43 +00:00
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-25 05:22:33 +00:00
|
|
|
const auto n = block.getByPosition(arguments[1]).column->getUInt(0);
|
2018-04-24 07:16:39 +00:00
|
|
|
return executeForN(block, arguments, result, n);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
static void executeForN(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto & column = block.getByPosition(arguments[0]).column;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-23 08:40:43 +00:00
|
|
|
if (const auto column_string = checkAndGetColumn<ColumnString>(column.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-16 04:59:32 +00:00
|
|
|
auto column_fixed = ColumnFixedString::create(n);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
auto & out_chars = column_fixed->getChars();
|
|
|
|
const auto & in_chars = column_string->getChars();
|
|
|
|
const auto & in_offsets = column_string->getOffsets();
|
|
|
|
|
|
|
|
out_chars.resize_fill(in_offsets.size() * n);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < in_offsets.size(); ++i)
|
|
|
|
{
|
|
|
|
const size_t off = i ? in_offsets[i - 1] : 0;
|
|
|
|
const size_t len = in_offsets[i] - off - 1;
|
|
|
|
if (len > n)
|
|
|
|
throw Exception("String too long for type FixedString(" + toString(n) + ")",
|
|
|
|
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
|
|
|
memcpy(&out_chars[i * n], &in_chars[off], len);
|
|
|
|
}
|
|
|
|
|
2017-12-16 04:59:32 +00:00
|
|
|
block.getByPosition(result).column = std::move(column_fixed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-07-21 06:35:58 +00:00
|
|
|
else if (const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(column.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
const auto src_n = column_fixed_string->getN();
|
|
|
|
if (src_n > n)
|
2018-05-07 02:01:11 +00:00
|
|
|
throw Exception{"String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-16 04:59:32 +00:00
|
|
|
auto column_fixed = ColumnFixedString::create(n);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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);
|
2017-12-16 04:59:32 +00:00
|
|
|
|
|
|
|
block.getByPosition(result).column = std::move(column_fixed);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
2013-12-03 10:19:57 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2016-07-23 23:02:14 +00:00
|
|
|
/// Monotonicity.
|
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
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static bool has() { return true; }
|
2017-12-02 02:47:12 +00:00
|
|
|
static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return { true };
|
|
|
|
}
|
2015-11-29 08:06:29 +00:00
|
|
|
};
|
|
|
|
|
2018-08-22 17:26:11 +00:00
|
|
|
struct UnknownMonotonicity
|
|
|
|
{
|
|
|
|
static bool has() { return false; }
|
|
|
|
static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &)
|
|
|
|
{
|
|
|
|
return { false };
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2015-11-29 11:58:44 +00:00
|
|
|
template <typename T>
|
|
|
|
struct ToIntMonotonicity
|
2015-11-29 08:06:29 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static bool has() { return true; }
|
|
|
|
|
2017-12-02 02:47:12 +00:00
|
|
|
static UInt64 divideByRangeOfType(UInt64 x)
|
|
|
|
{
|
|
|
|
if constexpr (sizeof(T) < sizeof(UInt64))
|
|
|
|
return x >> (sizeof(T) * 8);
|
|
|
|
else
|
|
|
|
return 0;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
|
|
|
{
|
2018-12-20 11:42:45 +00:00
|
|
|
if (!type.isValueRepresentedByNumber())
|
2018-12-19 06:57:37 +00:00
|
|
|
return {};
|
|
|
|
|
2018-12-24 15:46:55 +00:00
|
|
|
/// If type is same, the conversion is always monotonic.
|
|
|
|
/// (Enum has separate case, because it is different data type)
|
2018-12-21 11:37:33 +00:00
|
|
|
if (checkAndGetDataType<DataTypeNumber<T>>(&type) ||
|
2018-09-07 14:37:26 +00:00
|
|
|
checkAndGetDataType<DataTypeEnum<T>>(&type))
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
return { true, true, true };
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-12-24 15:46:55 +00:00
|
|
|
/// Float cases.
|
2018-12-24 13:25:48 +00:00
|
|
|
|
2018-12-24 15:46:55 +00:00
|
|
|
/// When converting to Float, the conversion is always monotonic.
|
|
|
|
if (std::is_floating_point_v<T>)
|
|
|
|
return {true, true, true};
|
|
|
|
|
|
|
|
/// If converting from Float, for monotonicity, arguments must fit in range of result type.
|
2018-09-07 14:37:26 +00:00
|
|
|
if (WhichDataType(type).isFloat())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-12-24 15:46:55 +00:00
|
|
|
if (left.isNull() || right.isNull())
|
|
|
|
return {};
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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 {};
|
|
|
|
}
|
|
|
|
|
2018-12-24 15:46:55 +00:00
|
|
|
/// Integer cases.
|
|
|
|
|
2018-12-25 18:40:47 +00:00
|
|
|
const bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger();
|
|
|
|
const bool to_is_unsigned = std::is_unsigned_v<T>;
|
2018-12-24 15:46:55 +00:00
|
|
|
|
2018-12-25 18:40:47 +00:00
|
|
|
const size_t size_of_from = type.getSizeOfValueInMemory();
|
|
|
|
const size_t size_of_to = sizeof(T);
|
2018-12-24 15:46:55 +00:00
|
|
|
|
2018-12-25 18:40:47 +00:00
|
|
|
const bool left_in_first_half = left.isNull()
|
2018-12-24 15:46:55 +00:00
|
|
|
? from_is_unsigned
|
2018-12-25 18:40:47 +00:00
|
|
|
: (left.get<Int64>() >= 0);
|
2018-12-24 15:46:55 +00:00
|
|
|
|
2018-12-25 18:40:47 +00:00
|
|
|
const bool right_in_first_half = right.isNull()
|
2018-12-24 15:46:55 +00:00
|
|
|
? !from_is_unsigned
|
2018-12-25 18:40:47 +00:00
|
|
|
: (right.get<Int64>() >= 0);
|
2018-12-25 18:06:38 +00:00
|
|
|
|
2018-12-24 15:46:55 +00:00
|
|
|
/// Size of type is the same.
|
|
|
|
if (size_of_from == size_of_to)
|
|
|
|
{
|
|
|
|
if (from_is_unsigned == to_is_unsigned)
|
|
|
|
return {true, true, true};
|
|
|
|
|
|
|
|
if (left_in_first_half == right_in_first_half)
|
|
|
|
return {true};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return {};
|
2018-12-24 15:46:55 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-12-24 15:46:55 +00:00
|
|
|
/// Size of type is expanded.
|
|
|
|
if (size_of_from < size_of_to)
|
|
|
|
{
|
|
|
|
if (from_is_unsigned == to_is_unsigned)
|
|
|
|
return {true, true, true};
|
|
|
|
|
|
|
|
if (!to_is_unsigned)
|
|
|
|
return {true, true, true};
|
|
|
|
|
|
|
|
/// signed -> unsigned. If arguments from the same half, then function is monotonic.
|
|
|
|
if (left_in_first_half == right_in_first_half)
|
|
|
|
return {true};
|
2018-12-25 18:40:47 +00:00
|
|
|
|
|
|
|
return {};
|
2018-12-24 15:46:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Size of type is shrinked.
|
|
|
|
if (size_of_from > size_of_to)
|
|
|
|
{
|
|
|
|
/// Function cannot be monotonic on unbounded ranges.
|
|
|
|
if (left.isNull() || right.isNull())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
if (from_is_unsigned == to_is_unsigned)
|
|
|
|
{
|
|
|
|
/// all bits other than that fits, must be same.
|
|
|
|
if (divideByRangeOfType(left.get<UInt64>()) == divideByRangeOfType(right.get<UInt64>()))
|
|
|
|
return {true};
|
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// When signedness is changed, it's also required for arguments to be from the same half.
|
2018-12-24 16:13:04 +00:00
|
|
|
/// And they must be in the same half after converting to the result type.
|
2018-12-24 15:46:55 +00:00
|
|
|
if (left_in_first_half == right_in_first_half
|
2018-12-24 16:13:04 +00:00
|
|
|
&& (T(left.get<Int64>()) >= 0) == (T(right.get<Int64>()) >= 0)
|
2018-12-24 15:46:55 +00:00
|
|
|
&& divideByRangeOfType(left.get<UInt64>()) == divideByRangeOfType(right.get<UInt64>()))
|
|
|
|
return {true};
|
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
__builtin_unreachable();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-11-29 08:06:29 +00:00
|
|
|
};
|
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/** The monotonicity for the `toString` function is mainly determined for test purposes.
|
|
|
|
* It is doubtful that anyone is looking to optimize queries with conditions `toString(CounterID) = 34`.
|
2015-11-29 11:58:44 +00:00
|
|
|
*/
|
|
|
|
struct ToStringMonotonicity
|
2015-11-29 08:06:29 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static bool has() { return true; }
|
|
|
|
|
|
|
|
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
|
|
|
{
|
|
|
|
IFunction::Monotonicity positive(true, true);
|
|
|
|
IFunction::Monotonicity not_monotonic;
|
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// `toString` function is monotonous if the argument is Date or DateTime, or non-negative numbers with the same number of symbols.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
if (checkAndGetDataType<DataTypeDate>(&type)
|
2017-04-01 07:20:54 +00:00
|
|
|
|| typeid_cast<const DataTypeDateTime *>(&type))
|
|
|
|
return positive;
|
|
|
|
|
|
|
|
if (left.isNull() || right.isNull())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
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
|
|
|
};
|
|
|
|
|
2017-10-30 02:18:06 +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 NameToUUID { static constexpr auto name = "toUUID"; };
|
|
|
|
|
|
|
|
using FunctionToUInt8 = FunctionConvert<DataTypeUInt8, NameToUInt8, ToIntMonotonicity<UInt8>>;
|
|
|
|
using FunctionToUInt16 = FunctionConvert<DataTypeUInt16, NameToUInt16, ToIntMonotonicity<UInt16>>;
|
|
|
|
using FunctionToUInt32 = FunctionConvert<DataTypeUInt32, NameToUInt32, ToIntMonotonicity<UInt32>>;
|
|
|
|
using FunctionToUInt64 = FunctionConvert<DataTypeUInt64, NameToUInt64, ToIntMonotonicity<UInt64>>;
|
|
|
|
using FunctionToInt8 = FunctionConvert<DataTypeInt8, NameToInt8, ToIntMonotonicity<Int8>>;
|
|
|
|
using FunctionToInt16 = FunctionConvert<DataTypeInt16, NameToInt16, ToIntMonotonicity<Int16>>;
|
|
|
|
using FunctionToInt32 = FunctionConvert<DataTypeInt32, NameToInt32, ToIntMonotonicity<Int32>>;
|
|
|
|
using FunctionToInt64 = FunctionConvert<DataTypeInt64, NameToInt64, ToIntMonotonicity<Int64>>;
|
|
|
|
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, PositiveMonotonicity>;
|
|
|
|
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, PositiveMonotonicity>;
|
|
|
|
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToIntMonotonicity<UInt16>>;
|
|
|
|
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToIntMonotonicity<UInt32>>;
|
|
|
|
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToIntMonotonicity<UInt128>>;
|
|
|
|
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
|
|
|
|
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>>;
|
2018-08-22 17:26:11 +00:00
|
|
|
using FunctionToDecimal32 = FunctionConvert<DataTypeDecimal<Decimal32>, NameToDecimal32, UnknownMonotonicity>;
|
|
|
|
using FunctionToDecimal64 = FunctionConvert<DataTypeDecimal<Decimal64>, NameToDecimal64, UnknownMonotonicity>;
|
|
|
|
using FunctionToDecimal128 = FunctionConvert<DataTypeDecimal<Decimal128>, NameToDecimal128, UnknownMonotonicity>;
|
2017-10-30 02:18:06 +00:00
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
template <typename DataType> struct FunctionTo;
|
2017-10-30 02:18:06 +00:00
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
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; };
|
2017-07-04 10:42:53 +00:00
|
|
|
template <> struct FunctionTo<DataTypeUUID> { using Type = FunctionToUUID; };
|
2015-12-29 13:51:12 +00:00
|
|
|
template <> struct FunctionTo<DataTypeString> { using Type = FunctionToString; };
|
|
|
|
template <> struct FunctionTo<DataTypeFixedString> { using Type = FunctionToFixedString; };
|
2018-08-22 16:16:39 +00:00
|
|
|
template <> struct FunctionTo<DataTypeDecimal<Decimal32>> { using Type = FunctionToDecimal32; };
|
|
|
|
template <> struct FunctionTo<DataTypeDecimal<Decimal64>> { using Type = FunctionToDecimal64; };
|
|
|
|
template <> struct FunctionTo<DataTypeDecimal<Decimal128>> { using Type = FunctionToDecimal128; };
|
2017-10-30 02:18:06 +00:00
|
|
|
|
2015-12-29 13:51:12 +00:00
|
|
|
template <typename FieldType> struct FunctionTo<DataTypeEnum<FieldType>>
|
2017-04-01 07:20:54 +00:00
|
|
|
: FunctionTo<DataTypeNumber<FieldType>>
|
2015-12-29 13:51:12 +00:00
|
|
|
{
|
|
|
|
};
|
2015-12-25 15:43:38 +00:00
|
|
|
|
2017-10-30 02:18:06 +00:00
|
|
|
struct NameToUInt8OrZero { static constexpr auto name = "toUInt8OrZero"; };
|
|
|
|
struct NameToUInt16OrZero { static constexpr auto name = "toUInt16OrZero"; };
|
|
|
|
struct NameToUInt32OrZero { static constexpr auto name = "toUInt32OrZero"; };
|
|
|
|
struct NameToUInt64OrZero { static constexpr auto name = "toUInt64OrZero"; };
|
|
|
|
struct NameToInt8OrZero { static constexpr auto name = "toInt8OrZero"; };
|
|
|
|
struct NameToInt16OrZero { static constexpr auto name = "toInt16OrZero"; };
|
|
|
|
struct NameToInt32OrZero { static constexpr auto name = "toInt32OrZero"; };
|
|
|
|
struct NameToInt64OrZero { static constexpr auto name = "toInt64OrZero"; };
|
|
|
|
struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; };
|
|
|
|
struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; };
|
2018-02-11 23:57:07 +00:00
|
|
|
struct NameToDateOrZero { static constexpr auto name = "toDateOrZero"; };
|
|
|
|
struct NameToDateTimeOrZero { static constexpr auto name = "toDateTimeOrZero"; };
|
|
|
|
|
|
|
|
using FunctionToUInt8OrZero = FunctionConvertFromString<DataTypeUInt8, NameToUInt8OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToUInt16OrZero = FunctionConvertFromString<DataTypeUInt16, NameToUInt16OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToUInt32OrZero = FunctionConvertFromString<DataTypeUInt32, NameToUInt32OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToUInt64OrZero = FunctionConvertFromString<DataTypeUInt64, NameToUInt64OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToInt8OrZero = FunctionConvertFromString<DataTypeInt8, NameToInt8OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToInt16OrZero = FunctionConvertFromString<DataTypeInt16, NameToInt16OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToInt32OrZero = FunctionConvertFromString<DataTypeInt32, NameToInt32OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToInt64OrZero = FunctionConvertFromString<DataTypeInt64, NameToInt64OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToFloat32OrZero = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToFloat64OrZero = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToDateOrZero = FunctionConvertFromString<DataTypeDate, NameToDateOrZero, ConvertFromStringExceptionMode::Zero>;
|
|
|
|
using FunctionToDateTimeOrZero = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrZero, ConvertFromStringExceptionMode::Zero>;
|
2016-08-02 05:32:09 +00:00
|
|
|
|
2017-12-25 05:22:33 +00:00
|
|
|
struct NameToUInt8OrNull { static constexpr auto name = "toUInt8OrNull"; };
|
|
|
|
struct NameToUInt16OrNull { static constexpr auto name = "toUInt16OrNull"; };
|
|
|
|
struct NameToUInt32OrNull { static constexpr auto name = "toUInt32OrNull"; };
|
|
|
|
struct NameToUInt64OrNull { static constexpr auto name = "toUInt64OrNull"; };
|
|
|
|
struct NameToInt8OrNull { static constexpr auto name = "toInt8OrNull"; };
|
|
|
|
struct NameToInt16OrNull { static constexpr auto name = "toInt16OrNull"; };
|
|
|
|
struct NameToInt32OrNull { static constexpr auto name = "toInt32OrNull"; };
|
|
|
|
struct NameToInt64OrNull { static constexpr auto name = "toInt64OrNull"; };
|
|
|
|
struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; };
|
|
|
|
struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; };
|
2018-02-11 23:57:07 +00:00
|
|
|
struct NameToDateOrNull { static constexpr auto name = "toDateOrNull"; };
|
|
|
|
struct NameToDateTimeOrNull { static constexpr auto name = "toDateTimeOrNull"; };
|
|
|
|
|
|
|
|
using FunctionToUInt8OrNull = FunctionConvertFromString<DataTypeUInt8, NameToUInt8OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToUInt16OrNull = FunctionConvertFromString<DataTypeUInt16, NameToUInt16OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToUInt32OrNull = FunctionConvertFromString<DataTypeUInt32, NameToUInt32OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToUInt64OrNull = FunctionConvertFromString<DataTypeUInt64, NameToUInt64OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToInt8OrNull = FunctionConvertFromString<DataTypeInt8, NameToInt8OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToInt16OrNull = FunctionConvertFromString<DataTypeInt16, NameToInt16OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToInt32OrNull = FunctionConvertFromString<DataTypeInt32, NameToInt32OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToInt64OrNull = FunctionConvertFromString<DataTypeInt64, NameToInt64OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToFloat32OrNull = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToFloat64OrNull = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToDateOrNull = FunctionConvertFromString<DataTypeDate, NameToDateOrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
using FunctionToDateTimeOrNull = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrNull, ConvertFromStringExceptionMode::Null>;
|
|
|
|
|
|
|
|
struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTimeBestEffort"; };
|
|
|
|
struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; };
|
|
|
|
struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; };
|
|
|
|
|
|
|
|
using FunctionParseDateTimeBestEffort = FunctionConvertFromString<
|
|
|
|
DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>;
|
|
|
|
using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString<
|
|
|
|
DataTypeDateTime, NameParseDateTimeBestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>;
|
|
|
|
using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString<
|
|
|
|
DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>;
|
2017-12-25 05:22:33 +00:00
|
|
|
|
2015-12-25 15:43:38 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
class PreparedFunctionCast : public PreparedFunctionImpl
|
2015-12-23 08:11:11 +00:00
|
|
|
{
|
2017-12-07 12:09:55 +00:00
|
|
|
public:
|
2018-04-24 07:16:39 +00:00
|
|
|
using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t, size_t)>;
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
explicit PreparedFunctionCast(WrapperType && wrapper_function, const char * name)
|
|
|
|
: wrapper_function(std::move(wrapper_function)), name(name) {}
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
protected:
|
2018-04-24 07:16:39 +00:00
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
2018-02-02 08:33:36 +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));
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
wrapper_function(block, new_arguments, result, input_rows_count);
|
2018-02-02 08:33:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool useDefaultImplementationForNulls() const override { return false; }
|
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
2018-09-27 15:55:22 +00:00
|
|
|
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
2018-02-02 08:33:36 +00:00
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
2017-12-07 12:09:55 +00:00
|
|
|
|
|
|
|
private:
|
2018-02-02 08:33:36 +00:00
|
|
|
WrapperType wrapper_function;
|
|
|
|
const char * name;
|
|
|
|
};
|
|
|
|
|
2018-07-11 19:51:18 +00:00
|
|
|
|
|
|
|
struct NameCast { static constexpr auto name = "CAST"; };
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
class FunctionCast final : public IFunctionBase
|
|
|
|
{
|
|
|
|
public:
|
2018-04-24 07:16:39 +00:00
|
|
|
using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t, size_t)>;
|
2018-02-06 19:34:53 +00:00
|
|
|
using MonotonicityForRange = std::function<Monotonicity(const IDataType &, const Field &, const Field &)>;
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
FunctionCast(const Context & context, const char * name, MonotonicityForRange && monotonicity_for_range
|
|
|
|
, const DataTypes & argument_types, const DataTypePtr & return_type)
|
|
|
|
: context(context), name(name), monotonicity_for_range(monotonicity_for_range)
|
|
|
|
, argument_types(argument_types), return_type(return_type)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
|
|
|
const DataTypePtr & getReturnType() const override { return return_type; }
|
|
|
|
|
2018-09-21 14:15:21 +00:00
|
|
|
PreparedFunctionPtr prepare(const Block & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
2018-02-02 08:33:36 +00:00
|
|
|
{
|
2018-06-07 10:37:06 +00:00
|
|
|
return std::make_shared<PreparedFunctionCast>(
|
|
|
|
prepareUnpackDictionaries(getArgumentTypes()[0], getReturnType()), name);
|
2018-02-02 08:33:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context;
|
2018-02-02 08:33:36 +00:00
|
|
|
const char * name;
|
|
|
|
MonotonicityForRange monotonicity_for_range;
|
|
|
|
|
|
|
|
DataTypes argument_types;
|
|
|
|
DataTypePtr return_type;
|
2015-12-23 08:11:11 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
template <typename DataType>
|
2018-07-11 19:51:18 +00:00
|
|
|
WrapperType createWrapper(const DataTypePtr & from_type, const DataType * const, bool requested_result_is_nullable) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-07-11 19:51:18 +00:00
|
|
|
FunctionPtr function;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-07-11 19:51:18 +00:00
|
|
|
if (requested_result_is_nullable && checkAndGetDataType<DataTypeString>(from_type.get()))
|
|
|
|
{
|
|
|
|
/// In case when converting to Nullable type, we apply different parsing rule,
|
|
|
|
/// that will not throw an exception but return NULL in case of malformed input.
|
|
|
|
function = FunctionConvertFromString<DataType, NameCast, ConvertFromStringExceptionMode::Null>::create(context);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
function = FunctionTo<DataType>::Type::create(context);
|
|
|
|
|
|
|
|
/// Check conversion using underlying function
|
|
|
|
{
|
|
|
|
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-07-11 19:51:18 +00:00
|
|
|
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
function->execute(block, arguments, result, input_rows_count);
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
WrapperType createStringWrapper(const DataTypePtr & from_type) const
|
|
|
|
{
|
|
|
|
FunctionPtr function = FunctionToString::create(context);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Check conversion using underlying function
|
2017-11-05 05:32:22 +00:00
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
|
2017-11-05 05:32:22 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
2017-11-05 05:32:22 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
function->execute(block, arguments, result, input_rows_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
static WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-09-07 14:37:26 +00:00
|
|
|
if (!isStringOrFixedString(from_type))
|
2018-02-18 02:22:32 +00:00
|
|
|
throw Exception{"CAST AS FixedString is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
FunctionToFixedString::executeForN(block, arguments, result, N);
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-09-07 16:38:40 +00:00
|
|
|
WrapperType createUUIDWrapper(const DataTypePtr & from_type, const DataTypeUUID * const, bool requested_result_is_nullable) const
|
|
|
|
{
|
|
|
|
if (requested_result_is_nullable)
|
|
|
|
throw Exception{"CAST AS Nullable(UUID) is not implemented", ErrorCodes::NOT_IMPLEMENTED};
|
|
|
|
|
|
|
|
FunctionPtr function = FunctionTo<DataTypeUUID>::Type::create(context);
|
|
|
|
|
|
|
|
/// Check conversion using underlying function
|
|
|
|
{
|
|
|
|
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
|
|
|
|
}
|
|
|
|
|
|
|
|
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
function->execute(block, arguments, result, input_rows_count);
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-08-31 08:59:21 +00:00
|
|
|
template <typename FieldType>
|
|
|
|
WrapperType createDecimalWrapper(const DataTypePtr & from_type, const DataTypeDecimal<FieldType> * to_type) const
|
|
|
|
{
|
|
|
|
using ToDataType = DataTypeDecimal<FieldType>;
|
|
|
|
|
|
|
|
TypeIndex type_index = from_type->getTypeId();
|
|
|
|
UInt32 scale = to_type->getScale();
|
|
|
|
|
|
|
|
return [type_index, scale] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
callOnIndexAndDataType<ToDataType>(type_index, [&](const auto & types) -> bool
|
|
|
|
{
|
|
|
|
using Types = std::decay_t<decltype(types)>;
|
|
|
|
using LeftDataType = typename Types::LeftType;
|
|
|
|
using RightDataType = typename Types::RightType;
|
|
|
|
|
|
|
|
ConvertImpl<LeftDataType, RightDataType, NameCast>::execute(block, arguments, result, input_rows_count, scale);
|
|
|
|
return true;
|
|
|
|
});
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray * to_type) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// Conversion from String through parsing.
|
2017-07-21 06:35:58 +00:00
|
|
|
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
ConvertImplGenericFromString::execute(block, arguments, result);
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2017-12-16 04:59:32 +00:00
|
|
|
DataTypePtr from_nested_type;
|
|
|
|
DataTypePtr to_nested_type;
|
2017-07-21 06:35:58 +00:00
|
|
|
auto from_type = checkAndGetDataType<DataTypeArray>(from_type_untyped.get());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// get the most nested type
|
2017-12-16 04:59:32 +00:00
|
|
|
if (from_type && to_type)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
from_nested_type = from_type->getNestedType();
|
|
|
|
to_nested_type = to_type->getNestedType();
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
from_type = checkAndGetDataType<DataTypeArray>(from_nested_type.get());
|
|
|
|
to_type = checkAndGetDataType<DataTypeArray>(to_nested_type.get());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// both from_type and to_type should be nullptr now is array types had same dimensions
|
2018-01-10 19:52:39 +00:00
|
|
|
if ((from_type == nullptr) != (to_type == nullptr))
|
2018-08-31 08:59:21 +00:00
|
|
|
throw Exception{"CAST AS Array can only be performed between same-dimensional array types or from String",
|
|
|
|
ErrorCodes::TYPE_MISMATCH};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Prepare nested type conversion
|
2018-06-07 10:37:06 +00:00
|
|
|
const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-08 03:52:33 +00:00
|
|
|
return [nested_function, from_nested_type, to_nested_type](
|
2018-04-24 07:16:39 +00:00
|
|
|
Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-16 04:59:32 +00:00
|
|
|
const auto & array_arg = block.getByPosition(arguments.front());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-16 04:59:32 +00:00
|
|
|
if (const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(array_arg.column.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// create block for converting nested column containing original and result columns
|
2017-12-08 03:52:33 +00:00
|
|
|
Block nested_block
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
{ col_array->getDataPtr(), from_nested_type, "" },
|
|
|
|
{ nullptr, to_nested_type, "" }
|
|
|
|
};
|
|
|
|
|
|
|
|
/// convert nested column
|
2018-04-24 07:16:39 +00:00
|
|
|
nested_function(nested_block, {0}, 1, nested_block.rows());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// set converted nested column to result
|
2017-12-16 04:59:32 +00:00
|
|
|
block.getByPosition(result).column = ColumnArray::create(nested_block.getByPosition(1).column, col_array->getOffsetsPtr());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2018-02-18 02:22:32 +00:00
|
|
|
throw Exception{"Illegal column " + array_arg.column->getName() + " for function CAST AS Array", ErrorCodes::LOGICAL_ERROR};
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
WrapperType createTupleWrapper(const DataTypePtr & from_type_untyped, const DataTypeTuple * to_type) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// Conversion from String through parsing.
|
2017-07-21 06:35:58 +00:00
|
|
|
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
ConvertImplGenericFromString::execute(block, arguments, result);
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto from_type = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get());
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!from_type)
|
2018-08-31 08:59:21 +00:00
|
|
|
throw Exception{"CAST AS Tuple can only be performed between tuple types or from String.\nLeft type: "
|
|
|
|
+ from_type_untyped->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (from_type->getElements().size() != to_type->getElements().size())
|
2018-02-18 02:22:32 +00:00
|
|
|
throw Exception{"CAST AS Tuple can only be performed between tuple types with the same number of elements or from String.\n"
|
|
|
|
"Left type: " + from_type->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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
|
2019-01-04 12:10:00 +00:00
|
|
|
for (const auto idx_type : ext::enumerate(from_type->getElements()))
|
2018-06-07 10:37:06 +00:00
|
|
|
element_wrappers.push_back(prepareUnpackDictionaries(idx_type.second, to_element_types[idx_type.first]));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-08 00:50:25 +00:00
|
|
|
return [element_wrappers, from_element_types, to_element_types]
|
2018-04-24 07:16:39 +00:00
|
|
|
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto col = block.getByPosition(arguments.front()).column.get();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// copy tuple elements to a separate block
|
|
|
|
Block element_block;
|
|
|
|
|
2017-12-08 00:50:25 +00:00
|
|
|
size_t tuple_size = from_element_types.size();
|
|
|
|
const ColumnTuple & column_tuple = typeid_cast<const ColumnTuple &>(*col);
|
|
|
|
|
|
|
|
/// create columns for source elements
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
element_block.insert({ column_tuple.getColumns()[i], from_element_types[i], "" });
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// create columns for converted elements
|
|
|
|
for (const auto & to_element_type : to_element_types)
|
|
|
|
element_block.insert({ nullptr, to_element_type, "" });
|
|
|
|
|
|
|
|
/// insert column for converted tuple
|
|
|
|
element_block.insert({ nullptr, std::make_shared<DataTypeTuple>(to_element_types), "" });
|
|
|
|
|
|
|
|
/// invoke conversion for each element
|
2019-01-04 12:10:00 +00:00
|
|
|
for (const auto idx_element_wrapper : ext::enumerate(element_wrappers))
|
2017-04-01 07:20:54 +00:00
|
|
|
idx_element_wrapper.second(element_block, { idx_element_wrapper.first },
|
2018-04-24 07:16:39 +00:00
|
|
|
tuple_size + idx_element_wrapper.first, input_rows_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-08 00:50:25 +00:00
|
|
|
Columns converted_columns(tuple_size);
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
converted_columns[i] = element_block.getByPosition(tuple_size + i).column;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-14 01:43:19 +00:00
|
|
|
block.getByPosition(result).column = ColumnTuple::create(converted_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename FieldType>
|
2018-02-02 08:33:36 +00:00
|
|
|
WrapperType createEnumWrapper(const DataTypePtr & from_type, const DataTypeEnum<FieldType> * to_type) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
using EnumType = DataTypeEnum<FieldType>;
|
|
|
|
using Function = typename FunctionTo<EnumType>::Type;
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
if (const auto from_enum8 = checkAndGetDataType<DataTypeEnum8>(from_type.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
checkEnumToEnumConversion(from_enum8, to_type);
|
2017-07-21 06:35:58 +00:00
|
|
|
else if (const auto from_enum16 = checkAndGetDataType<DataTypeEnum16>(from_type.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
checkEnumToEnumConversion(from_enum16, to_type);
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
if (checkAndGetDataType<DataTypeString>(from_type.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
return createStringToEnumWrapper<ColumnString, EnumType>();
|
2017-07-21 06:35:58 +00:00
|
|
|
else if (checkAndGetDataType<DataTypeFixedString>(from_type.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
return createStringToEnumWrapper<ColumnFixedString, EnumType>();
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (isNumber(from_type) || isEnum(from_type))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
auto function = Function::create(context);
|
|
|
|
|
|
|
|
/// Check conversion using underlying function
|
2017-11-05 05:32:22 +00:00
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
|
2017-11-05 05:32:22 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
2017-11-05 05:32:22 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
function->execute(block, arguments, result, input_rows_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
else
|
2018-02-18 02:22:32 +00:00
|
|
|
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() +
|
|
|
|
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename EnumTypeFrom, typename EnumTypeTo>
|
2018-02-02 08:33:36 +00:00
|
|
|
void checkEnumToEnumConversion(const EnumTypeFrom * from_type, const EnumTypeTo * to_type) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
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>;
|
|
|
|
using EnumValues = std::vector<NameValuePair>;
|
|
|
|
|
|
|
|
EnumValues name_intersection;
|
|
|
|
std::set_intersection(std::begin(from_values), std::end(from_values),
|
|
|
|
std::begin(to_values), std::end(to_values), std::back_inserter(name_intersection),
|
|
|
|
[] (auto && from, auto && to) { return from.first < to.first; });
|
|
|
|
|
|
|
|
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)
|
2018-02-18 02:22:32 +00:00
|
|
|
throw Exception{"Enum conversion changes value for element '" + name_value.first +
|
|
|
|
"' from " + toString(old_value) + " to " + toString(new_value), ErrorCodes::CANNOT_CONVERT_TYPE};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-08-26 00:50:48 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
template <typename ColumnStringType, typename EnumType>
|
2018-02-02 08:33:36 +00:00
|
|
|
WrapperType createStringToEnumWrapper() const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-06 19:34:53 +00:00
|
|
|
const char * function_name = name;
|
2018-04-24 07:16:39 +00:00
|
|
|
return [function_name] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto first_col = block.getByPosition(arguments.front()).column.get();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
auto & col_with_type_and_name = block.getByPosition(result);
|
2017-12-18 01:11:48 +00:00
|
|
|
const auto & result_type = typeid_cast<const EnumType &>(*col_with_type_and_name.type);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (const auto col = typeid_cast<const ColumnStringType *>(first_col))
|
|
|
|
{
|
|
|
|
const auto size = col->size();
|
|
|
|
|
|
|
|
auto res = result_type.createColumn();
|
|
|
|
auto & out_data = static_cast<typename EnumType::ColumnType &>(*res).getData();
|
|
|
|
out_data.resize(size);
|
|
|
|
|
|
|
|
for (const auto i : ext::range(0, size))
|
|
|
|
out_data[i] = result_type.getValue(col->getDataAt(i));
|
|
|
|
|
2017-12-16 04:59:32 +00:00
|
|
|
col_with_type_and_name.column = std::move(res);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2018-02-18 02:22:32 +00:00
|
|
|
throw Exception{"Unexpected column " + first_col->getName() + " as first argument of function " + function_name,
|
2017-07-23 08:40:43 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR};
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
WrapperType createIdentityWrapper(const DataTypePtr &) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
WrapperType createNothingWrapper(const IDataType * to_type) const
|
2017-12-09 20:56:53 +00:00
|
|
|
{
|
2017-12-18 04:07:26 +00:00
|
|
|
ColumnPtr res = to_type->createColumnConstWithDefaultValue(1);
|
2018-04-24 07:16:39 +00:00
|
|
|
return [res] (Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
2017-12-09 20:56:53 +00:00
|
|
|
{
|
|
|
|
/// Column of Nothing type is trivially convertible to any other column
|
2018-04-24 07:16:39 +00:00
|
|
|
block.getByPosition(result).column = res->cloneResized(input_rows_count)->convertToFullColumnIfConst();
|
2017-12-09 20:56:53 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-06-07 10:37:06 +00:00
|
|
|
WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-09-27 15:55:22 +00:00
|
|
|
const auto * from_low_cardinality = typeid_cast<const DataTypeLowCardinality *>(from_type.get());
|
|
|
|
const auto * to_low_cardinality = typeid_cast<const DataTypeLowCardinality *>(to_type.get());
|
|
|
|
const auto & from_nested = from_low_cardinality ? from_low_cardinality->getDictionaryType() : from_type;
|
|
|
|
const auto & to_nested = to_low_cardinality ? to_low_cardinality->getDictionaryType() : to_type;
|
2018-09-19 11:11:52 +00:00
|
|
|
|
2017-12-09 06:32:22 +00:00
|
|
|
if (from_type->onlyNull())
|
2017-12-08 03:52:33 +00:00
|
|
|
{
|
2018-09-19 11:11:52 +00:00
|
|
|
if (!to_nested->isNullable())
|
2018-02-22 21:20:02 +00:00
|
|
|
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
2017-12-08 03:52:33 +00:00
|
|
|
{
|
|
|
|
auto & res = block.getByPosition(result);
|
2018-04-24 07:16:39 +00:00
|
|
|
res.column = res.type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst();
|
2017-12-08 03:52:33 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2018-06-07 10:37:06 +00:00
|
|
|
auto wrapper = prepareRemoveNullable(from_nested, to_nested);
|
2018-09-27 15:55:22 +00:00
|
|
|
if (!from_low_cardinality && !to_low_cardinality)
|
2018-06-07 10:37:06 +00:00
|
|
|
return wrapper;
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
return [wrapper, from_low_cardinality, to_low_cardinality]
|
2018-06-07 10:37:06 +00:00
|
|
|
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
auto & arg = block.getByPosition(arguments[0]);
|
|
|
|
auto & res = block.getByPosition(result);
|
|
|
|
|
|
|
|
ColumnPtr res_indexes;
|
2018-08-08 16:05:50 +00:00
|
|
|
/// For some types default can't be casted (for example, String to Int). In that case convert column to full.
|
|
|
|
bool src_converted_to_full_column = false;
|
2018-06-07 10:37:06 +00:00
|
|
|
|
|
|
|
{
|
|
|
|
/// Replace argument and result columns (and types) to dictionary key columns (and types).
|
|
|
|
/// Call nested wrapper in order to cast dictionary keys. Then restore block.
|
|
|
|
auto prev_arg_col = arg.column;
|
|
|
|
auto prev_arg_type = arg.type;
|
|
|
|
auto prev_res_type = res.type;
|
|
|
|
|
|
|
|
auto tmp_rows_count = input_rows_count;
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (to_low_cardinality)
|
|
|
|
res.type = to_low_cardinality->getDictionaryType();
|
2018-08-08 16:05:50 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (from_low_cardinality)
|
2018-06-07 10:37:06 +00:00
|
|
|
{
|
2018-09-27 15:55:22 +00:00
|
|
|
auto * col_low_cardinality = typeid_cast<const ColumnLowCardinality *>(prev_arg_col.get());
|
|
|
|
arg.column = col_low_cardinality->getDictionary().getNestedColumn();
|
|
|
|
arg.type = from_low_cardinality->getDictionaryType();
|
2018-06-07 10:37:06 +00:00
|
|
|
|
2018-08-08 16:05:50 +00:00
|
|
|
/// TODO: Make map with defaults conversion.
|
|
|
|
src_converted_to_full_column = !removeNullable(arg.type)->equals(*removeNullable(res.type));
|
|
|
|
if (src_converted_to_full_column)
|
2018-09-27 15:55:22 +00:00
|
|
|
arg.column = arg.column->index(col_low_cardinality->getIndexes(), 0);
|
2018-08-08 16:05:50 +00:00
|
|
|
else
|
2018-09-27 15:55:22 +00:00
|
|
|
res_indexes = col_low_cardinality->getIndexesPtr();
|
2018-08-08 16:05:50 +00:00
|
|
|
|
2018-06-07 10:37:06 +00:00
|
|
|
tmp_rows_count = arg.column->size();
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Perform the requested conversion.
|
|
|
|
wrapper(block, arguments, result, tmp_rows_count);
|
|
|
|
|
|
|
|
arg.column = prev_arg_col;
|
|
|
|
arg.type = prev_arg_type;
|
|
|
|
res.type = prev_res_type;
|
|
|
|
}
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (to_low_cardinality)
|
2018-06-07 10:37:06 +00:00
|
|
|
{
|
2018-09-27 15:55:22 +00:00
|
|
|
auto res_column = to_low_cardinality->createColumn();
|
|
|
|
auto * col_low_cardinality = typeid_cast<ColumnLowCardinality *>(res_column.get());
|
2018-06-07 10:37:06 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
if (from_low_cardinality && !src_converted_to_full_column)
|
2018-06-07 10:37:06 +00:00
|
|
|
{
|
|
|
|
auto res_keys = std::move(res.column);
|
2018-09-27 15:55:22 +00:00
|
|
|
col_low_cardinality->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes);
|
2018-06-07 10:37:06 +00:00
|
|
|
}
|
|
|
|
else
|
2018-09-27 15:55:22 +00:00
|
|
|
col_low_cardinality->insertRangeFromFullColumn(*res.column, 0, res.column->size());
|
2018-06-07 10:37:06 +00:00
|
|
|
|
|
|
|
res.column = std::move(res_column);
|
|
|
|
}
|
2018-08-08 16:05:50 +00:00
|
|
|
else if (!src_converted_to_full_column)
|
2018-06-07 18:14:37 +00:00
|
|
|
res.column = res.column->index(*res_indexes, 0);
|
2018-06-07 10:37:06 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
WrapperType prepareRemoveNullable(const DataTypePtr & from_type, const DataTypePtr & to_type) const
|
|
|
|
{
|
|
|
|
/// Determine whether pre-processing and/or post-processing must take place during conversion.
|
|
|
|
|
2018-07-20 11:08:54 +00:00
|
|
|
bool source_is_nullable = from_type->isNullable();
|
|
|
|
bool result_is_nullable = to_type->isNullable();
|
2017-12-08 03:52:33 +00:00
|
|
|
|
2018-07-11 19:51:18 +00:00
|
|
|
auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-07-11 19:51:18 +00:00
|
|
|
if (result_is_nullable)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-07-11 19:51:18 +00:00
|
|
|
return [wrapper, source_is_nullable]
|
|
|
|
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// Create a temporary block on which to perform the operation.
|
2017-07-21 06:35:58 +00:00
|
|
|
auto & res = block.getByPosition(result);
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto & ret_type = res.type;
|
|
|
|
const auto & nullable_type = static_cast<const DataTypeNullable &>(*ret_type);
|
|
|
|
const auto & nested_type = nullable_type.getNestedType();
|
|
|
|
|
|
|
|
Block tmp_block;
|
2018-07-11 19:51:18 +00:00
|
|
|
if (source_is_nullable)
|
2017-04-01 07:20:54 +00:00
|
|
|
tmp_block = createBlockWithNestedColumns(block, arguments);
|
|
|
|
else
|
|
|
|
tmp_block = block;
|
|
|
|
|
|
|
|
size_t tmp_res_index = block.columns();
|
|
|
|
tmp_block.insert({nullptr, nested_type, ""});
|
|
|
|
|
|
|
|
/// Perform the requested conversion.
|
2018-04-24 07:16:39 +00:00
|
|
|
wrapper(tmp_block, arguments, tmp_res_index, input_rows_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto & tmp_res = tmp_block.getByPosition(tmp_res_index);
|
2018-07-11 21:43:09 +00:00
|
|
|
|
2018-07-11 19:51:18 +00:00
|
|
|
res.column = wrapInNullable(tmp_res.column, Block({block.getByPosition(arguments[0]), tmp_res}), {0}, 1, input_rows_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
}
|
2018-07-11 19:51:18 +00:00
|
|
|
else if (source_is_nullable)
|
2018-02-22 21:20:02 +00:00
|
|
|
{
|
|
|
|
/// Conversion from Nullable to non-Nullable.
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
return [wrapper] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
2018-02-22 21:20:02 +00:00
|
|
|
{
|
|
|
|
Block tmp_block = createBlockWithNestedColumns(block, arguments, result);
|
|
|
|
|
|
|
|
/// Check that all values are not-NULL.
|
|
|
|
|
|
|
|
const auto & col = block.getByPosition(arguments[0]).column;
|
|
|
|
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
|
|
|
|
const auto & null_map = nullable_col.getNullMapData();
|
|
|
|
|
|
|
|
if (!memoryIsZero(null_map.data(), null_map.size()))
|
|
|
|
throw Exception{"Cannot convert NULL value to non-Nullable type",
|
|
|
|
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
wrapper(tmp_block, arguments, result, input_rows_count);
|
2018-02-22 21:20:02 +00:00
|
|
|
block.getByPosition(result).column = tmp_block.getByPosition(result).column;
|
|
|
|
};
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
|
|
|
return wrapper;
|
|
|
|
}
|
|
|
|
|
2018-08-31 08:59:21 +00:00
|
|
|
/// 'from_type' and 'to_type' are nested types in case of Nullable.
|
|
|
|
/// 'requested_result_is_nullable' is true if CAST to Nullable type is requested.
|
2018-07-11 19:51:18 +00:00
|
|
|
WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (from_type->equals(*to_type))
|
|
|
|
return createIdentityWrapper(from_type);
|
2018-09-07 14:37:26 +00:00
|
|
|
else if (WhichDataType(from_type).isNothing())
|
2018-02-22 21:20:02 +00:00
|
|
|
return createNothingWrapper(to_type.get());
|
2018-08-31 08:59:21 +00:00
|
|
|
|
|
|
|
WrapperType ret;
|
|
|
|
|
|
|
|
auto make_default_wrapper = [&](const auto & types) -> bool
|
|
|
|
{
|
|
|
|
using Types = std::decay_t<decltype(types)>;
|
|
|
|
using ToDataType = typename Types::LeftType;
|
|
|
|
|
|
|
|
if constexpr (
|
|
|
|
std::is_same_v<ToDataType, DataTypeUInt8> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeUInt16> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeUInt32> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeUInt64> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeInt8> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeInt16> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeInt32> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeInt64> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeFloat32> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeFloat64> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeDate> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeDateTime>)
|
|
|
|
{
|
|
|
|
ret = createWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
if constexpr (
|
|
|
|
std::is_same_v<ToDataType, DataTypeEnum8> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeEnum16>)
|
|
|
|
{
|
|
|
|
ret = createEnumWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
if constexpr (
|
|
|
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
|
|
|
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
|
|
|
|
{
|
|
|
|
ret = createDecimalWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
|
|
|
|
return true;
|
|
|
|
}
|
2018-09-07 16:38:40 +00:00
|
|
|
if constexpr (std::is_same_v<ToDataType, DataTypeUUID>)
|
|
|
|
{
|
|
|
|
if (isStringOrFixedString(from_type))
|
|
|
|
{
|
|
|
|
ret = createUUIDWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
2018-08-31 08:59:21 +00:00
|
|
|
|
|
|
|
return false;
|
|
|
|
};
|
|
|
|
|
|
|
|
if (callOnIndexAndDataType<void>(to_type->getTypeId(), make_default_wrapper))
|
|
|
|
return ret;
|
|
|
|
|
|
|
|
switch (to_type->getTypeId())
|
|
|
|
{
|
|
|
|
case TypeIndex::String:
|
|
|
|
return createStringWrapper(from_type);
|
|
|
|
case TypeIndex::FixedString:
|
|
|
|
return createFixedStringWrapper(from_type, checkAndGetDataType<DataTypeFixedString>(to_type.get())->getN());
|
|
|
|
|
|
|
|
case TypeIndex::Array:
|
|
|
|
return createArrayWrapper(from_type, checkAndGetDataType<DataTypeArray>(to_type.get()));
|
|
|
|
case TypeIndex::Tuple:
|
|
|
|
return createTupleWrapper(from_type, checkAndGetDataType<DataTypeTuple>(to_type.get()));
|
|
|
|
|
|
|
|
default:
|
|
|
|
break;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// It's possible to use ConvertImplGenericFromString to convert from String to AggregateFunction,
|
|
|
|
/// but it is disabled because deserializing aggregate functions state might be unsafe.
|
|
|
|
|
2018-08-31 08:59:21 +00:00
|
|
|
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
|
|
|
|
ErrorCodes::CANNOT_CONVERT_TYPE};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-02-02 08:33:36 +00:00
|
|
|
};
|
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
class FunctionBuilderCast : public FunctionBuilderImpl
|
2018-02-02 08:33:36 +00:00
|
|
|
{
|
|
|
|
public:
|
|
|
|
using MonotonicityForRange = FunctionCast::MonotonicityForRange;
|
|
|
|
|
|
|
|
static constexpr auto name = "CAST";
|
2018-02-06 19:34:53 +00:00
|
|
|
static FunctionBuilderPtr create(const Context & context) { return std::make_shared<FunctionBuilderCast>(context); }
|
2018-02-02 08:33:36 +00:00
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
FunctionBuilderCast(const Context & context) : context(context) {}
|
2018-02-02 08:33:36 +00:00
|
|
|
|
2018-02-09 19:32:12 +00:00
|
|
|
String getName() const override { return name; }
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
size_t getNumberOfArguments() const override { return 2; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
protected:
|
|
|
|
|
|
|
|
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
|
|
|
{
|
|
|
|
DataTypes data_types(arguments.size());
|
|
|
|
|
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
|
|
|
data_types[i] = arguments[i].type;
|
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
auto monotonicity = getMonotonicityInformation(arguments.front().type, return_type.get());
|
|
|
|
return std::make_shared<FunctionCast>(context, name, std::move(monotonicity), data_types, return_type);
|
2018-02-02 08:33:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
|
|
{
|
|
|
|
const auto type_col = checkAndGetColumnConst<ColumnString>(arguments.back().column.get());
|
|
|
|
if (!type_col)
|
2018-08-31 08:59:21 +00:00
|
|
|
throw Exception("Second argument to " + getName() + " must be a constant string describing type",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
return DataTypeFactory::instance().get(type_col->getValue<String>());
|
|
|
|
}
|
|
|
|
|
|
|
|
bool useDefaultImplementationForNulls() const override { return false; }
|
2018-09-27 15:55:22 +00:00
|
|
|
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
private:
|
2017-07-24 01:00:31 +00:00
|
|
|
template <typename DataType>
|
|
|
|
static auto monotonicityForType(const DataType * const)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return FunctionTo<DataType>::Type::Monotonic::get;
|
|
|
|
}
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
MonotonicityForRange getMonotonicityInformation(const DataTypePtr & from_type, const IDataType * to_type) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeUInt8>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeUInt16>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeUInt32>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeUInt64>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeInt8>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeInt16>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeInt32>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeInt64>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeFloat32>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeFloat64>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeDate>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeDateTime>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeString>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (isEnum(from_type))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeEnum8>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2019-01-04 12:10:00 +00:00
|
|
|
if (const auto type = checkAndGetDataType<DataTypeEnum16>(to_type))
|
2018-02-02 08:33:36 +00:00
|
|
|
return monotonicityForType(type);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
/// other types like Null, FixedString, Array and Tuple have no monotonicity defined
|
2018-02-02 08:33:36 +00:00
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-12-29 13:51:12 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
const Context & context;
|
2015-12-23 08:11:11 +00:00
|
|
|
};
|
|
|
|
|
2011-10-15 23:40:56 +00:00
|
|
|
}
|