Functions simplification [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-07-23 11:40:43 +03:00
parent efaa44bea8
commit ac5bc79057
27 changed files with 359 additions and 881 deletions

View File

@ -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};
}

View File

@ -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)};

View File

@ -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);
}
};

View File

@ -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));
}
}

View File

@ -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};

View File

@ -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)))

View File

@ -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);

View File

@ -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;

View File

@ -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

View File

@ -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(),

View File

@ -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(),

View File

@ -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;

View File

@ -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() +

View File

@ -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);

View File

@ -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();

View File

@ -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(

View File

@ -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

View File

@ -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()

View File

@ -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)

View File

@ -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(),

View File

@ -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(),

View File

@ -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);
}

View File

@ -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(),

View File

@ -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(),

View File

@ -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)
{

View File

@ -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);

View File

@ -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`.