mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Functions simplification [#CLICKHOUSE-2].
This commit is contained in:
parent
efaa44bea8
commit
ac5bc79057
@ -350,16 +350,10 @@ private:
|
||||
|
||||
for (const auto & br : branches)
|
||||
{
|
||||
if (! (ArraySourceCreator<TResult, UInt8>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, UInt16>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, UInt32>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, UInt64>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, Int8>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, Int16>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, Int32>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, Int64>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, Float32>::execute(sources, block, args, br)
|
||||
|| ArraySourceCreator<TResult, Float64>::execute(sources, block, args, br)
|
||||
if (!(dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return ArraySourceCreator<TResult, typename std::decay<decltype(*arg)>::type>::execute(sources, block, args, br);
|
||||
})
|
||||
|| ArraySourceCreator<TResult, Null>::execute(sources, block, args, br)))
|
||||
throw Exception{"Unexpected type of Array column in function multiIf with numeric Array arguments", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
@ -206,16 +206,10 @@ private:
|
||||
{
|
||||
NumericSourcePtr<TResult> source;
|
||||
|
||||
if (! (NumericSourceCreator<TResult, UInt8>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, UInt16>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, UInt32>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, UInt64>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, Int8>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, Int16>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, Int32>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, Int64>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, Float32>::execute(source, block, args, br)
|
||||
|| NumericSourceCreator<TResult, Float64>::execute(source, block, args, br)
|
||||
if (!(dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return NumericSourceCreator<TResult, typename std::decay<decltype(*arg)>::type>::execute(sources, block, args, br);
|
||||
})
|
||||
|| NumericSourceCreator<TResult, Null>::execute(source, block, args, br)))
|
||||
throw CondException{CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT, toString(br.index)};
|
||||
|
||||
|
@ -197,31 +197,19 @@ struct ThenPredicate final : public PredicateBase<TType>
|
||||
if (index2 != elseArg(args))
|
||||
{
|
||||
/// We have a pair Cond-Then. Process the next Then.
|
||||
if (! (ThenPredicate<TCombined, UInt8>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, UInt16>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, UInt32>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, UInt64>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, Int8>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, Int16>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, Int32>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, Int64>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, Float32>::execute(index2 + 1, block, args, result, builder, branches)
|
||||
|| ThenPredicate<TCombined, Float64>::execute(index2 + 1, block, args, result, builder, branches)))
|
||||
if (!dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return ThenPredicate<TCombined, typename std::decay<decltype(*arg)>::type>::execute(index2 + 1, block, args, result, builder, branches);
|
||||
}))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We have an Else which ends the multiIf. Process it.
|
||||
if (! (ElsePredicate<TCombined, UInt8>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, UInt16>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, UInt32>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, UInt64>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, Int8>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, Int16>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, Int32>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, Int64>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, Float32>::execute(index2, block, args, result, builder, branches)
|
||||
|| ElsePredicate<TCombined, Float64>::execute(index2, block, args, result, builder, branches)))
|
||||
if (!dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return ElsePredicate<TCombined, typename std::decay<decltype(*arg)>::type>::execute(index2, block, args, result, builder, branches);
|
||||
}))
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -259,16 +247,10 @@ struct FirstThenPredicate final
|
||||
using Void = NumberTraits::Enriched::Void<NumberTraits::HasNoNull>;
|
||||
Branches branches;
|
||||
|
||||
return ThenPredicate<Void, UInt8>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, UInt16>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, UInt32>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, UInt64>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, Int8>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, Int16>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, Int32>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, Int64>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, Float32>::execute(firstThen(), block, args, result, builder, branches)
|
||||
|| ThenPredicate<Void, Float64>::execute(firstThen(), block, args, result, builder, branches)
|
||||
return dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return ThenPredicate<Void, typename std::decay<decltype(*arg)>::type>::execute(firstThen(), block, args, result, builder, branches);
|
||||
})
|
||||
|| ThenPredicate<Void, Null>::execute(firstThen(), block, args, result, builder, branches);
|
||||
}
|
||||
};
|
||||
|
@ -84,4 +84,32 @@ inline Field toField(const T & x)
|
||||
|
||||
ColumnPtr convertConstTupleToTupleOfConstants(const ColumnConst & column);
|
||||
|
||||
|
||||
template <typename F>
|
||||
bool dispatchForFirstType(F && f)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename HeadArg, typename... TailArgs, typename F>
|
||||
bool dispatchForFirstType(F && f)
|
||||
{
|
||||
if (f(static_cast<HeadArg *>(nullptr)))
|
||||
return true;
|
||||
return dispatchForFirstType<TailArgs...>(std::forward<F>(f));
|
||||
}
|
||||
|
||||
|
||||
template <typename F>
|
||||
bool dispatchForFirstNumericType(F && f)
|
||||
{
|
||||
return dispatchForFirstType<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>(std::forward<F>(f));
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
bool dispatchForFirstIntegerType(F && f)
|
||||
{
|
||||
return dispatchForFirstType<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64>(std::forward<F>(f));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -482,7 +482,8 @@ template <> struct IsDateOrDateTime<DataTypeDateTime> { static constexpr auto va
|
||||
* least(Date, Date) -> Date
|
||||
* greatest(Date, Date) -> Date
|
||||
* All other operations are not defined and return InvalidType, operations on
|
||||
* distinct date types are also undefined (e.g. DataTypeDate - DataTypeDateTime) */
|
||||
* distinct date types are also undefined (e.g. DataTypeDate - DataTypeDateTime)
|
||||
*/
|
||||
template <template <typename, typename> class Operation, typename LeftDataType, typename RightDataType>
|
||||
struct DateBinaryOperationTraits
|
||||
{
|
||||
@ -498,11 +499,7 @@ struct DateBinaryOperationTraits
|
||||
Else<
|
||||
If<IsIntegral<LeftDataType>::value && IsDateOrDateTime<RightDataType>::value,
|
||||
Then<RightDataType>,
|
||||
Else<InvalidType>
|
||||
>
|
||||
>
|
||||
>
|
||||
>,
|
||||
Else<InvalidType>>>>>,
|
||||
Else<
|
||||
If<std::is_same<Op, MinusImpl<T0, T1>>::value,
|
||||
Then<
|
||||
@ -513,24 +510,13 @@ struct DateBinaryOperationTraits
|
||||
Else<
|
||||
If<IsIntegral<RightDataType>::value,
|
||||
Then<LeftDataType>,
|
||||
Else<InvalidType>
|
||||
>
|
||||
>
|
||||
>
|
||||
>,
|
||||
Else<InvalidType>
|
||||
>
|
||||
>,
|
||||
Else<InvalidType>>>>>,
|
||||
Else<InvalidType>>>,
|
||||
Else<
|
||||
If<std::is_same<T0, T1>::value
|
||||
&& (std::is_same<Op, LeastImpl<T0, T1>>::value || std::is_same<Op, GreatestImpl<T0, T1>>::value),
|
||||
Then<LeftDataType>,
|
||||
Else<InvalidType>
|
||||
>
|
||||
>
|
||||
>
|
||||
>
|
||||
>;
|
||||
Else<InvalidType>>>>>>;
|
||||
};
|
||||
|
||||
|
||||
@ -542,21 +528,43 @@ struct BinaryOperationTraits
|
||||
If<IsDateOrDateTime<LeftDataType>::value || IsDateOrDateTime<RightDataType>::value,
|
||||
Then<
|
||||
typename DateBinaryOperationTraits<
|
||||
Operation, LeftDataType, RightDataType
|
||||
>::ResultDataType
|
||||
>,
|
||||
Operation, LeftDataType, RightDataType>::ResultDataType>,
|
||||
Else<
|
||||
typename DataTypeFromFieldType<
|
||||
typename Operation<
|
||||
typename LeftDataType::FieldType,
|
||||
typename RightDataType::FieldType
|
||||
>::ResultType
|
||||
>::Type
|
||||
>
|
||||
>;
|
||||
typename RightDataType::FieldType>::ResultType>::Type>>;
|
||||
};
|
||||
|
||||
|
||||
template <typename F>
|
||||
bool dispatchForFirstNumericOrTimeDataType(F && f)
|
||||
{
|
||||
return dispatchForFirstType<
|
||||
DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64,
|
||||
DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64,
|
||||
DataTypeFloat32, DataTypeFloat64,
|
||||
DataTypeDate, DataTypeDateTime>(std::forward<F>(f));
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
bool dispatchForFirstNumericDataType(F && f)
|
||||
{
|
||||
return dispatchForFirstType<
|
||||
DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64,
|
||||
DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64,
|
||||
DataTypeFloat32, DataTypeFloat64>(std::forward<F>(f));
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
bool dispatchForFirstIntegerDataType(F && f)
|
||||
{
|
||||
return dispatchForFirstType<
|
||||
DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64,
|
||||
DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64>(std::forward<F>(f));
|
||||
}
|
||||
|
||||
|
||||
template <template <typename, typename> class Op, typename Name>
|
||||
class FunctionBinaryArithmetic : public IFunction
|
||||
{
|
||||
@ -587,7 +595,7 @@ private:
|
||||
{
|
||||
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
|
||||
|
||||
if (typeid_cast<const RightDataType *>(&*arguments[1]))
|
||||
if (typeid_cast<const RightDataType *>(arguments[1].get()))
|
||||
return checkRightTypeImpl<ResultDataType>(type_res);
|
||||
|
||||
return false;
|
||||
@ -596,24 +604,15 @@ private:
|
||||
template <typename T0>
|
||||
bool checkLeftType(const DataTypes & arguments, DataTypePtr & type_res) const
|
||||
{
|
||||
if (typeid_cast<const T0 *>(&*arguments[0]))
|
||||
if (typeid_cast<const T0 *>(arguments[0].get()))
|
||||
{
|
||||
if ( checkRightType<T0, DataTypeDate>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeDateTime>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt8>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt16>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt32>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt64>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt8>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt16>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt32>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt64>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeFloat32>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeFloat64>(arguments, type_res))
|
||||
return true;
|
||||
else
|
||||
if (!dispatchForFirstNumericOrTimeDataType([&] (auto arg)
|
||||
{
|
||||
return checkRightType<T0, typename std::decay<decltype(*arg)>::type>(arguments, type_res);
|
||||
}))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
@ -726,43 +725,25 @@ private:
|
||||
{
|
||||
if (auto col_left = checkAndGetColumn<ColumnVector<typename LeftDataType::FieldType>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
if ( executeRightType<LeftDataType, DataTypeDate>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt8>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt16>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt32>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt64>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt8>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt16>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt32>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt64>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeFloat32>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeFloat64>(block, arguments, result, col_left))
|
||||
return true;
|
||||
else
|
||||
if (!dispatchForFirstNumericOrTimeDataType([&] (auto arg)
|
||||
{
|
||||
return executeRightType<LeftDataType, typename std::decay<decltype(*arg)>::type>(block, arguments, result, col_left);
|
||||
}))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
return true;
|
||||
}
|
||||
else if (auto col_left = checkAndGetColumnConst<ColumnVector<typename LeftDataType::FieldType>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
if ( executeRightType<LeftDataType, DataTypeDate>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt8>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt16>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt32>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeUInt64>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt8>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt16>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt32>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeInt64>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeFloat32>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeFloat64>(block, arguments, result, col_left))
|
||||
return true;
|
||||
else
|
||||
if (!dispatchForFirstNumericOrTimeDataType([&] (auto arg)
|
||||
{
|
||||
return executeRightType<LeftDataType, typename std::decay<decltype(*arg)>::type>(block, arguments, result, col_left);
|
||||
}))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
@ -780,18 +761,10 @@ public:
|
||||
{
|
||||
DataTypePtr type_res;
|
||||
|
||||
if (!( checkLeftType<DataTypeDate>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeDateTime>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeUInt8>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeUInt16>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeUInt32>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeUInt64>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt8>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt16>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt32>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt64>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeFloat32>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeFloat64>(arguments, type_res)))
|
||||
if (!dispatchForFirstNumericOrTimeDataType([&] (auto arg)
|
||||
{
|
||||
return checkLeftType<typename std::decay<decltype(*arg)>::type>(arguments, type_res);
|
||||
}))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -800,18 +773,10 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeLeftType<DataTypeDate>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeDateTime>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeUInt8>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeUInt16>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeUInt32>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeUInt64>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeInt8>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeInt16>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeInt32>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeInt64>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeFloat32>(block, arguments, result)
|
||||
|| executeLeftType<DataTypeFloat64>(block, arguments, result)))
|
||||
if (!dispatchForFirstNumericOrTimeDataType([&] (auto arg)
|
||||
{
|
||||
return executeLeftType<typename std::decay<decltype(*arg)>::type>(block, arguments, result);
|
||||
}))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -834,7 +799,7 @@ private:
|
||||
template <typename T0>
|
||||
bool checkType(const DataTypes & arguments, DataTypePtr & result) const
|
||||
{
|
||||
if (typeid_cast<const T0 *>(&*arguments[0]))
|
||||
if (typeid_cast<const T0 *>(arguments[0].get()))
|
||||
{
|
||||
result = std::make_shared<DataTypeNumber<typename Op<typename T0::FieldType>::ResultType>>();
|
||||
return true;
|
||||
@ -858,16 +823,6 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConst<ColumnVector<T0>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
using ResultType = typename Op<T0>::ResultType;
|
||||
|
||||
ResultType res = 0;
|
||||
UnaryOperationImpl<T0, Op<T0>>::constant(col->template getValue<T0>(), res);
|
||||
block.getByPosition(result).column = DataTypeNumber<ResultType>().createConstColumn(col->size(), toField(res));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
@ -885,34 +840,24 @@ public:
|
||||
{
|
||||
DataTypePtr result;
|
||||
|
||||
if (!( checkType<DataTypeUInt8>(arguments, result)
|
||||
|| checkType<DataTypeUInt16>(arguments, result)
|
||||
|| checkType<DataTypeUInt32>(arguments, result)
|
||||
|| checkType<DataTypeUInt64>(arguments, result)
|
||||
|| checkType<DataTypeInt8>(arguments, result)
|
||||
|| checkType<DataTypeInt16>(arguments, result)
|
||||
|| checkType<DataTypeInt32>(arguments, result)
|
||||
|| checkType<DataTypeInt64>(arguments, result)
|
||||
|| checkType<DataTypeFloat32>(arguments, result)
|
||||
|| checkType<DataTypeFloat64>(arguments, result)))
|
||||
if (!dispatchForFirstNumericDataType([&] (auto arg)
|
||||
{
|
||||
return checkType<typename std::decay<decltype(*arg)>::type>(arguments, result);
|
||||
}))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeType<UInt8>(block, arguments, result)
|
||||
|| executeType<UInt16>(block, arguments, result)
|
||||
|| executeType<UInt32>(block, arguments, result)
|
||||
|| executeType<UInt64>(block, arguments, result)
|
||||
|| executeType<Int8>(block, arguments, result)
|
||||
|| executeType<Int16>(block, arguments, result)
|
||||
|| executeType<Int32>(block, arguments, result)
|
||||
|| executeType<Int64>(block, arguments, result)
|
||||
|| executeType<Float32>(block, arguments, result)
|
||||
|| executeType<Float64>(block, arguments, result)))
|
||||
if (!dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return executeType<typename std::decay<decltype(*arg)>::type>(block, arguments, result);
|
||||
}))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -1177,14 +1122,11 @@ public:
|
||||
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
|
||||
|
||||
const auto first_arg = arguments.front().get();
|
||||
if (!checkDataType<DataTypeUInt8>(first_arg)
|
||||
&& !checkDataType<DataTypeUInt16>(first_arg)
|
||||
&& !checkDataType<DataTypeUInt32>(first_arg)
|
||||
&& !checkDataType<DataTypeUInt64>(first_arg)
|
||||
&& !checkDataType<DataTypeInt8>(first_arg)
|
||||
&& !checkDataType<DataTypeInt16>(first_arg)
|
||||
&& !checkDataType<DataTypeInt32>(first_arg)
|
||||
&& !checkDataType<DataTypeInt64>(first_arg))
|
||||
|
||||
if (!dispatchForFirstIntegerDataType([&] (auto arg)
|
||||
{
|
||||
return checkDataType<typename std::decay<decltype(*arg)>::type>(first_arg);
|
||||
}))
|
||||
throw Exception{
|
||||
"Illegal type " + first_arg->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
@ -1210,14 +1152,10 @@ public:
|
||||
{
|
||||
const auto value_col = block.getByPosition(arguments.front()).column.get();
|
||||
|
||||
if (!execute<UInt8>(block, arguments, result, value_col)
|
||||
&& !execute<UInt16>(block, arguments, result, value_col)
|
||||
&& !execute<UInt32>(block, arguments, result, value_col)
|
||||
&& !execute<UInt64>(block, arguments, result, value_col)
|
||||
&& !execute<Int8>(block, arguments, result, value_col)
|
||||
&& !execute<Int16>(block, arguments, result, value_col)
|
||||
&& !execute<Int32>(block, arguments, result, value_col)
|
||||
&& !execute<Int64>(block, arguments, result, value_col))
|
||||
if (!dispatchForFirstIntegerType([&] (auto arg)
|
||||
{
|
||||
return execute<typename std::decay<decltype(*arg)>::type>(block, arguments, result, value_col);
|
||||
}))
|
||||
throw Exception{
|
||||
"Illegal column " + value_col->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
@ -755,16 +755,10 @@ private:
|
||||
template <typename T>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
return executeNumberNumber<T, UInt8>(block, arguments, result)
|
||||
|| executeNumberNumber<T, UInt16>(block, arguments, result)
|
||||
|| executeNumberNumber<T, UInt32>(block, arguments, result)
|
||||
|| executeNumberNumber<T, UInt64>(block, arguments, result)
|
||||
|| executeNumberNumber<T, Int8>(block, arguments, result)
|
||||
|| executeNumberNumber<T, Int16>(block, arguments, result)
|
||||
|| executeNumberNumber<T, Int32>(block, arguments, result)
|
||||
|| executeNumberNumber<T, Int64>(block, arguments, result)
|
||||
|| executeNumberNumber<T, Float32>(block, arguments, result)
|
||||
|| executeNumberNumber<T, Float64>(block, arguments, result)
|
||||
return dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return executeNumberNumber<T, typename std::decay<decltype(*arg)>::type>(block, arguments, result);
|
||||
})
|
||||
|| executeNumberNumber<T, Null>(block, arguments, result);
|
||||
}
|
||||
|
||||
@ -1147,16 +1141,10 @@ private:
|
||||
/// Perform function on the given block. Internal version.
|
||||
void perform(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (!(executeNumber<UInt8>(block, arguments, result)
|
||||
|| executeNumber<UInt16>(block, arguments, result)
|
||||
|| executeNumber<UInt32>(block, arguments, result)
|
||||
|| executeNumber<UInt64>(block, arguments, result)
|
||||
|| executeNumber<Int8>(block, arguments, result)
|
||||
|| executeNumber<Int16>(block, arguments, result)
|
||||
|| executeNumber<Int32>(block, arguments, result)
|
||||
|| executeNumber<Int64>(block, arguments, result)
|
||||
|| executeNumber<Float32>(block, arguments, result)
|
||||
|| executeNumber<Float64>(block, arguments, result)
|
||||
if (!(dispatchForFirstNumericType([&] (auto arg)
|
||||
{
|
||||
return executeNumber<typename std::decay<decltype(*arg)>::type>(block, arguments, result);
|
||||
})
|
||||
|| executeConst(block, arguments, result)
|
||||
|| executeString(block, arguments, result)
|
||||
|| executeGeneric(block, arguments, result)))
|
||||
|
@ -176,6 +176,9 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnWithTypeAndName & arg_from = block.getByPosition(arguments[0]);
|
||||
@ -199,16 +202,6 @@ public:
|
||||
convert(charset_from, charset_to, col_from->getChars(), col_from->getOffsets(), col_to->getChars(), col_to->getOffsets());
|
||||
res.column = col_to;
|
||||
}
|
||||
else if (const ColumnConst * col_from = checkAndGetColumnConst<ColumnString>(arg_from.column.get()))
|
||||
{
|
||||
auto full_column_holder = col_from->cloneResized(1)->convertToFullColumnIfConst();
|
||||
const ColumnString * col_from_full = static_cast<const ColumnString *>(full_column_holder.get());
|
||||
|
||||
auto col_to_full = std::make_shared<ColumnString>();
|
||||
convert(charset_from, charset_to, col_from_full->getChars(), col_from_full->getOffsets(), col_to_full->getChars(), col_to_full->getOffsets());
|
||||
|
||||
res.column = DataTypeString().createConstColumn(col_from->size(), (*col_to_full)[0].get<String>());
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column passed as first argument of function " + getName() + " (must be ColumnString).",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
@ -81,6 +81,8 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto & col_type_name = block.getByPosition(arguments[0]);
|
||||
@ -117,24 +119,6 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (const auto col_in = checkAndGetColumnConst<ColumnFixedString>(column.get()))
|
||||
{
|
||||
const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(&col_in->getDataColumn());
|
||||
if (!column_fixed_string || column_fixed_string->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
String data_in = col_in->getValue<String>();
|
||||
|
||||
char buf[IPV6_MAX_TEXT_LENGTH + 1];
|
||||
char * dst = buf;
|
||||
formatIPv6(reinterpret_cast<const unsigned char *>(data_in.data()), dst);
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_in->size(), String(buf));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -175,6 +159,9 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto & col_type_name = block.getByPosition(arguments[0]);
|
||||
@ -242,51 +229,6 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (const auto col_in = checkAndGetColumnConst<ColumnFixedString>(column.get()))
|
||||
{
|
||||
const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(&col_in->getDataColumn());
|
||||
if (!column_fixed_string || column_fixed_string->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto ipv6_zeroed_tail_bytes = checkAndGetColumnConst<ColumnVector<UInt8>>(col_ipv6_zeroed_tail_bytes.get());
|
||||
if (!ipv6_zeroed_tail_bytes)
|
||||
throw Exception("Illegal type " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of argument 2 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getValue<UInt8>();
|
||||
if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||
throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto ipv4_zeroed_tail_bytes = checkAndGetColumnConst<ColumnVector<UInt8>>(col_ipv4_zeroed_tail_bytes.get());
|
||||
if (!ipv4_zeroed_tail_bytes)
|
||||
throw Exception("Illegal type " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
|
||||
" of argument 3 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getValue<UInt8>();
|
||||
if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||
throw Exception("Illegal value for argument 3 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||
" of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
String data_in = col_in->getValue<String>();
|
||||
|
||||
char buf[IPV6_MAX_TEXT_LENGTH + 1];
|
||||
char * dst = buf;
|
||||
|
||||
const auto address = reinterpret_cast<const unsigned char *>(data_in.data());
|
||||
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
|
||||
cutAddress(address, dst, zeroed_tail_bytes_count);
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_in->size(), String(buf));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -464,6 +406,8 @@ public:
|
||||
memcpy(dst, tmp, sizeof(tmp));
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
@ -488,13 +432,6 @@ public:
|
||||
src_offset = offsets_src[i];
|
||||
}
|
||||
}
|
||||
else if (const auto col_in = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
String out(ipv6_bytes_length, 0);
|
||||
ipv6_scan(col_in->getValue<String>().data(), reinterpret_cast<unsigned char *>(&out[0]));
|
||||
|
||||
block.getByPosition(result).column = DataTypeFixedString(ipv6_bytes_length).createConstColumn(col_in->size(), out);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -560,6 +497,8 @@ public:
|
||||
*(out++) = '\0';
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
@ -587,15 +526,6 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConst<ColumnUInt32>(column.get()))
|
||||
{
|
||||
char buf[16];
|
||||
char * pos = buf;
|
||||
formatIP(col->getValue<UInt32>(), pos);
|
||||
|
||||
auto col_res = DataTypeString().createConstColumn(col->size(), String(buf));
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -649,6 +579,8 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
@ -671,11 +603,6 @@ public:
|
||||
prev_offset = offsets_src[i];
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
auto col_res = DataTypeUInt32().createConstColumn(col->size(), toField(parseIPv4(col->getValue<String>().c_str())));
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -743,6 +670,8 @@ public:
|
||||
*(out++) = '\0';
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
@ -770,15 +699,6 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConst<ColumnUInt32>(column.get()))
|
||||
{
|
||||
char buf[16];
|
||||
char * pos = buf;
|
||||
formatIP(col->getValue<UInt32>(), pos);
|
||||
|
||||
auto col_res = DataTypeString().createConstColumn(col->size(), String(buf));
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -807,6 +727,8 @@ public:
|
||||
return std::make_shared<DataTypeFixedString>(16);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto & col_type_name = block.getByPosition(arguments[0]);
|
||||
@ -825,13 +747,6 @@ public:
|
||||
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += ipv6_bytes_length, ++i)
|
||||
mapIPv4ToIPv6(vec_in[i], &vec_res[out_offset]);
|
||||
}
|
||||
else if (const auto col_in = checkAndGetColumnConst<ColumnVector<UInt32>>(column.get()))
|
||||
{
|
||||
std::string buf;
|
||||
buf.resize(ipv6_bytes_length);
|
||||
mapIPv4ToIPv6(col_in->getValue<UInt32>(), reinterpret_cast<unsigned char *>(&buf[0]));
|
||||
block.getByPosition(result).column = DataTypeFixedString(ipv6_bytes_length).createConstColumn(ipv6_bytes_length, buf);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -889,6 +804,8 @@ public:
|
||||
out[17] = '\0';
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
@ -914,15 +831,6 @@ public:
|
||||
offsets_res[i] = current_offset;
|
||||
}
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConst<ColumnUInt64>(column.get()))
|
||||
{
|
||||
unsigned char buf[18];
|
||||
unsigned char * pos = buf;
|
||||
formatMAC(col->getValue<UInt64>(), pos);
|
||||
|
||||
auto col_res = DataTypeString().createConstColumn(col->size(), String(reinterpret_cast<const char *>(buf)));
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -1005,6 +913,8 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
@ -1034,15 +944,6 @@ public:
|
||||
prev_offset = current_offset;
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
UInt64 res = 0;
|
||||
StringRef src = col->getDataAt(0);
|
||||
if (src.size >= Impl::min_string_size && src.size <= Impl::max_string_size)
|
||||
res = Impl::parse(src.data);
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(col->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -1078,6 +979,8 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
|
||||
@ -1116,23 +1019,6 @@ public:
|
||||
offsets_res[i] = dst_offset;
|
||||
}
|
||||
}
|
||||
else if (const auto col_in = checkAndGetColumnConst<ColumnFixedString>(column.get()))
|
||||
{
|
||||
const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(&col_in->getDataColumn());
|
||||
if (!column_fixed_string || column_fixed_string->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
String data_in = col_in->getValue<String>();
|
||||
|
||||
char buf[uuid_text_length];
|
||||
formatUUID(reinterpret_cast<const UInt8 *>(data_in.data()), reinterpret_cast<UInt8 *>(buf));
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_in->size(), String(buf, uuid_text_length));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -1194,6 +1080,8 @@ public:
|
||||
return std::make_shared<DataTypeFixedString>(uuid_bytes_length);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
|
||||
@ -1257,23 +1145,6 @@ public:
|
||||
dst_offset += uuid_bytes_length;
|
||||
}
|
||||
}
|
||||
else if (const auto col_in = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
String data_in = col_in->getValue<String>();
|
||||
|
||||
String res;
|
||||
|
||||
if (data_in.size() == uuid_text_length)
|
||||
{
|
||||
char buf[uuid_bytes_length];
|
||||
parseUUID(reinterpret_cast<const UInt8 *>(data_in.data()), reinterpret_cast<UInt8 *>(buf));
|
||||
res.assign(buf, uuid_bytes_length);
|
||||
}
|
||||
else
|
||||
res.resize(uuid_bytes_length, '\0');
|
||||
|
||||
block.getByPosition(result).column = DataTypeFixedString(uuid_bytes_length).createConstColumn(col_in->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
@ -1337,7 +1208,6 @@ public:
|
||||
bool tryExecuteUInt(const IColumn * col, ColumnPtr & col_res)
|
||||
{
|
||||
const ColumnVector<T> * col_vec = checkAndGetColumn<ColumnVector<T>>(col);
|
||||
const ColumnConst * col_const = checkAndGetColumnConst<ColumnVector<T>>(col);
|
||||
|
||||
static constexpr size_t MAX_UINT_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte.
|
||||
|
||||
@ -1373,16 +1243,6 @@ public:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (col_const)
|
||||
{
|
||||
char buf[MAX_UINT_HEX_LENGTH];
|
||||
char * pos = buf;
|
||||
executeOneUInt<T>(col_const->template getValue<T>(), pos);
|
||||
|
||||
col_res = DataTypeString().createConstColumn(col_const->size(), String(buf));
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
@ -1404,7 +1264,6 @@ public:
|
||||
bool tryExecuteString(const IColumn * col, ColumnPtr & col_res)
|
||||
{
|
||||
const ColumnString * col_str_in = checkAndGetColumn<ColumnString>(col);
|
||||
const ColumnConst * col_const_in = checkAndGetColumnConstStringOrFixedString(col);
|
||||
|
||||
if (col_str_in)
|
||||
{
|
||||
@ -1440,19 +1299,6 @@ public:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (col_const_in)
|
||||
{
|
||||
String src = col_const_in->getValue<String>();
|
||||
String res(src.size() * 2, '\0');
|
||||
char * pos = &res[0];
|
||||
const UInt8 * src_ptr = reinterpret_cast<const UInt8 *>(src.c_str());
|
||||
/// Let's write zero into res[res.size()]. Starting with C++ 11, this is correct.
|
||||
executeOneString(src_ptr, src_ptr + src.size(), pos);
|
||||
|
||||
col_res = DataTypeString().createConstColumn(col_const_in->size(), res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
@ -1507,6 +1353,8 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const IColumn * column = block.getByPosition(arguments[0]).column.get();
|
||||
@ -1572,6 +1420,8 @@ public:
|
||||
++out;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
@ -1608,16 +1458,6 @@ public:
|
||||
|
||||
out_vec.resize(pos - begin);
|
||||
}
|
||||
else if(const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
String src = col->getValue<String>();
|
||||
String res(src.size(), '\0');
|
||||
char * pos = &res[0];
|
||||
unhexOne(src.c_str(), src.c_str() + src.size(), pos);
|
||||
res = res.substr(0, pos - &res[0] - 1);
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -1658,6 +1498,8 @@ public:
|
||||
return std::make_shared<DataTypeArray>(arguments[0]);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
template <typename T>
|
||||
bool tryExecute(const IColumn * column, ColumnPtr & out_column)
|
||||
{
|
||||
@ -1690,23 +1532,6 @@ public:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<T>>(column))
|
||||
{
|
||||
Array res;
|
||||
|
||||
T x = col_from->template getValue<T>();
|
||||
for (size_t i = 0; i < sizeof(T) * 8; ++i)
|
||||
{
|
||||
T bit = static_cast<T>(1) << i;
|
||||
if (x & bit)
|
||||
{
|
||||
res.push_back(static_cast<UInt64>(bit));
|
||||
}
|
||||
}
|
||||
|
||||
out_column = DataTypeArray(std::make_shared<DataTypeNumber<T>>()).createConstColumn(col_from->size(), res);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
@ -1757,11 +1582,11 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
bool tryExecuteString(const IColumn * col, ColumnPtr & col_res)
|
||||
{
|
||||
const ColumnString * col_str_in = checkAndGetColumn<ColumnString>(col);
|
||||
const ColumnConst * col_const_in = checkAndGetColumnConstStringOrFixedString(col);
|
||||
|
||||
if (col_str_in)
|
||||
{
|
||||
@ -1800,13 +1625,6 @@ public:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (col_const_in)
|
||||
{
|
||||
std::string res(col_const_in->getValue<String>().c_str());
|
||||
col_res = DataTypeString().createConstColumn(col_const_in->size(), res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
|
@ -82,12 +82,6 @@ struct ConvertImpl
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
|
||||
}
|
||||
else if (auto col_from
|
||||
= checkAndGetColumnConst<ColumnVector<FromFieldType>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeNumber<ToFieldType>().createConstColumn(
|
||||
col_from->size(), toField(static_cast<ToFieldType>(col_from->template getValue<FromFieldType>())));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -240,13 +234,6 @@ struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
else if (const auto col_from = checkAndGetColumnConst<ColumnVector<FromFieldType>>(col_with_type_and_name.column.get()))
|
||||
{
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<std::vector<char>> write_buffer(buf);
|
||||
FormatImpl<FromDataType>::execute(col_from->template getValue<FromFieldType>(), write_buffer, &type, time_zone);
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_from->size(), std::string(&buf[0], write_buffer.count()));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -379,20 +366,6 @@ struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst * col_from = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
StringRef s = col_from->getDataAt(0);
|
||||
ReadBufferFromMemory read_buffer(s.data, s.size);
|
||||
ToFieldType x(0);
|
||||
parseImpl<ToDataType>(x, read_buffer, time_zone);
|
||||
|
||||
if (!read_buffer.eof()
|
||||
&& !(std::is_same<ToDataType, DataTypeDate>::value /// Special exception, that allows to parse string with DateTime as Date.
|
||||
&& s.size == strlen("YYYY-MM-DD hh:mm:ss")))
|
||||
throwExceptionForIncompletelyParsedValue(read_buffer, block, arguments, result);
|
||||
|
||||
block.getByPosition(result).column = DataTypeNumber<ToFieldType>().createConstColumn(col_from->size(), toField(x));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -450,15 +423,6 @@ struct ConvertOrZeroImpl
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst * col_from = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
StringRef s = col_from->getDataAt(0);
|
||||
ReadBufferFromMemory read_buffer(s.data, s.size);
|
||||
ToFieldType x = 0;
|
||||
if (!tryParseImpl<ToDataType>(x, read_buffer) || !read_buffer.eof())
|
||||
x = 0;
|
||||
block.getByPosition(result).column = DataTypeNumber<ToFieldType>().createConstColumn(col_from->size(), toField(x));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -505,19 +469,6 @@ struct ConvertImplGenericFromString
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
else if (checkAndGetColumnConstStringOrFixedString(&col_from))
|
||||
{
|
||||
StringRef s = col_from.getDataAt(0);
|
||||
ReadBufferFromMemory read_buffer(s.data, s.size);
|
||||
|
||||
auto tmp_col = data_type_to.createColumn();
|
||||
data_type_to.deserializeTextEscaped(*tmp_col, read_buffer);
|
||||
|
||||
if (!read_buffer.eof())
|
||||
throwExceptionForIncompletelyParsedValue(read_buffer, block, arguments, result);
|
||||
|
||||
block.getByPosition(result).column = data_type_to.createConstColumn(size, (*tmp_col)[0]);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of conversion function from string",
|
||||
@ -589,10 +540,6 @@ struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (checkColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -640,16 +587,6 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
|
||||
data_to.resize(offset_to);
|
||||
}
|
||||
else if (const ColumnConst * col_from = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
String s = col_from->getValue<String>();
|
||||
|
||||
size_t bytes_to_copy = s.size();
|
||||
while (bytes_to_copy > 0 && s[bytes_to_copy - 1] == 0)
|
||||
--bytes_to_copy;
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_from->size(), s.substr(0, bytes_to_copy));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -686,6 +623,9 @@ public:
|
||||
return getReturnTypeInternal(arguments);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
try
|
||||
@ -801,8 +741,7 @@ private:
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
return std::make_shared<ToDataType2>();
|
||||
@ -828,8 +767,7 @@ private:
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
return std::make_shared<ToDataType2>();
|
||||
@ -848,8 +786,7 @@ private:
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of 2nd argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
return std::make_shared<ToDataType2>();
|
||||
@ -881,6 +818,8 @@ public:
|
||||
return std::make_shared<ToDataType>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
@ -926,6 +865,9 @@ public:
|
||||
out_return_type = std::make_shared<DataTypeFixedString>(n);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto n = getSize(block.getByPosition(arguments[1]));
|
||||
@ -936,19 +878,7 @@ public:
|
||||
{
|
||||
const auto & column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const auto column_const = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
if (column_const->getValue<String>().size() > n)
|
||||
throw Exception("String too long for type FixedString(" + toString(n) + ")",
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
||||
String resized_string = column_const->getValue<String>();
|
||||
resized_string.resize(n);
|
||||
|
||||
block.getByPosition(result).column = DataTypeFixedString(n).createConstColumn(
|
||||
column_const->size(), std::move(resized_string));
|
||||
}
|
||||
else if (const auto column_string = checkAndGetColumn<ColumnString>(column.get()))
|
||||
if (const auto column_string = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
const auto column_fixed = std::make_shared<ColumnFixedString>(n);
|
||||
ColumnPtr result_ptr = column_fixed;
|
||||
@ -977,8 +907,7 @@ public:
|
||||
if (src_n > n)
|
||||
throw Exception{
|
||||
"String too long for type FixedString(" + toString(n) + ")",
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE
|
||||
};
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE};
|
||||
|
||||
const auto column_fixed = std::make_shared<ColumnFixedString>(n);
|
||||
block.getByPosition(result).column = column_fixed;
|
||||
@ -1242,8 +1171,7 @@ private:
|
||||
!checkDataType<DataTypeFixedString>(from_type.get()))
|
||||
throw Exception{
|
||||
"CAST AS FixedString is only implemented for types String and FixedString",
|
||||
ErrorCodes::NOT_IMPLEMENTED
|
||||
};
|
||||
ErrorCodes::NOT_IMPLEMENTED};
|
||||
|
||||
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
@ -1279,8 +1207,7 @@ private:
|
||||
if (from_type || to_type)
|
||||
throw Exception{
|
||||
"CAST AS Array can only be performed between same-dimensional array types or from String",
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
ErrorCodes::TYPE_MISMATCH};
|
||||
|
||||
/// Prepare nested type conversion
|
||||
const auto nested_function = prepareImpl(from_nested_type, to_nested_type.get());
|
||||
@ -1290,10 +1217,6 @@ private:
|
||||
{
|
||||
auto array_arg = block.getByPosition(arguments.front());
|
||||
|
||||
/// @todo add const variant which retains array constness
|
||||
if (const auto col_const_array = checkAndGetColumnConst<ColumnArray>(array_arg.column.get()))
|
||||
array_arg.column = col_const_array->convertToFullColumn();
|
||||
|
||||
if (auto col_array = checkAndGetColumn<ColumnArray>(array_arg.column.get()))
|
||||
{
|
||||
auto res = new ColumnArray(nullptr, col_array->getOffsetsColumn());
|
||||
@ -1345,15 +1268,13 @@ private:
|
||||
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
|
||||
};
|
||||
ErrorCodes::TYPE_MISMATCH};
|
||||
|
||||
if (from_type->getElements().size() != to_type->getElements().size())
|
||||
throw Exception{
|
||||
"CAST AS Tuple can only be performed between tuple types with the same number of elements or from String.\n"
|
||||
"Left type: " + from_type->getName() + ", right type: " + to_type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
ErrorCodes::TYPE_MISMATCH};
|
||||
|
||||
const auto & from_element_types = from_type->getElements();
|
||||
const auto & to_element_types = to_type->getElements();
|
||||
@ -1376,8 +1297,6 @@ private:
|
||||
/// @todo retain constness
|
||||
if (const auto column_tuple = typeid_cast<const ColumnTuple *>(col))
|
||||
element_block = column_tuple->getData();
|
||||
else if (const auto column_const_tuple = checkAndGetColumnConst<ColumnTuple>(col))
|
||||
element_block = static_cast<const ColumnTuple &>(*convertConstTupleToTupleOfConstants(*column_const_tuple)).getData();
|
||||
|
||||
/// create columns for converted elements
|
||||
for (const auto & to_element_type : to_element_types)
|
||||
@ -1436,8 +1355,7 @@ private:
|
||||
throw Exception{
|
||||
"Conversion from " + from_type->getName() + " to " + to_type->getName() +
|
||||
" is not supported",
|
||||
ErrorCodes::CANNOT_CONVERT_TYPE
|
||||
};
|
||||
ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
}
|
||||
|
||||
template <typename EnumTypeFrom, typename EnumTypeTo>
|
||||
@ -1463,8 +1381,7 @@ private:
|
||||
throw Exception{
|
||||
"Enum conversion changes value for element '" + name_value.first +
|
||||
"' from " + toString(old_value) + " to " + toString(new_value),
|
||||
ErrorCodes::CANNOT_CONVERT_TYPE
|
||||
};
|
||||
ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
}
|
||||
};
|
||||
|
||||
@ -1492,17 +1409,11 @@ private:
|
||||
|
||||
result_col = res;
|
||||
}
|
||||
else if (const auto const_col = checkAndGetColumnConstStringOrFixedString(first_col))
|
||||
{
|
||||
result_col = result_type.createConstColumn(const_col->size(),
|
||||
nearestFieldType(result_type.getValue(const_col->getValue<String>())));
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
"Unexpected column " + first_col->getName() + " as first argument of function " +
|
||||
name,
|
||||
ErrorCodes::LOGICAL_ERROR
|
||||
};
|
||||
ErrorCodes::LOGICAL_ERROR};
|
||||
};
|
||||
}
|
||||
|
||||
@ -1647,8 +1558,7 @@ private:
|
||||
throw Exception{
|
||||
"Conversion from " + from_type->getName() + " to " + to_type->getName() +
|
||||
" is not supported",
|
||||
ErrorCodes::CANNOT_CONVERT_TYPE
|
||||
};
|
||||
ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
}
|
||||
|
||||
template <typename DataType> static auto monotonicityForType(const DataType * const)
|
||||
@ -1766,6 +1676,9 @@ public:
|
||||
prepareMonotonicityInformation(from_inner_type, to_inner_type);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
/// drop second argument, pass others
|
||||
|
@ -494,7 +494,6 @@ struct DateTimeTransformImpl
|
||||
|
||||
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
|
||||
const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get());
|
||||
const auto * const_source = checkAndGetColumnConst<ColumnVector<FromType>>(source_col.get());
|
||||
|
||||
const ColumnConst * time_zone_column = nullptr;
|
||||
|
||||
@ -518,12 +517,6 @@ struct DateTimeTransformImpl
|
||||
block.getByPosition(result).column = col_to;
|
||||
Op::vector(sources->getData(), col_to->getData(), time_zone);
|
||||
}
|
||||
else if (const_source)
|
||||
{
|
||||
ToType res;
|
||||
Op::constant(const_source->template getValue<FromType>(), res, time_zone);
|
||||
block.getByPosition(result).column = DataTypeNumber<ToType>().createConstColumn(const_source->size(), toField(res));
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -578,6 +571,9 @@ public:
|
||||
return std::make_shared<ToDataType>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
@ -736,6 +732,8 @@ public:
|
||||
return std::make_shared<DataTypeDateTime>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (const ColumnUInt32 * times = typeid_cast<const ColumnUInt32 *>(block.getByPosition(arguments[0]).column.get()))
|
||||
@ -753,11 +751,6 @@ public:
|
||||
|
||||
block.getByPosition(result).column = res_holder;
|
||||
}
|
||||
else if (auto const_times = checkAndGetColumnConst<ColumnUInt32>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeUInt32().createConstColumn(
|
||||
block.rows(), toField(const_times->getValue<UInt32>() / TIME_SLOT_SIZE * TIME_SLOT_SIZE));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
|
@ -215,6 +215,9 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
/// The dictionary key that defines the "point of view".
|
||||
@ -248,11 +251,6 @@ public:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(vec_from[i], dict);
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeNumber<T>().createConstColumn(
|
||||
col_from->size(), toField(Transform::apply(col_from->template getValue<T>(), dict)));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + name,
|
||||
@ -441,6 +439,9 @@ public:
|
||||
return std::make_shared<DataTypeArray>(arguments[0]);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
/// The dictionary key that defines the "point of view".
|
||||
@ -486,19 +487,6 @@ public:
|
||||
res_offsets[i] = res_values.size();
|
||||
}
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
Array res;
|
||||
|
||||
T cur = col_from->template getValue<T>();
|
||||
while (cur)
|
||||
{
|
||||
res.push_back(static_cast<typename NearestFieldType<T>::Type>(cur));
|
||||
cur = Transform::toParent(cur, dict);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(col_from->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + name,
|
||||
@ -724,6 +712,9 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
RegionsNames::Language language = RegionsNames::Language::RU;
|
||||
@ -754,13 +745,6 @@ public:
|
||||
col_to->insertDataWithTerminatingZero(name_ref.data, name_ref.size + 1);
|
||||
}
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<UInt32>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
UInt32 region_id = col_from->getValue<UInt32>();
|
||||
const StringRef & name_ref = dict.getRegionName(region_id, language);
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_from->size(), name_ref.toString());
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of the first argument of function " + getName(),
|
||||
|
@ -60,6 +60,8 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeType<UInt8>(block, arguments, result)
|
||||
@ -117,16 +119,6 @@ private:
|
||||
}
|
||||
data_to.resize(buf_to.count());
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
std::string res;
|
||||
{
|
||||
WriteBufferFromString buf(res);
|
||||
writeBitmask<T>(col_from->template getValue<T>(), buf);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_from->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
@ -160,6 +152,8 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeType<UInt8>(block, arguments, result)
|
||||
@ -203,11 +197,6 @@ private:
|
||||
}
|
||||
data_to.resize(buf_to.count());
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(
|
||||
col_from->size(), formatReadableSizeWithBinarySuffix(col_from->template getValue<T>()));
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
|
@ -199,6 +199,8 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
|
||||
@ -217,12 +219,6 @@ public:
|
||||
reinterpret_cast<const char *>(&data[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
}
|
||||
else if (const ColumnConst * col_from = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeUInt64().createConstColumn(
|
||||
col_from->size(),
|
||||
Impl::apply(col_from->getValue<String>().data(), col_from->getValue<String>().size()));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -254,6 +250,8 @@ public:
|
||||
return std::make_shared<DataTypeFixedString>(Impl::length);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
|
||||
@ -273,15 +271,6 @@ public:
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]),
|
||||
&chars_to[i * Impl::length]);
|
||||
}
|
||||
else if (const ColumnConst * col_from = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
String data = col_from->getValue<String>();
|
||||
|
||||
String hash(Impl::length, 0);
|
||||
Impl::apply(data.data(), data.size(), reinterpret_cast<unsigned char *>(&hash[0]));
|
||||
|
||||
block.getByPosition(result).column = DataTypeFixedString(Impl::length).createConstColumn(col_from->size(), hash);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
@ -316,11 +305,6 @@ private:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Impl::apply(vec_from[i]);
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<FromType>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeNumber<ToType>().createConstColumn(
|
||||
col_from->size(), toField(Impl::apply(col_from->template getValue<FromType>())));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
@ -344,6 +328,8 @@ public:
|
||||
return std::make_shared<DataTypeNumber<typename Impl::ReturnType>>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
@ -751,6 +737,9 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto arg_count = arguments.size();
|
||||
@ -783,13 +772,6 @@ private:
|
||||
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
}
|
||||
else if (const auto col_from = checkAndGetColumnConst<ColumnString>(col_untyped))
|
||||
{
|
||||
String from = col_from->getValue<String>();
|
||||
block.getByPosition(result).column = DataTypeUInt64().createConstColumn(
|
||||
col_from->size(),
|
||||
URLHashImpl::apply(from.data(), from.size()));
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
|
||||
@ -823,13 +805,6 @@ private:
|
||||
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
}
|
||||
else if (const auto col_from = checkAndGetColumnConst<ColumnString>(col_untyped))
|
||||
{
|
||||
String from = col_from->getValue<String>();
|
||||
block.getByPosition(result).column = DataTypeUInt64().createConstColumn(
|
||||
col_from->size(),
|
||||
URLHierarchyHashImpl::apply(level, from.data(), from.size()));
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionsArithmetic.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -210,15 +211,8 @@ private:
|
||||
|
||||
void convertToUInt8(const IColumn * column, UInt8Container & res)
|
||||
{
|
||||
if (!convertTypeToUInt8< Int8 >(column, res) &&
|
||||
!convertTypeToUInt8< Int16>(column, res) &&
|
||||
!convertTypeToUInt8< Int32>(column, res) &&
|
||||
!convertTypeToUInt8< Int64>(column, res) &&
|
||||
!convertTypeToUInt8< UInt16>(column, res) &&
|
||||
!convertTypeToUInt8< UInt32>(column, res) &&
|
||||
!convertTypeToUInt8< UInt64>(column, res) &&
|
||||
!convertTypeToUInt8<Float32>(column, res) &&
|
||||
!convertTypeToUInt8<Float64>(column, res))
|
||||
if (!dispatchForFirstType<UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>(
|
||||
[&] (auto arg) { return convertTypeToUInt8<typename std::decay<decltype(*arg)>::type>(column, res); }))
|
||||
throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
@ -239,15 +233,8 @@ private:
|
||||
|
||||
void executeUInt8Other(const UInt8Container & uint8_vec, IColumn * column, UInt8Container & res)
|
||||
{
|
||||
if (!executeUInt8Type< Int8 >(uint8_vec, column, res) &&
|
||||
!executeUInt8Type< Int16>(uint8_vec, column, res) &&
|
||||
!executeUInt8Type< Int32>(uint8_vec, column, res) &&
|
||||
!executeUInt8Type< Int64>(uint8_vec, column, res) &&
|
||||
!executeUInt8Type< UInt16>(uint8_vec, column, res) &&
|
||||
!executeUInt8Type< UInt32>(uint8_vec, column, res) &&
|
||||
!executeUInt8Type< UInt64>(uint8_vec, column, res) &&
|
||||
!executeUInt8Type<Float32>(uint8_vec, column, res) &&
|
||||
!executeUInt8Type<Float64>(uint8_vec, column, res))
|
||||
if (!dispatchForFirstType<UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>(
|
||||
[&] (auto arg) { return executeUInt8Type<typename std::decay<decltype(*arg)>::type>(uint8_vec, column, res); }))
|
||||
throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
@ -392,16 +379,6 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
UInt8 res = 0;
|
||||
UnaryOperationImpl<T, Impl<T>>::constant(col->template getValue<T>(), res);
|
||||
|
||||
auto col_res = DataTypeUInt8().createConstColumn(col->size(), toField(res));
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
@ -425,18 +402,11 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeType<UInt8>(block, arguments, result)
|
||||
|| executeType<UInt16>(block, arguments, result)
|
||||
|| executeType<UInt32>(block, arguments, result)
|
||||
|| executeType<UInt64>(block, arguments, result)
|
||||
|| executeType<Int8>(block, arguments, result)
|
||||
|| executeType<Int16>(block, arguments, result)
|
||||
|| executeType<Int32>(block, arguments, result)
|
||||
|| executeType<Int64>(block, arguments, result)
|
||||
|| executeType<Float32>(block, arguments, result)
|
||||
|| executeType<Float64>(block, arguments, result)))
|
||||
if (!dispatchForFirstNumericType([&] (auto arg) { return executeType<typename std::decay<decltype(*arg)>::type>(block, arguments, result); }))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
@ -135,21 +135,12 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto col = checkAndGetColumnConst<ColumnVector<FieldType>>(arg))
|
||||
{
|
||||
const FieldType src[Impl::rows_per_iteration] { col->template getValue<FieldType>() };
|
||||
Float64 dst[Impl::rows_per_iteration];
|
||||
|
||||
Impl::execute(src, dst);
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(col->size(), dst[0]);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto arg = block.getByPosition(arguments[0]).column.get();
|
||||
|
@ -175,6 +175,8 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
/// Execute the function on the block.
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
|
||||
};
|
||||
@ -1037,6 +1039,9 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// The level at which the line has zero length.
|
||||
@ -1053,31 +1058,6 @@ public:
|
||||
|
||||
const auto & src = *block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (src.isConst())
|
||||
{
|
||||
auto res_column_holder = DataTypeString().createConstColumn(block.rows(), String());
|
||||
ColumnConst & res_column = static_cast<ColumnConst &>(*res_column_holder);
|
||||
block.getByPosition(result).column = res_column_holder;
|
||||
|
||||
if (executeConstNumber<UInt8>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<UInt16>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<UInt32>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<UInt64>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int8>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int16>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int32>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int64>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<Float32>(src, res_column, min, max, max_width)
|
||||
|| executeConstNumber<Float64>(src, res_column, min, max, max_width))
|
||||
{
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto res_column = std::make_shared<ColumnString>();
|
||||
block.getByPosition(result).column = res_column;
|
||||
|
||||
@ -1097,7 +1077,6 @@ public:
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
@ -1156,21 +1135,6 @@ private:
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static bool executeConstNumber(const IColumn & src, ColumnConst & dst, Int64 min, Int64 max, Float64 max_width)
|
||||
{
|
||||
if (auto col = checkAndGetColumnConst<ColumnVector<T>>(&src))
|
||||
{
|
||||
String res;
|
||||
fill(col->template getValue<T>(), res, min, max, max_width);
|
||||
dst.getDataColumn().cut(0, 0);
|
||||
dst.getDataColumn().insert(res);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -1211,6 +1175,8 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto in = block.getByPosition(arguments.front()).column.get();
|
||||
@ -1244,12 +1210,6 @@ public:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto in = checkAndGetColumnConst<ColumnVector<T>>(in_untyped))
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeUInt8().createConstColumn(in->size(), toField(Impl::execute(in->template getValue<T>())));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
@ -1689,8 +1649,6 @@ void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & argu
|
||||
auto & src = block.getByPosition(arguments[0]);
|
||||
size_t size = block.rows();
|
||||
|
||||
if (!src.column->isConst())
|
||||
{
|
||||
auto res_col = std::make_shared<ColumnUInt64>(size);
|
||||
auto & res_data = static_cast<ColumnUInt64 &>(*res_col).getData();
|
||||
block.getByPosition(result).column = res_col;
|
||||
@ -1708,18 +1666,6 @@ void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & argu
|
||||
res_data[i] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(tmp.data()), tmp.size());
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
String tmp;
|
||||
{
|
||||
WriteBufferFromString out(tmp);
|
||||
src.type->serializeTextEscaped(*src.column->cut(0, 1)->convertToFullColumnIfConst(), 0, out);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = DataTypeUInt64().createConstColumn(size,
|
||||
UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(tmp.data()), tmp.size()));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void FunctionHasColumnInTable::getReturnTypeAndPrerequisitesImpl(
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNull.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -83,15 +83,6 @@ public:
|
||||
}
|
||||
data_to.resize(pos);
|
||||
}
|
||||
else if (auto col_from = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
T value = col_from->template getValue<T>();
|
||||
std::string res(reinterpret_cast<const char *>(&value), sizeof(T));
|
||||
while (!res.empty() && res[res.length() - 1] == '\0')
|
||||
res.erase(res.end() - 1);
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col_from->size(), res);
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
@ -100,6 +91,8 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeType<UInt8>(block, arguments, result)
|
||||
@ -118,6 +111,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template<typename ToDataType, typename Name>
|
||||
class FunctionReinterpretStringAs : public IFunction
|
||||
{
|
||||
@ -144,6 +138,8 @@ public:
|
||||
return std::make_shared<ToDataType>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (ColumnString * col_from = typeid_cast<ColumnString *>(block.getByPosition(arguments[0]).column.get()))
|
||||
@ -187,14 +183,6 @@ public:
|
||||
offset += step;
|
||||
}
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
ToFieldType value = 0;
|
||||
String str = col->getValue<String>();
|
||||
memcpy(&value, str.data(), std::min(sizeof(ToFieldType), str.length()));
|
||||
auto col_res = DataTypeNumber<ToFieldType>().createConstColumn(col->size(), toField(value));
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
|
@ -988,12 +988,6 @@ struct Cruncher
|
||||
|
||||
Op::apply(col->getData(), scale, vec_res);
|
||||
}
|
||||
|
||||
static inline void apply(Block & block, const ColumnConst * col, const ColumnNumbers & arguments, size_t result, size_t scale)
|
||||
{
|
||||
T res = Op::apply(col->getValue<T>(), scale);
|
||||
block.getByPosition(result).column = DataTypeNumber<T>().createConstColumn(col->size(), toField(res));
|
||||
}
|
||||
};
|
||||
|
||||
/** Select the appropriate processing algorithm depending on the scale.
|
||||
@ -1053,12 +1047,6 @@ private:
|
||||
Dispatcher<T, ColumnVector<T>, rounding_mode>::apply(block, col, arguments, result);
|
||||
return true;
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
Dispatcher<T, ColumnConst, rounding_mode>::apply(block, col, arguments, result);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -1106,6 +1094,9 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
if (!( executeForType<UInt8>(block, arguments, result)
|
||||
|
@ -727,6 +727,8 @@ public:
|
||||
return std::make_shared<DataTypeNumber<ResultType>>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
@ -759,12 +761,6 @@ public:
|
||||
Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res);
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
ResultType res = 0;
|
||||
Impl::constant(col->getValue<String>(), res);
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(col->size(), toField(res));
|
||||
}
|
||||
else if (const ColumnArray * col = checkAndGetColumn<ColumnArray>(column.get()))
|
||||
{
|
||||
auto col_res = std::make_shared<ColumnVector<ResultType>>();
|
||||
@ -774,12 +770,6 @@ public:
|
||||
vec_res.resize(col->size());
|
||||
Impl::array(col->getOffsets(), vec_res);
|
||||
}
|
||||
else if (const ColumnConst * col = checkAndGetColumnConst<ColumnArray>(column.get()))
|
||||
{
|
||||
ResultType res = 0;
|
||||
Impl::constant_array(col->getValue<Array>(), res);
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(col->size(), toField(res));
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
@ -807,10 +797,12 @@ public:
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeString>(&*arguments[0]) && !checkDataType<DataTypeFixedString>(&*arguments[0])
|
||||
@ -821,6 +813,7 @@ public:
|
||||
return arguments[0]->clone();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
@ -837,14 +830,7 @@ public:
|
||||
block.getByPosition(result).column = col_res;
|
||||
ReverseImpl::vector_fixed(col->getChars(), col->getN(), col_res->getChars());
|
||||
}
|
||||
else if (const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
String res;
|
||||
ReverseImpl::constant(col->getValue<String>(), res);
|
||||
auto col_res = DataTypeString().createConstColumn(col->size(), res);
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else if (checkColumn<ColumnArray>(column.get()) || checkColumnConst<ColumnArray>(column.get()))
|
||||
else if (checkColumn<ColumnArray>(column.get()))
|
||||
{
|
||||
FunctionArrayReverse().execute(block, arguments, result);
|
||||
}
|
||||
@ -1455,6 +1441,7 @@ private:
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeString>(arguments[0].get()))
|
||||
@ -1468,6 +1455,9 @@ private:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto & column = block.getByPosition(arguments[0]).column;
|
||||
@ -1518,13 +1508,6 @@ private:
|
||||
|
||||
dst_data.resize_assume_reserved(dst_offset);
|
||||
}
|
||||
else if (const auto col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
String in_data = col->getValue<String>();
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col->size(),
|
||||
in_data.size() == 0 ? in_data : in_data.back() == trailing_char_str.front() ? in_data : in_data + trailing_char_str);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
|
@ -148,6 +148,7 @@ public:
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return is_injective;
|
||||
@ -162,6 +163,8 @@ public:
|
||||
return arguments[0]->clone();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
@ -177,12 +180,6 @@ public:
|
||||
block.getByPosition(result).column = col_res;
|
||||
Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars());
|
||||
}
|
||||
else if (const ColumnConst * col = checkAndGetColumnConst<ColumnString>(column.get()))
|
||||
{
|
||||
String res;
|
||||
Impl::constant(col->getValue<String>(), res);
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(col->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
|
@ -339,11 +339,11 @@ public:
|
||||
{
|
||||
Generator generator;
|
||||
generator.init(block, arguments);
|
||||
size_t arrayArgumentPosition = arguments[generator.getStringsArgumentPosition()];
|
||||
size_t array_argument_position = arguments[generator.getStringsArgumentPosition()];
|
||||
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(block.getByPosition(arrayArgumentPosition).column.get());
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(block.getByPosition(array_argument_position).column.get());
|
||||
const ColumnConst * col_const_str =
|
||||
checkAndGetColumnConstStringOrFixedString(block.getByPosition(arrayArgumentPosition).column.get());
|
||||
checkAndGetColumnConstStringOrFixedString(block.getByPosition(array_argument_position).column.get());
|
||||
|
||||
auto col_res = std::make_shared<ColumnArray>(std::make_shared<ColumnString>());
|
||||
ColumnPtr col_res_holder = col_res;
|
||||
@ -411,8 +411,8 @@ public:
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(col_const_str->size(), dst);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal columns " + block.getByPosition(arrayArgumentPosition).column->getName()
|
||||
+ ", " + block.getByPosition(arrayArgumentPosition).column->getName()
|
||||
throw Exception("Illegal columns " + block.getByPosition(array_argument_position).column->getName()
|
||||
+ ", " + block.getByPosition(array_argument_position).column->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -960,6 +960,9 @@ public:
|
||||
return 3;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeString>(&*arguments[0]) && !checkDataType<DataTypeFixedString>(&*arguments[0]))
|
||||
@ -1008,13 +1011,6 @@ public:
|
||||
block.getByPosition(result).column = col_res;
|
||||
Impl::vector_fixed(col->getChars(), col->getN(), needle, replacement, col_res->getChars(), col_res->getOffsets());
|
||||
}
|
||||
else if (const ColumnConst * col = typeid_cast<const ColumnConst *>(column_src.get()))
|
||||
{
|
||||
String res;
|
||||
Impl::constant(col->getValue<String>(), needle, replacement, res);
|
||||
auto col_res = DataTypeString().createConstColumn(col->size(), res);
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
|
||||
|
@ -139,6 +139,9 @@ public:
|
||||
return 2;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeString>(&*arguments[0]))
|
||||
@ -170,23 +173,6 @@ public:
|
||||
ColumnString::Offsets_t & offsets_res = col_res->getOffsets();
|
||||
Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res);
|
||||
}
|
||||
else if (auto col = checkAndGetColumnConstStringOrFixedString(column.get()))
|
||||
{
|
||||
String data = col->getValue<String>();
|
||||
ColumnString::Chars_t vdata(reinterpret_cast<const ColumnString::Chars_t::value_type *>(data.c_str()),
|
||||
reinterpret_cast<const ColumnString::Chars_t::value_type *>(data.c_str() + data.size() + 1));
|
||||
ColumnString::Offsets_t offsets(1, vdata.size());
|
||||
ColumnString::Chars_t res_vdata;
|
||||
ColumnString::Offsets_t res_offsets;
|
||||
Impl::vector(vdata, offsets, col_needle->getValue<String>(), res_vdata, res_offsets);
|
||||
|
||||
String res;
|
||||
|
||||
if (!res_offsets.empty())
|
||||
res.assign(&res_vdata[0], &res_vdata[res_vdata.size() - 1]);
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createConstColumn(col->size(), res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
|
@ -109,19 +109,6 @@ void DecodeURLComponentImpl::vector(const ColumnString::Chars_t & data, const Co
|
||||
}
|
||||
|
||||
|
||||
void DecodeURLComponentImpl::constant(const std::string & str,
|
||||
std::string & res_data)
|
||||
{
|
||||
ColumnString src;
|
||||
ColumnString dst;
|
||||
src.insert(str);
|
||||
|
||||
vector(src.getChars(), src.getOffsets(), dst.getChars(), dst.getOffsets());
|
||||
|
||||
res_data = dst[0].get<String>();
|
||||
}
|
||||
|
||||
|
||||
void DecodeURLComponentImpl::vector_fixed(const ColumnString::Chars_t & data, size_t n,
|
||||
ColumnString::Chars_t & res_data)
|
||||
{
|
||||
|
@ -2,7 +2,11 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <DataTypes/DataTypeNull.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -281,8 +285,47 @@ void IFunction::getLambdaArgumentTypes(DataTypes & arguments) const
|
||||
}
|
||||
|
||||
|
||||
static bool allArgumentsAreConstants(const Block & block, const ColumnNumbers & args)
|
||||
{
|
||||
for (auto arg : args)
|
||||
if (!typeid_cast<const ColumnConst *>(block.getByPosition(arg).column.get()))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
void IFunction::execute(Block & block, const ColumnNumbers & args, size_t result)
|
||||
{
|
||||
/// Default implementation if all arguments are constants.
|
||||
if (!args.empty() && allArgumentsAreConstants(block, args) && useDefaultImplementationForConstants())
|
||||
{
|
||||
ColumnNumbers arguments_to_remain_constants = getArgumentsThatAreAlwaysConstant();
|
||||
|
||||
Block temporary_block;
|
||||
|
||||
for (auto arg : args)
|
||||
{
|
||||
const ColumnWithTypeAndName & column = block.getByPosition(arg);
|
||||
|
||||
if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg))
|
||||
temporary_block.insert(column);
|
||||
else
|
||||
temporary_block.insert({ static_cast<const ColumnConst *>(column.column.get())->getDataColumnPtr(), column.type, column.name });
|
||||
}
|
||||
|
||||
temporary_block.insert(block.getByPosition(result));
|
||||
|
||||
size_t arguments_size = args.size();
|
||||
ColumnNumbers temporary_argument_numbers(arguments_size);
|
||||
for (size_t i = 0; i < arguments_size; ++i)
|
||||
temporary_argument_numbers[i] = i;
|
||||
|
||||
execute(temporary_block, temporary_argument_numbers, arguments_size);
|
||||
|
||||
block.getByPosition(result).column = std::make_shared<ColumnConst>(temporary_block.getByPosition(arguments_size).column, block.rows());
|
||||
return;
|
||||
}
|
||||
|
||||
auto strategy = chooseStrategy(block, args);
|
||||
Block processed_block = preProcessBlock(strategy, block, args, result);
|
||||
|
||||
|
@ -157,6 +157,18 @@ public:
|
||||
/// that correspond to nullable columns and null columns.
|
||||
virtual bool hasSpecialSupportForNulls() const { return false; }
|
||||
|
||||
/** If the function have non-zero number of arguments,
|
||||
* and if all arguments are constant, that we could automatically provide default implementation:
|
||||
* arguments are converted to ordinary columns with single value, then function is executed as usual,
|
||||
* and then the result is converted to constant column.
|
||||
*/
|
||||
virtual bool useDefaultImplementationForConstants() const { return false; }
|
||||
|
||||
/** Some arguments could remain constant during this implementation.
|
||||
*/
|
||||
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
|
||||
|
||||
|
||||
/** Lets you know if the function is monotonic in a range of values.
|
||||
* This is used to work with the index in a sorted chunk of data.
|
||||
* And allows to use the index not only when it is written, for example `date >= const`, but also, for example, `toMonth(date) >= 11`.
|
||||
|
Loading…
Reference in New Issue
Block a user