mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Part 6.
This commit is contained in:
parent
50dcba6df4
commit
740fad66f3
@ -52,17 +52,17 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column_src = columns[arguments[0]].column;
|
||||
const ColumnPtr column_needle = columns[arguments[1]].column;
|
||||
const ColumnPtr column_replacement = columns[arguments[2]].column;
|
||||
const ColumnPtr column_src = arguments[0].column;
|
||||
const ColumnPtr column_needle = arguments[1].column;
|
||||
const ColumnPtr column_replacement = arguments[2].column;
|
||||
|
||||
if (!isColumnConst(*column_needle) || !isColumnConst(*column_replacement))
|
||||
throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
const IColumn * c1 = columns[arguments[1]].column.get();
|
||||
const IColumn * c2 = columns[arguments[2]].column.get();
|
||||
const IColumn * c1 = arguments[1].column.get();
|
||||
const IColumn * c2 = arguments[2].column.get();
|
||||
const ColumnConst * c1_const = typeid_cast<const ColumnConst *>(c1);
|
||||
const ColumnConst * c2_const = typeid_cast<const ColumnConst *>(c2);
|
||||
String needle = c1_const->getValue<String>();
|
||||
@ -75,17 +75,17 @@ public:
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
Impl::vector(col->getChars(), col->getOffsets(), needle, replacement, col_res->getChars(), col_res->getOffsets());
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_src.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
Impl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), needle, replacement, col_res->getChars(), col_res->getOffsets());
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
"Illegal column " + arguments[0].column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
@ -52,20 +52,20 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column = columns[arguments[0]].column;
|
||||
const ColumnPtr column = arguments[0].column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnFixedString::create(col_fixed->getN());
|
||||
Impl::vectorFixed(col_fixed->getChars(), col_fixed->getN(), col_res->getChars());
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
|
@ -69,9 +69,9 @@ DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const ColumnsWithTypeAndNam
|
||||
return type;
|
||||
}
|
||||
|
||||
void FunctionModelEvaluate::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
ColumnPtr FunctionModelEvaluate::executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const
|
||||
{
|
||||
const auto * name_col = checkAndGetColumnConst<ColumnString>(columns[arguments[0]].column.get());
|
||||
const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
||||
if (!name_col)
|
||||
throw Exception("First argument of function " + getName() + " must be a constant string",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -85,7 +85,7 @@ void FunctionModelEvaluate::executeImpl(ColumnsWithTypeAndName & columns, const
|
||||
column_ptrs.reserve(arguments.size());
|
||||
for (auto arg : ext::range(1, arguments.size()))
|
||||
{
|
||||
auto & column = columns[arguments[arg]].column;
|
||||
auto & column = arguments[arg].column;
|
||||
column_ptrs.push_back(column.get());
|
||||
if (auto full_column = column->convertToFullColumnIfConst())
|
||||
{
|
||||
@ -130,7 +130,7 @@ void FunctionModelEvaluate::executeImpl(ColumnsWithTypeAndName & columns, const
|
||||
res = ColumnNullable::create(res, null_map);
|
||||
}
|
||||
|
||||
columns[result].column = res;
|
||||
return res;
|
||||
}
|
||||
|
||||
void registerFunctionsExternalModels(FunctionFactory & factory)
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
const ExternalModelsLoader & models_loader;
|
||||
|
@ -543,9 +543,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(columns[arguments[0]].column.get()))
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(arguments[0].column.get()))
|
||||
{
|
||||
auto col_to = ColumnFixedString::create(Impl::length);
|
||||
|
||||
@ -566,10 +566,10 @@ public:
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
else if (
|
||||
const ColumnFixedString * col_from_fix = checkAndGetColumn<ColumnFixedString>(columns[arguments[0]].column.get()))
|
||||
const ColumnFixedString * col_from_fix = checkAndGetColumn<ColumnFixedString>(arguments[0].column.get()))
|
||||
{
|
||||
auto col_to = ColumnFixedString::create(Impl::length);
|
||||
const typename ColumnFixedString::Chars & data = col_from_fix->getChars();
|
||||
@ -582,10 +582,10 @@ public:
|
||||
Impl::apply(
|
||||
reinterpret_cast<const char *>(&data[i * length]), length, reinterpret_cast<uint8_t *>(&chars_to[i * Impl::length]));
|
||||
}
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -604,11 +604,11 @@ private:
|
||||
using ToType = typename Impl::ReturnType;
|
||||
|
||||
template <typename FromType>
|
||||
void executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
ColumnPtr executeType(ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<FromType>, ColumnDecimal<FromType>, ColumnVector<FromType>>;
|
||||
|
||||
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(columns[arguments[0]].column.get()))
|
||||
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(arguments[0].column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create();
|
||||
|
||||
@ -620,10 +620,10 @@ private:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Impl::apply(vec_from[i]);
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -647,25 +647,37 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
const IDataType * from_type = arguments[0].type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (which.isUInt8()) executeType<UInt8>(columns, arguments, result);
|
||||
else if (which.isUInt16()) executeType<UInt16>(columns, arguments, result);
|
||||
else if (which.isUInt32()) executeType<UInt32>(columns, arguments, result);
|
||||
else if (which.isUInt64()) executeType<UInt64>(columns, arguments, result);
|
||||
else if (which.isInt8()) executeType<Int8>(columns, arguments, result);
|
||||
else if (which.isInt16()) executeType<Int16>(columns, arguments, result);
|
||||
else if (which.isInt32()) executeType<Int32>(columns, arguments, result);
|
||||
else if (which.isInt64()) executeType<Int64>(columns, arguments, result);
|
||||
else if (which.isDate()) executeType<UInt16>(columns, arguments, result);
|
||||
else if (which.isDateTime()) executeType<UInt32>(columns, arguments, result);
|
||||
else if (which.isDecimal32()) executeType<Decimal32>(columns, arguments, result);
|
||||
else if (which.isDecimal64()) executeType<Decimal64>(columns, arguments, result);
|
||||
if (which.isUInt8())
|
||||
return executeType<UInt8>(arguments);
|
||||
else if (which.isUInt16())
|
||||
return executeType<UInt16>(arguments);
|
||||
else if (which.isUInt32())
|
||||
return executeType<UInt32>(arguments);
|
||||
else if (which.isUInt64())
|
||||
return executeType<UInt64>(arguments);
|
||||
else if (which.isInt8())
|
||||
return executeType<Int8>(arguments);
|
||||
else if (which.isInt16())
|
||||
return executeType<Int16>(arguments);
|
||||
else if (which.isInt32())
|
||||
return executeType<Int32>(arguments);
|
||||
else if (which.isInt64())
|
||||
return executeType<Int64>(arguments);
|
||||
else if (which.isDate())
|
||||
return executeType<UInt16>(arguments);
|
||||
else if (which.isDateTime())
|
||||
return executeType<UInt32>(arguments);
|
||||
else if (which.isDecimal32())
|
||||
return executeType<Decimal32>(arguments);
|
||||
else if (which.isDecimal64())
|
||||
return executeType<Decimal64>(arguments);
|
||||
else
|
||||
throw Exception("Illegal type " + columns[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
@ -689,9 +701,9 @@ public:
|
||||
#endif
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
selector.selectAndExecute(columns, arguments, result, input_rows_count);
|
||||
return selector.selectAndExecute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
@ -886,7 +898,7 @@ private:
|
||||
}
|
||||
else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column))
|
||||
{
|
||||
String value = col_from_const->getValue<String>().data();
|
||||
String value = col_from_const->getValue<String>();
|
||||
const ToType hash = Impl::apply(value.data(), value.size());
|
||||
const size_t size = vec_to.size();
|
||||
|
||||
@ -1041,7 +1053,7 @@ public:
|
||||
return std::make_shared<DataTypeNumber<ToType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
size_t rows = input_rows_count;
|
||||
auto col_to = ColumnVector<ToType>::create(rows);
|
||||
@ -1057,13 +1069,10 @@ public:
|
||||
/// The function supports arbitrary number of arguments of arbitrary types.
|
||||
|
||||
bool is_first_argument = true;
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & col = columns[arguments[i]];
|
||||
for (const auto & col : arguments)
|
||||
executeForArgument(col.type.get(), col.column.get(), vec_to, is_first_argument);
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
};
|
||||
|
||||
@ -1086,9 +1095,9 @@ public:
|
||||
#endif
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
selector.selectAndExecute(columns, arguments, result, input_rows_count);
|
||||
return selector.selectAndExecute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
@ -1118,7 +1127,7 @@ struct URLHierarchyHashImpl
|
||||
{
|
||||
static size_t findLevelLength(const UInt64 level, const char * begin, const char * end)
|
||||
{
|
||||
auto pos = begin;
|
||||
const auto * pos = begin;
|
||||
|
||||
/// Let's parse everything that goes before the path
|
||||
|
||||
@ -1192,7 +1201,7 @@ public:
|
||||
throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " +
|
||||
toString(arg_count) + ", should be 1 or 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
const auto first_arg = arguments.front().get();
|
||||
const auto * first_arg = arguments.front().get();
|
||||
if (!WhichDataType(first_arg).isString())
|
||||
throw Exception{"Illegal type " + first_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -1209,24 +1218,24 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto arg_count = arguments.size();
|
||||
|
||||
if (arg_count == 1)
|
||||
executeSingleArg(columns, arguments, result);
|
||||
return executeSingleArg(arguments);
|
||||
else if (arg_count == 2)
|
||||
executeTwoArgs(columns, arguments, result);
|
||||
return executeTwoArgs(arguments);
|
||||
else
|
||||
throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
||||
private:
|
||||
void executeSingleArg(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result) const
|
||||
ColumnPtr executeSingleArg(ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
const auto col_untyped = columns[arguments.front()].column.get();
|
||||
const auto * col_untyped = arguments.front().column.get();
|
||||
|
||||
if (const auto col_from = checkAndGetColumn<ColumnString>(col_untyped))
|
||||
if (const auto * col_from = checkAndGetColumn<ColumnString>(col_untyped))
|
||||
{
|
||||
const auto size = col_from->size();
|
||||
auto col_to = ColumnUInt64::create(size);
|
||||
@ -1245,23 +1254,23 @@ private:
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + columns[arguments[0]].column->getName() +
|
||||
throw Exception{"Illegal column " + arguments[0].column->getName() +
|
||||
" of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
void executeTwoArgs(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result) const
|
||||
ColumnPtr executeTwoArgs(ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
const auto level_col = columns[arguments.back()].column.get();
|
||||
const auto * level_col = arguments.back().column.get();
|
||||
if (!isColumnConst(*level_col))
|
||||
throw Exception{"Second argument of function " + getName() + " must be an integral constant", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
const auto level = level_col->get64(0);
|
||||
|
||||
const auto col_untyped = columns[arguments.front()].column.get();
|
||||
if (const auto col_from = checkAndGetColumn<ColumnString>(col_untyped))
|
||||
const auto * col_untyped = arguments.front().column.get();
|
||||
if (const auto * col_from = checkAndGetColumn<ColumnString>(col_untyped))
|
||||
{
|
||||
const auto size = col_from->size();
|
||||
auto col_to = ColumnUInt64::create(size);
|
||||
@ -1281,10 +1290,10 @@ private:
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + columns[arguments[0]].column->getName() +
|
||||
throw Exception{"Illegal column " + arguments[0].column->getName() +
|
||||
" of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
};
|
||||
|
@ -10,13 +10,13 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
std::vector<FunctionJSONHelpers::Move> FunctionJSONHelpers::prepareMoves(const char * function_name, ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t first_index_argument, size_t num_index_arguments)
|
||||
std::vector<FunctionJSONHelpers::Move> FunctionJSONHelpers::prepareMoves(const char * function_name, ColumnsWithTypeAndName & columns, size_t first_index_argument, size_t num_index_arguments)
|
||||
{
|
||||
std::vector<Move> moves;
|
||||
moves.reserve(num_index_arguments);
|
||||
for (const auto i : ext::range(first_index_argument, first_index_argument + num_index_arguments))
|
||||
{
|
||||
const auto & column = columns[arguments[i]];
|
||||
const auto & column = columns[i];
|
||||
if (!isString(column.type) && !isInteger(column.type))
|
||||
throw Exception{"The argument " + std::to_string(i + 1) + " of function " + String(function_name)
|
||||
+ " should be a string specifying key or an integer specifying index, illegal type: " + column.type->getName(),
|
||||
|
@ -55,22 +55,22 @@ public:
|
||||
class Executor
|
||||
{
|
||||
public:
|
||||
static void run(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count)
|
||||
static ColumnPtr run(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count)
|
||||
{
|
||||
MutableColumnPtr to{columns[result_pos].type->createColumn()};
|
||||
MutableColumnPtr to{result_type->createColumn()};
|
||||
to->reserve(input_rows_count);
|
||||
|
||||
if (arguments.size() < 1)
|
||||
if (arguments.empty())
|
||||
throw Exception{"Function " + String(Name::name) + " requires at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
const auto & first_column = columns[arguments[0]];
|
||||
const auto & first_column = arguments[0];
|
||||
if (!isString(first_column.type))
|
||||
throw Exception{"The first argument of function " + String(Name::name) + " should be a string containing JSON, illegal type: " + first_column.type->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
const ColumnPtr & arg_json = first_column.column;
|
||||
auto col_json_const = typeid_cast<const ColumnConst *>(arg_json.get());
|
||||
auto col_json_string
|
||||
const auto * col_json_const = typeid_cast<const ColumnConst *>(arg_json.get());
|
||||
const auto * col_json_string
|
||||
= typeid_cast<const ColumnString *>(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get());
|
||||
|
||||
if (!col_json_string)
|
||||
@ -79,8 +79,8 @@ public:
|
||||
const ColumnString::Chars & chars = col_json_string->getChars();
|
||||
const ColumnString::Offsets & offsets = col_json_string->getOffsets();
|
||||
|
||||
size_t num_index_arguments = Impl<JSONParser>::getNumberOfIndexArguments(columns, arguments);
|
||||
std::vector<Move> moves = prepareMoves(Name::name, columns, arguments, 1, num_index_arguments);
|
||||
size_t num_index_arguments = Impl<JSONParser>::getNumberOfIndexArguments(arguments);
|
||||
std::vector<Move> moves = prepareMoves(Name::name, arguments, 1, num_index_arguments);
|
||||
|
||||
/// Preallocate memory in parser if necessary.
|
||||
JSONParser parser;
|
||||
@ -94,8 +94,8 @@ public:
|
||||
Impl<JSONParser> impl;
|
||||
|
||||
/// prepare() does Impl-specific preparation before handling each row.
|
||||
if constexpr (has_member_function_prepare<void (Impl<JSONParser>::*)(const char *, const ColumnsWithTypeAndName &, const ColumnNumbers &, size_t)>::value)
|
||||
impl.prepare(Name::name, columns, arguments, result_pos);
|
||||
if constexpr (has_member_function_prepare<void (Impl<JSONParser>::*)(const char *, const ColumnsWithTypeAndName &, const DataTypePtr &)>::value)
|
||||
impl.prepare(Name::name, arguments, result_type);
|
||||
|
||||
using Element = typename JSONParser::Element;
|
||||
|
||||
@ -121,7 +121,7 @@ public:
|
||||
/// Perform moves.
|
||||
Element element;
|
||||
std::string_view last_key;
|
||||
bool moves_ok = performMoves<JSONParser>(columns, arguments, i, document, moves, element, last_key);
|
||||
bool moves_ok = performMoves<JSONParser>(arguments, i, document, moves, element, last_key);
|
||||
|
||||
if (moves_ok)
|
||||
added_to_column = impl.insertResultToColumn(*to, element, last_key);
|
||||
@ -131,7 +131,7 @@ public:
|
||||
if (!added_to_column)
|
||||
to->insertDefault();
|
||||
}
|
||||
columns[result_pos].column = std::move(to);
|
||||
return to;
|
||||
}
|
||||
};
|
||||
|
||||
@ -166,11 +166,11 @@ private:
|
||||
String key;
|
||||
};
|
||||
|
||||
static std::vector<Move> prepareMoves(const char * function_name, ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t first_index_argument, size_t num_index_arguments);
|
||||
static std::vector<Move> prepareMoves(const char * function_name, ColumnsWithTypeAndName & columns, size_t first_index_argument, size_t num_index_arguments);
|
||||
|
||||
/// Performs moves of types MoveType::Index and MoveType::ConstIndex.
|
||||
template <typename JSONParser>
|
||||
static bool performMoves(const ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t row,
|
||||
static bool performMoves(const ColumnsWithTypeAndName & arguments, size_t row,
|
||||
const typename JSONParser::Element & document, const std::vector<Move> & moves,
|
||||
typename JSONParser::Element & element, std::string_view & last_key)
|
||||
{
|
||||
@ -196,14 +196,14 @@ private:
|
||||
}
|
||||
case MoveType::Index:
|
||||
{
|
||||
Int64 index = (*columns[arguments[j + 1]].column)[row].get<Int64>();
|
||||
Int64 index = (*arguments[j + 1].column)[row].get<Int64>();
|
||||
if (!moveToElementByIndex<JSONParser>(res_element, index, key))
|
||||
return false;
|
||||
break;
|
||||
}
|
||||
case MoveType::Key:
|
||||
{
|
||||
key = std::string_view{(*columns[arguments[j + 1]].column).getDataAt(row)};
|
||||
key = std::string_view{(*arguments[j + 1].column).getDataAt(row)};
|
||||
if (!moveToElementByKey<JSONParser>(res_element, key))
|
||||
return false;
|
||||
break;
|
||||
@ -286,21 +286,18 @@ public:
|
||||
return Impl<DummyJSONParser>::getReturnType(Name::name, arguments);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
/// Choose JSONParser.
|
||||
#if USE_SIMDJSON
|
||||
if (context.getSettingsRef().allow_simdjson)
|
||||
{
|
||||
FunctionJSONHelpers::Executor<Name, Impl, SimdJSONParser>::run(columns, arguments, result_pos, input_rows_count);
|
||||
return;
|
||||
}
|
||||
return FunctionJSONHelpers::Executor<Name, Impl, SimdJSONParser>::run(arguments, result_type, input_rows_count);
|
||||
#endif
|
||||
|
||||
#if USE_RAPIDJSON
|
||||
FunctionJSONHelpers::Executor<Name, Impl, RapidJSONParser>::run(columns, arguments, result_pos, input_rows_count);
|
||||
return FunctionJSONHelpers::Executor<Name, Impl, RapidJSONParser>::run(arguments, result_type, input_rows_count);
|
||||
#else
|
||||
FunctionJSONHelpers::Executor<Name, Impl, DummyJSONParser>::run(columns, arguments, result_pos, input_rows_count);
|
||||
return FunctionJSONHelpers::Executor<Name, Impl, DummyJSONParser>::run(arguments, result_type, input_rows_count);
|
||||
#endif
|
||||
}
|
||||
|
||||
@ -334,7 +331,7 @@ public:
|
||||
|
||||
static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared<DataTypeUInt8>(); }
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &)
|
||||
{
|
||||
@ -362,7 +359,7 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers &) { return 0; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &)
|
||||
{
|
||||
@ -386,7 +383,7 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
@ -416,7 +413,7 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view & last_key)
|
||||
{
|
||||
@ -450,7 +447,7 @@ public:
|
||||
return std::make_shared<DataTypeEnum<Int8>>(values);
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
@ -492,7 +489,7 @@ public:
|
||||
return std::make_shared<DataTypeNumber<NumberType>>();
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
@ -557,7 +554,7 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
@ -582,7 +579,7 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
@ -909,11 +906,11 @@ public:
|
||||
return DataTypeFactory::instance().get(col_type_const->getValue<String>());
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 2; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; }
|
||||
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result_pos)
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type)
|
||||
{
|
||||
extract_tree = JSONExtractTree<JSONParser>::build(function_name, columns[result_pos].type);
|
||||
extract_tree = JSONExtractTree<JSONParser>::build(function_name, result_type);
|
||||
}
|
||||
|
||||
bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
@ -950,11 +947,10 @@ public:
|
||||
return std::make_unique<DataTypeArray>(tuple_type);
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 2; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; }
|
||||
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result_pos)
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type)
|
||||
{
|
||||
const auto & result_type = columns[result_pos].type;
|
||||
const auto tuple_type = typeid_cast<const DataTypeArray *>(result_type.get())->getNestedType();
|
||||
const auto value_type = typeid_cast<const DataTypeTuple *>(tuple_type.get())->getElements()[1];
|
||||
extract_tree = JSONExtractTree<JSONParser>::build(function_name, value_type);
|
||||
@ -1002,7 +998,7 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
@ -1106,7 +1102,7 @@ public:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
@ -1138,7 +1134,7 @@ public:
|
||||
return std::make_unique<DataTypeArray>(tuple_type);
|
||||
}
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 1; }
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; }
|
||||
|
||||
bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
{
|
||||
|
@ -342,7 +342,7 @@ struct OperationApplier<Op, OperationApplierImpl, 0>
|
||||
|
||||
|
||||
template <class Op>
|
||||
static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & result_info, size_t input_rows_count)
|
||||
static ColumnPtr executeForTernaryLogicImpl(ColumnRawPtrs arguments, const DataTypePtr & result_type, size_t input_rows_count)
|
||||
{
|
||||
/// Combine all constant columns into a single constant value.
|
||||
UInt8 const_3v_value = 0;
|
||||
@ -351,11 +351,10 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn
|
||||
/// If the constant value uniquely determines the result, return it.
|
||||
if (has_consts && (arguments.empty() || Op::isSaturatedValueTernary(const_3v_value)))
|
||||
{
|
||||
result_info.column = ColumnConst::create(
|
||||
buildColumnFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()),
|
||||
return ColumnConst::create(
|
||||
buildColumnFromTernaryData(UInt8Container({const_3v_value}), result_type->isNullable()),
|
||||
input_rows_count
|
||||
);
|
||||
return;
|
||||
}
|
||||
|
||||
const auto result_column = has_consts ?
|
||||
@ -363,7 +362,7 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn
|
||||
|
||||
OperationApplier<Op, AssociativeGenericApplierImpl>::apply(arguments, result_column->getData(), has_consts);
|
||||
|
||||
result_info.column = buildColumnFromTernaryData(result_column->getData(), result_info.type->isNullable());
|
||||
return buildColumnFromTernaryData(result_column->getData(), result_type->isNullable());
|
||||
}
|
||||
|
||||
|
||||
@ -418,7 +417,7 @@ struct TypedExecutorInvoker<Op>
|
||||
|
||||
/// Types of all of the arguments are guaranteed to be non-nullable here
|
||||
template <class Op>
|
||||
static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & result_info, size_t input_rows_count)
|
||||
static ColumnPtr basicExecuteImpl(ColumnRawPtrs arguments, size_t input_rows_count)
|
||||
{
|
||||
/// Combine all constant columns into a single constant value.
|
||||
UInt8 const_val = 0;
|
||||
@ -429,8 +428,7 @@ static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & re
|
||||
{
|
||||
if (!arguments.empty())
|
||||
const_val = Op::apply(const_val, 0);
|
||||
result_info.column = DataTypeUInt8().createColumnConst(input_rows_count, toField(const_val));
|
||||
return;
|
||||
return DataTypeUInt8().createColumnConst(input_rows_count, toField(const_val));
|
||||
}
|
||||
|
||||
/// If the constant value is a neutral element, let's forget about it.
|
||||
@ -448,8 +446,7 @@ static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & re
|
||||
else
|
||||
FastApplierImpl<Op>::apply(*arguments[0], *arguments[1], col_res->getData());
|
||||
|
||||
result_info.column = std::move(col_res);
|
||||
return;
|
||||
return col_res;
|
||||
}
|
||||
|
||||
/// Convert all columns to UInt8
|
||||
@ -470,7 +467,7 @@ static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & re
|
||||
|
||||
OperationApplier<Op, AssociativeApplierImpl>::apply(uint8_args, col_res->getData(), has_consts);
|
||||
|
||||
result_info.column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
|
||||
}
|
||||
@ -511,18 +508,17 @@ DataTypePtr FunctionAnyArityLogical<Impl, Name>::getReturnTypeImpl(const DataTyp
|
||||
}
|
||||
|
||||
template <typename Impl, typename Name>
|
||||
void FunctionAnyArityLogical<Impl, Name>::executeImpl(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) const
|
||||
ColumnPtr FunctionAnyArityLogical<Impl, Name>::executeImpl(
|
||||
ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
|
||||
{
|
||||
ColumnRawPtrs args_in;
|
||||
for (const auto arg_index : arguments)
|
||||
args_in.push_back(columns[arg_index].column.get());
|
||||
for (const auto & arg_index : arguments)
|
||||
args_in.push_back(arg_index.column.get());
|
||||
|
||||
auto & result_info = columns[result_index];
|
||||
if (result_info.type->isNullable())
|
||||
executeForTernaryLogicImpl<Impl>(std::move(args_in), result_info, input_rows_count);
|
||||
if (result_type->isNullable())
|
||||
return executeForTernaryLogicImpl<Impl>(std::move(args_in), result_type, input_rows_count);
|
||||
else
|
||||
basicExecuteImpl<Impl>(std::move(args_in), result_info, input_rows_count);
|
||||
return basicExecuteImpl<Impl>(std::move(args_in), input_rows_count);
|
||||
}
|
||||
|
||||
|
||||
@ -554,9 +550,9 @@ DataTypePtr FunctionUnaryLogical<Impl, Name>::getReturnTypeImpl(const DataTypes
|
||||
}
|
||||
|
||||
template <template <typename> class Impl, typename T>
|
||||
bool functionUnaryExecuteType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
ColumnPtr functionUnaryExecuteType(ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
if (auto col = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
|
||||
if (auto col = checkAndGetColumn<ColumnVector<T>>(arguments[0].column.get()))
|
||||
{
|
||||
auto col_res = ColumnUInt8::create();
|
||||
|
||||
@ -564,29 +560,31 @@ bool functionUnaryExecuteType(ColumnsWithTypeAndName & columns, const ColumnNumb
|
||||
vec_res.resize(col->getData().size());
|
||||
UnaryOperationImpl<T, Impl<T>>::vector(col->getData(), vec_res);
|
||||
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
return col_res;
|
||||
}
|
||||
|
||||
return false;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <template <typename> class Impl, typename Name>
|
||||
void FunctionUnaryLogical<Impl, Name>::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
ColumnPtr FunctionUnaryLogical<Impl, Name>::executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const
|
||||
{
|
||||
if (!(functionUnaryExecuteType<Impl, UInt8>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, UInt16>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, UInt32>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, UInt64>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int8>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int16>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int32>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int64>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Float32>(columns, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Float64>(columns, arguments, result)))
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
ColumnPtr res;
|
||||
if ( !((res = functionUnaryExecuteType<Impl, UInt8>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, UInt16>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, UInt32>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, UInt64>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, Int8>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, Int16>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, Int32>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, Int64>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, Float32>(arguments))
|
||||
|| (res = functionUnaryExecuteType<Impl, Float64>(arguments))))
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -154,7 +154,7 @@ public:
|
||||
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) const override;
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override;
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
bool isCompilableImpl(const DataTypes &) const override { return useDefaultImplementationForNulls(); }
|
||||
@ -217,7 +217,7 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override;
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override;
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
bool isCompilableImpl(const DataTypes &) const override { return true; }
|
||||
|
@ -35,19 +35,19 @@ public:
|
||||
|
||||
String getName() const override { return "FunctionExpression"; }
|
||||
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) override
|
||||
{
|
||||
DB::Block expr_columns;
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const auto & argument = columns[arguments[i]];
|
||||
const auto & argument = arguments[i];
|
||||
/// Replace column name with value from argument_names.
|
||||
expr_columns.insert({argument.column, argument.type, signature->argument_names[i]});
|
||||
}
|
||||
|
||||
expression_actions->execute(expr_columns);
|
||||
|
||||
columns[result].column = expr_columns.getByName(signature->return_name).column;
|
||||
return expr_columns.getByName(signature->return_name).column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
@ -79,9 +79,9 @@ public:
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
const DataTypePtr & getResultType() const override { return return_type; }
|
||||
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &, const ColumnNumbers &, size_t) const override
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &) const override
|
||||
{
|
||||
return std::make_unique<ExecutableFunctionExpression>(expression_actions, signature);
|
||||
}
|
||||
@ -119,11 +119,8 @@ public:
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
||||
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) override
|
||||
{
|
||||
ColumnsWithTypeAndName columns_to_catpure;
|
||||
columns_to_catpure.reserve(arguments.size());
|
||||
|
||||
Names names;
|
||||
DataTypes types;
|
||||
|
||||
@ -139,13 +136,10 @@ public:
|
||||
types.push_back(lambda_argument.type);
|
||||
}
|
||||
|
||||
for (const auto & argument : arguments)
|
||||
columns_to_catpure.push_back(columns[argument]);
|
||||
|
||||
auto function = std::make_unique<FunctionExpression>(expression_actions, types, names,
|
||||
capture->return_type, capture->return_name);
|
||||
auto function_adaptor = std::make_shared<FunctionBaseAdaptor>(std::move(function));
|
||||
columns[result].column = ColumnFunction::create(input_rows_count, std::move(function_adaptor), columns_to_catpure);
|
||||
return ColumnFunction::create(input_rows_count, std::move(function_adaptor), arguments);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -177,9 +171,9 @@ public:
|
||||
bool isDeterministicInScopeOfQuery() const override { return true; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override { return capture->captured_types; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
const DataTypePtr & getResultType() const override { return return_type; }
|
||||
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &, const ColumnNumbers &, size_t) const override
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName &) const override
|
||||
{
|
||||
return std::make_unique<ExecutableFunctionCapture>(expression_actions, capture);
|
||||
}
|
||||
|
@ -66,15 +66,15 @@ public:
|
||||
return Impl::getReturnType();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
const ColumnPtr & column_haystack = arguments[0].column;
|
||||
|
||||
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
|
||||
|
||||
const ColumnPtr & num_ptr = columns[arguments[1]].column;
|
||||
const ColumnPtr & num_ptr = arguments[1].column;
|
||||
const ColumnConst * col_const_num = nullptr;
|
||||
UInt32 edit_distance = 0;
|
||||
|
||||
@ -86,17 +86,17 @@ public:
|
||||
edit_distance = col_const_num->getValue<UInt32>();
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + columns[arguments[1]].column->getName()
|
||||
"Illegal column " + arguments[1].column->getName()
|
||||
+ ". The number is not const or does not fit in UInt32",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
|
||||
const ColumnPtr & arr_ptr = columns[arguments[2]].column;
|
||||
const ColumnPtr & arr_ptr = arguments[2].column;
|
||||
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
|
||||
|
||||
if (!col_const_arr)
|
||||
throw Exception(
|
||||
"Illegal column " + columns[arguments[2]].column->getName() + ". The array is not const",
|
||||
"Illegal column " + arguments[2].column->getName() + ". The array is not const",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
Array src_arr = col_const_arr->getValue<Array>();
|
||||
@ -124,12 +124,12 @@ public:
|
||||
Impl::vectorConstant(
|
||||
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res, edit_distance);
|
||||
else
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal column " + arguments[0].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if constexpr (Impl::is_column_array)
|
||||
columns[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
return ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
else
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -73,20 +73,20 @@ public:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
const ColumnPtr & column_haystack = arguments[0].column;
|
||||
|
||||
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
|
||||
|
||||
const ColumnPtr & arr_ptr = columns[arguments[1]].column;
|
||||
const ColumnPtr & arr_ptr = arguments[1].column;
|
||||
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
|
||||
|
||||
if (!col_const_arr)
|
||||
throw Exception(
|
||||
"Illegal column " + columns[arguments[1]].column->getName() + ". The array is not const",
|
||||
"Illegal column " + arguments[1].column->getName() + ". The array is not const",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
Array src_arr = col_const_arr->getValue<Array>();
|
||||
@ -114,7 +114,7 @@ public:
|
||||
if (col_haystack_vector)
|
||||
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
|
||||
else
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal column " + arguments[0].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
size_t refs_size = refs.size();
|
||||
size_t accum = refs_size;
|
||||
@ -122,7 +122,7 @@ public:
|
||||
for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size)
|
||||
offsets_res[i] = accum;
|
||||
|
||||
columns[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
return ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -75,20 +75,20 @@ public:
|
||||
return Impl::getReturnType();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
const ColumnPtr & column_haystack = arguments[0].column;
|
||||
|
||||
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
|
||||
|
||||
const ColumnPtr & arr_ptr = columns[arguments[1]].column;
|
||||
const ColumnPtr & arr_ptr = arguments[1].column;
|
||||
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
|
||||
|
||||
if (!col_const_arr)
|
||||
throw Exception(
|
||||
"Illegal column " + columns[arguments[1]].column->getName() + ". The array is not const",
|
||||
"Illegal column " + arguments[1].column->getName() + ". The array is not const",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
Array src_arr = col_const_arr->getValue<Array>();
|
||||
@ -115,12 +115,12 @@ public:
|
||||
if (col_haystack_vector)
|
||||
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
|
||||
else
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal column " + arguments[0].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if constexpr (Impl::is_column_array)
|
||||
columns[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
return ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
else
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -74,7 +74,7 @@ public:
|
||||
return std::make_shared<DataTypeNumber<ToType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create();
|
||||
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
|
||||
@ -83,7 +83,7 @@ public:
|
||||
vec_to.resize(size);
|
||||
RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), vec_to.size() * sizeof(ToType));
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -458,7 +458,7 @@ class Dispatcher
|
||||
FloatRoundingImpl<T, rounding_mode, scale_mode>,
|
||||
IntegerRoundingImpl<T, rounding_mode, scale_mode, tie_breaking_mode>>;
|
||||
|
||||
static void apply(ColumnsWithTypeAndName & columns, const ColumnVector<T> * col, Int64 scale_arg, size_t result)
|
||||
static ColumnPtr apply(const ColumnVector<T> * col, Int64 scale_arg)
|
||||
{
|
||||
auto col_res = ColumnVector<T>::create();
|
||||
|
||||
@ -484,10 +484,10 @@ class Dispatcher
|
||||
}
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
|
||||
static void apply(ColumnsWithTypeAndName & columns, const ColumnDecimal<T> * col, Int64 scale_arg, size_t result)
|
||||
static ColumnPtr apply(const ColumnDecimal<T> * col, Int64 scale_arg)
|
||||
{
|
||||
const typename ColumnDecimal<T>::Container & vec_src = col->getData();
|
||||
|
||||
@ -497,16 +497,16 @@ class Dispatcher
|
||||
if (!vec_res.empty())
|
||||
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col->getData(), vec_res, scale_arg);
|
||||
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
|
||||
public:
|
||||
static void apply(ColumnsWithTypeAndName & columns, const IColumn * column, Int64 scale_arg, size_t result)
|
||||
static ColumnPtr apply(const IColumn * column, Int64 scale_arg)
|
||||
{
|
||||
if constexpr (IsNumber<T>)
|
||||
apply(columns, checkAndGetColumn<ColumnVector<T>>(column), scale_arg, result);
|
||||
return apply(checkAndGetColumn<ColumnVector<T>>(column), scale_arg);
|
||||
else if constexpr (IsDecimalNumber<T>)
|
||||
apply(columns, checkAndGetColumn<ColumnDecimal<T>>(column), scale_arg, result);
|
||||
return apply(checkAndGetColumn<ColumnDecimal<T>>(column), scale_arg);
|
||||
}
|
||||
};
|
||||
|
||||
@ -520,7 +520,6 @@ public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRounding>(); }
|
||||
|
||||
public:
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
@ -532,7 +531,7 @@ public:
|
||||
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if ((arguments.size() < 1) || (arguments.size() > 2))
|
||||
if ((arguments.empty()) || (arguments.size() > 2))
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1 or 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
@ -545,11 +544,11 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
static Int64 getScaleArg(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
static Int64 getScaleArg(ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const IColumn & scale_column = *columns[arguments[1]].column;
|
||||
const IColumn & scale_column = *arguments[1].column;
|
||||
if (!isColumnConst(scale_column))
|
||||
throw Exception("Scale argument for rounding functions must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -566,11 +565,12 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & column = columns[arguments[0]];
|
||||
Int64 scale_arg = getScaleArg(columns, arguments);
|
||||
const ColumnWithTypeAndName & column = arguments[0];
|
||||
Int64 scale_arg = getScaleArg(arguments);
|
||||
|
||||
ColumnPtr res;
|
||||
auto call = [&](const auto & types) -> bool
|
||||
{
|
||||
using Types = std::decay_t<decltype(types)>;
|
||||
@ -579,7 +579,7 @@ public:
|
||||
if constexpr (IsDataTypeNumber<DataType> || IsDataTypeDecimal<DataType>)
|
||||
{
|
||||
using FieldType = typename DataType::FieldType;
|
||||
Dispatcher<FieldType, rounding_mode, tie_breaking_mode>::apply(columns, column.column.get(), scale_arg, result);
|
||||
res = Dispatcher<FieldType, rounding_mode, tie_breaking_mode>::apply(column.column.get(), scale_arg);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
@ -590,6 +590,8 @@ public:
|
||||
throw Exception("Illegal column " + column.name + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override
|
||||
@ -613,7 +615,6 @@ public:
|
||||
static constexpr auto name = "roundDown";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRoundDown>(); }
|
||||
|
||||
public:
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
@ -646,25 +647,25 @@ public:
|
||||
return getLeastSupertype({type_x, type_arr_nested});
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override
|
||||
{
|
||||
auto in_column = columns[arguments[0]].column;
|
||||
const auto & in_type = columns[arguments[0]].type;
|
||||
auto in_column = arguments[0].column;
|
||||
const auto & in_type = arguments[0].type;
|
||||
|
||||
auto array_column = columns[arguments[1]].column;
|
||||
const auto & array_type = columns[arguments[1]].type;
|
||||
auto array_column = arguments[1].column;
|
||||
const auto & array_type = arguments[1].type;
|
||||
|
||||
const auto & return_type = columns[result].type;
|
||||
const auto & return_type = result_type;
|
||||
auto column_result = return_type->createColumn();
|
||||
auto out = column_result.get();
|
||||
auto * out = column_result.get();
|
||||
|
||||
if (!in_type->equals(*return_type))
|
||||
in_column = castColumn(columns[arguments[0]], return_type);
|
||||
in_column = castColumn(arguments[0], return_type);
|
||||
|
||||
if (!array_type->equals(*return_type))
|
||||
array_column = castColumn(columns[arguments[1]], std::make_shared<DataTypeArray>(return_type));
|
||||
array_column = castColumn(arguments[1], std::make_shared<DataTypeArray>(return_type));
|
||||
|
||||
const auto in = in_column.get();
|
||||
const auto * in = in_column.get();
|
||||
auto boundaries = typeid_cast<const ColumnConst &>(*array_column).getValue<Array>();
|
||||
size_t num_boundaries = boundaries.size();
|
||||
if (!num_boundaries)
|
||||
@ -687,7 +688,7 @@ public:
|
||||
throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
columns[result].column = std::move(column_result);
|
||||
return column_result;
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -75,7 +75,7 @@ public:
|
||||
}
|
||||
|
||||
/// Initialize by the function arguments.
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*arguments*/) {}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
@ -136,12 +136,12 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void init(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
void init(ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(columns[arguments[0]].column.get());
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of first argument of function " + getName() + ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -204,12 +204,12 @@ public:
|
||||
SplitByCharImpl::checkArguments(arguments);
|
||||
}
|
||||
|
||||
void init(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
void init(ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(columns[arguments[0]].column.get());
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + arguments[0].column->getName()
|
||||
+ " of first argument of function " + getName() + ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -284,12 +284,12 @@ public:
|
||||
}
|
||||
|
||||
/// Initialize by the function arguments.
|
||||
void init(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
void init(ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(columns[arguments[1]].column.get());
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + columns[arguments[1]].column->getName()
|
||||
throw Exception("Illegal column " + arguments[1].column->getName()
|
||||
+ " of first argument of function " + getName() + ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -361,15 +361,15 @@ public:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
Generator generator;
|
||||
generator.init(columns, arguments);
|
||||
size_t array_argument_position = arguments[generator.getStringsArgumentPosition()];
|
||||
generator.init(arguments);
|
||||
const auto & array_argument = arguments[generator.getStringsArgumentPosition()];
|
||||
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(columns[array_argument_position].column.get());
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(array_argument.column.get());
|
||||
const ColumnConst * col_const_str =
|
||||
checkAndGetColumnConstStringOrFixedString(columns[array_argument_position].column.get());
|
||||
checkAndGetColumnConstStringOrFixedString(array_argument.column.get());
|
||||
|
||||
auto col_res = ColumnArray::create(ColumnString::create());
|
||||
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
|
||||
@ -419,7 +419,7 @@ public:
|
||||
res_offsets.push_back(current_dst_offset);
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
else if (col_const_str)
|
||||
{
|
||||
@ -433,11 +433,11 @@ public:
|
||||
while (generator.get(token_begin, token_end))
|
||||
dst.push_back(String(token_begin, token_end - token_begin));
|
||||
|
||||
columns[result].column = columns[result].type->createColumnConst(col_const_str->size(), dst);
|
||||
return result_type->createColumnConst(col_const_str->size(), dst);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal columns " + columns[array_argument_position].column->getName()
|
||||
+ ", " + columns[array_argument_position].column->getName()
|
||||
throw Exception("Illegal columns " + array_argument.column->getName()
|
||||
+ ", " + array_argument.column->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -536,19 +536,19 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
String delimiter;
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(columns[arguments[1]].column.get());
|
||||
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
|
||||
if (!col_delim)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
delimiter = col_delim->getValue<String>();
|
||||
}
|
||||
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(columns[arguments[0]].column.get()))
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arguments[0].column.get()))
|
||||
{
|
||||
Array src_arr = col_const_arr->getValue<Array>();
|
||||
String dst_str;
|
||||
@ -559,11 +559,11 @@ public:
|
||||
dst_str += src_arr[i].get<const String &>();
|
||||
}
|
||||
|
||||
columns[result].column = columns[result].type->createColumnConst(col_const_arr->size(), dst_str);
|
||||
return result_type->createColumnConst(col_const_arr->size(), dst_str);
|
||||
}
|
||||
else
|
||||
{
|
||||
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*columns[arguments[0]].column);
|
||||
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*arguments[0].column);
|
||||
const ColumnString & col_string = assert_cast<const ColumnString &>(col_arr.getData());
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
@ -573,7 +573,7 @@ public:
|
||||
delimiter.data(), delimiter.size(),
|
||||
col_res->getChars(), col_res->getOffsets());
|
||||
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -97,16 +97,16 @@ public:
|
||||
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
const ColumnPtr & column_needle = columns[arguments[1]].column;
|
||||
const ColumnPtr & column_haystack = arguments[0].column;
|
||||
const ColumnPtr & column_needle = arguments[1].column;
|
||||
|
||||
ColumnPtr column_start_pos = nullptr;
|
||||
if (arguments.size() >= 3)
|
||||
column_start_pos = columns[arguments[2]].column;
|
||||
column_start_pos = arguments[2].column;
|
||||
|
||||
const ColumnConst * col_haystack_const = typeid_cast<const ColumnConst *>(&*column_haystack);
|
||||
const ColumnConst * col_needle_const = typeid_cast<const ColumnConst *>(&*column_needle);
|
||||
@ -127,12 +127,9 @@ public:
|
||||
vec_res);
|
||||
|
||||
if (is_col_start_pos_const)
|
||||
columns[result].column
|
||||
= columns[result].type->createColumnConst(col_haystack_const->size(), toField(vec_res[0]));
|
||||
return result_type->createColumnConst(col_haystack_const->size(), toField(vec_res[0]));
|
||||
else
|
||||
columns[result].column = std::move(col_res);
|
||||
|
||||
return;
|
||||
return col_res;
|
||||
}
|
||||
}
|
||||
|
||||
@ -175,11 +172,11 @@ public:
|
||||
vec_res);
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal columns " + columns[arguments[0]].column->getName() + " and "
|
||||
+ columns[arguments[1]].column->getName() + " of arguments of function " + getName(),
|
||||
"Illegal columns " + arguments[0].column->getName() + " and "
|
||||
+ arguments[1].column->getName() + " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -51,12 +51,12 @@ public:
|
||||
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
const ColumnPtr & column_needle = columns[arguments[1]].column;
|
||||
const ColumnPtr & column_haystack = arguments[0].column;
|
||||
const ColumnPtr & column_needle = arguments[1].column;
|
||||
|
||||
const ColumnConst * col_haystack_const = typeid_cast<const ColumnConst *>(&*column_haystack);
|
||||
const ColumnConst * col_needle_const = typeid_cast<const ColumnConst *>(&*column_needle);
|
||||
@ -73,9 +73,7 @@ public:
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
}
|
||||
Impl::constantConstant(col_haystack_const->getValue<String>(), needle, res);
|
||||
columns[result].column
|
||||
= columns[result].type->createColumnConst(col_haystack_const->size(), toField(res));
|
||||
return;
|
||||
return result_type->createColumnConst(col_haystack_const->size(), toField(res));
|
||||
}
|
||||
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
@ -122,12 +120,12 @@ public:
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal columns " + columns[arguments[0]].column->getName() + " and "
|
||||
+ columns[arguments[1]].column->getName() + " of arguments of function " + getName(),
|
||||
"Illegal columns " + arguments[0].column->getName() + " and "
|
||||
+ arguments[1].column->getName() + " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_res);
|
||||
return col_res;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -29,7 +29,7 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*arguments*/) {}
|
||||
|
||||
/// Returns the position of the argument that is the column of rows
|
||||
static size_t getStringsArgumentPosition()
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*arguments*/) {}
|
||||
|
||||
/// Returns the position of the argument that is the column of rows
|
||||
static size_t getStringsArgumentPosition()
|
||||
|
@ -34,7 +34,7 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*arguments*/) {}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*arguments*/) {}
|
||||
|
||||
/// Returns the position of the argument that is the column of rows
|
||||
static size_t getStringsArgumentPosition()
|
||||
|
@ -78,10 +78,10 @@ public:
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return false; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_in_untyped = columns[arguments[0]].column;
|
||||
const double inverse_probability = assert_cast<const ColumnConst &>(*columns[arguments[1]].column).getValue<double>();
|
||||
auto col_in_untyped = arguments[0].column;
|
||||
const double inverse_probability = assert_cast<const ColumnConst &>(*arguments[1].column).getValue<double>();
|
||||
|
||||
if (inverse_probability < 0.0 || 1.0 < inverse_probability)
|
||||
{
|
||||
@ -113,7 +113,7 @@ public:
|
||||
ptr_to[offsets_to[i] - 1] = 0;
|
||||
}
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
else if (const ColumnFixedString * col_in_fixed = checkAndGetColumn<ColumnFixedString>(col_in_untyped.get()))
|
||||
{
|
||||
@ -131,12 +131,12 @@ public:
|
||||
auto * ptr_to = chars_to.data();
|
||||
fuzzBits(ptr_in, ptr_to, chars_to.size(), inverse_probability);
|
||||
|
||||
columns[result].column = std::move(col_to);
|
||||
return col_to;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user