mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Rename block to columns.
This commit is contained in:
parent
f81af6c958
commit
959424f28a
@ -116,29 +116,29 @@ template <typename FromDataType, typename ToDataType>
|
||||
struct CustomWeekTransformImpl
|
||||
{
|
||||
template <typename Transform>
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, Transform transform = {})
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, Transform transform = {})
|
||||
{
|
||||
const auto op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
|
||||
UInt8 week_mode = DEFAULT_WEEK_MODE;
|
||||
if (arguments.size() > 1)
|
||||
{
|
||||
if (const auto week_mode_column = checkAndGetColumnConst<ColumnUInt8>(block[arguments[1]].column.get()))
|
||||
if (const auto week_mode_column = checkAndGetColumnConst<ColumnUInt8>(columns[arguments[1]].column.get()))
|
||||
week_mode = week_mode_column->getValue<UInt8>();
|
||||
}
|
||||
|
||||
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
|
||||
const ColumnPtr source_col = block[arguments[0]].column;
|
||||
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(columns, arguments, 2, 0);
|
||||
const ColumnPtr source_col = columns[arguments[0]].column;
|
||||
if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
|
||||
{
|
||||
auto col_to = ToDataType::ColumnType::create();
|
||||
op.vector(sources->getData(), col_to->getData(), week_mode, time_zone);
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function "
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of first argument of function "
|
||||
+ Transform::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -683,25 +683,25 @@ struct Transformer
|
||||
template <typename FromDataType, typename ToDataType, typename Transform>
|
||||
struct DateTimeTransformImpl
|
||||
{
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, const Transform & transform = {})
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, const Transform & transform = {})
|
||||
{
|
||||
using Op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>;
|
||||
|
||||
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
|
||||
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(columns, arguments, 1, 0);
|
||||
|
||||
const ColumnPtr source_col = block[arguments[0]].column;
|
||||
const ColumnPtr source_col = columns[arguments[0]].column;
|
||||
if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
|
||||
{
|
||||
auto mutable_result_col = block[result].type->createColumn();
|
||||
auto mutable_result_col = columns[result].type->createColumn();
|
||||
auto * col_to = assert_cast<typename ToDataType::ColumnType *>(mutable_result_col.get());
|
||||
|
||||
Op::vector(sources->getData(), col_to->getData(), time_zone, transform);
|
||||
|
||||
block[result].column = std::move(mutable_result_col);
|
||||
columns[result].column = std::move(mutable_result_col);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + Transform::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -91,14 +91,14 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnPtr column_string = block[arguments[0]].column;
|
||||
const ColumnPtr column_string = columns[arguments[0]].column;
|
||||
const ColumnString * input = checkAndGetColumn<ColumnString>(column_string.get());
|
||||
|
||||
if (!input)
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
auto dst_column = ColumnString::create();
|
||||
@ -166,7 +166,7 @@ public:
|
||||
|
||||
dst_data.resize(dst_pos - dst);
|
||||
|
||||
block[result].column = std::move(dst_column);
|
||||
columns[result].column = std::move(dst_column);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -613,17 +613,17 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
}
|
||||
|
||||
/// Multiply aggregation state by integer constant: by merging it with itself specified number of times.
|
||||
void executeAggregateMultiply(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void executeAggregateMultiply(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
ColumnNumbers new_arguments = arguments;
|
||||
if (WhichDataType(block[new_arguments[1]].type).isAggregateFunction())
|
||||
if (WhichDataType(columns[new_arguments[1]].type).isAggregateFunction())
|
||||
std::swap(new_arguments[0], new_arguments[1]);
|
||||
|
||||
if (!isColumnConst(*block[new_arguments[1]].column))
|
||||
throw Exception{"Illegal column " + block[new_arguments[1]].column->getName()
|
||||
if (!isColumnConst(*columns[new_arguments[1]].column))
|
||||
throw Exception{"Illegal column " + columns[new_arguments[1]].column->getName()
|
||||
+ " of argument of aggregation state multiply. Should be integer constant", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
const IColumn & agg_state_column = *block[new_arguments[0]].column;
|
||||
const IColumn & agg_state_column = *columns[new_arguments[0]].column;
|
||||
bool agg_state_is_const = isColumnConst(agg_state_column);
|
||||
const ColumnAggregateFunction & column = typeid_cast<const ColumnAggregateFunction &>(
|
||||
agg_state_is_const ? assert_cast<const ColumnConst &>(agg_state_column).getDataColumn() : agg_state_column);
|
||||
@ -647,7 +647,7 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
auto & vec_to = column_to->getData();
|
||||
auto & vec_from = column_from->getData();
|
||||
|
||||
UInt64 m = typeid_cast<const ColumnConst *>(block[new_arguments[1]].column.get())->getValue<UInt64>();
|
||||
UInt64 m = typeid_cast<const ColumnConst *>(columns[new_arguments[1]].column.get())->getValue<UInt64>();
|
||||
|
||||
// Since we merge the function states by ourselves, we have to have an
|
||||
// Arena for this. Pass it to the resulting column so that the arena
|
||||
@ -674,16 +674,16 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
}
|
||||
|
||||
if (agg_state_is_const)
|
||||
block[result].column = ColumnConst::create(std::move(column_to), input_rows_count);
|
||||
columns[result].column = ColumnConst::create(std::move(column_to), input_rows_count);
|
||||
else
|
||||
block[result].column = std::move(column_to);
|
||||
columns[result].column = std::move(column_to);
|
||||
}
|
||||
|
||||
/// Merge two aggregation states together.
|
||||
void executeAggregateAddition(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void executeAggregateAddition(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
const IColumn & lhs_column = *block[arguments[0]].column;
|
||||
const IColumn & rhs_column = *block[arguments[1]].column;
|
||||
const IColumn & lhs_column = *columns[arguments[0]].column;
|
||||
const IColumn & rhs_column = *columns[arguments[1]].column;
|
||||
|
||||
bool lhs_is_const = isColumnConst(lhs_column);
|
||||
bool rhs_is_const = isColumnConst(rhs_column);
|
||||
@ -707,30 +707,30 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
}
|
||||
|
||||
if (lhs_is_const && rhs_is_const)
|
||||
block[result].column = ColumnConst::create(std::move(column_to), input_rows_count);
|
||||
columns[result].column = ColumnConst::create(std::move(column_to), input_rows_count);
|
||||
else
|
||||
block[result].column = std::move(column_to);
|
||||
columns[result].column = std::move(column_to);
|
||||
}
|
||||
|
||||
void executeDateTimeIntervalPlusMinus(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments,
|
||||
void executeDateTimeIntervalPlusMinus(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments,
|
||||
size_t result, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const
|
||||
{
|
||||
ColumnNumbers new_arguments = arguments;
|
||||
|
||||
/// Interval argument must be second.
|
||||
if (WhichDataType(block[arguments[1]].type).isDateOrDateTime())
|
||||
if (WhichDataType(columns[arguments[1]].type).isDateOrDateTime())
|
||||
std::swap(new_arguments[0], new_arguments[1]);
|
||||
|
||||
/// Change interval argument type to its representation
|
||||
ColumnsWithTypeAndName new_block = block;
|
||||
new_block[new_arguments[1]].type = std::make_shared<DataTypeNumber<DataTypeInterval::FieldType>>();
|
||||
ColumnsWithTypeAndName new_columns = columns;
|
||||
new_columns[new_arguments[1]].type = std::make_shared<DataTypeNumber<DataTypeInterval::FieldType>>();
|
||||
|
||||
ColumnsWithTypeAndName new_arguments_with_type_and_name =
|
||||
{new_block[new_arguments[0]], new_block[new_arguments[1]]};
|
||||
{new_columns[new_arguments[0]], new_columns[new_arguments[1]]};
|
||||
auto function = function_builder->build(new_arguments_with_type_and_name);
|
||||
|
||||
function->execute(new_block, new_arguments, result, input_rows_count);
|
||||
block[result].column = new_block[result].column;
|
||||
function->execute(new_columns, new_arguments, result, input_rows_count);
|
||||
columns[result].column = new_columns[result].column;
|
||||
}
|
||||
|
||||
public:
|
||||
@ -851,12 +851,12 @@ public:
|
||||
return type_res;
|
||||
}
|
||||
|
||||
bool executeFixedString(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
bool executeFixedString(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
using OpImpl = FixedStringOperationImpl<Op<UInt8, UInt8>>;
|
||||
|
||||
auto col_left_raw = block[arguments[0]].column.get();
|
||||
auto col_right_raw = block[arguments[1]].column.get();
|
||||
auto col_left_raw = columns[arguments[0]].column.get();
|
||||
auto col_right_raw = columns[arguments[1]].column.get();
|
||||
if (auto col_left_const = checkAndGetColumnConst<ColumnFixedString>(col_left_raw))
|
||||
{
|
||||
if (auto col_right_const = checkAndGetColumnConst<ColumnFixedString>(col_right_raw))
|
||||
@ -872,7 +872,7 @@ public:
|
||||
col_right->getChars().data(),
|
||||
out_chars.data(),
|
||||
out_chars.size());
|
||||
block[result].column = ColumnConst::create(std::move(col_res), col_left_raw->size());
|
||||
columns[result].column = ColumnConst::create(std::move(col_res), col_left_raw->size());
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@ -922,14 +922,14 @@ public:
|
||||
out_chars.size(),
|
||||
col_left->getN());
|
||||
}
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename A, typename B>
|
||||
bool executeNumeric(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result [[maybe_unused]], const A & left, const B & right) const
|
||||
bool executeNumeric(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result [[maybe_unused]], const A & left, const B & right) const
|
||||
{
|
||||
using LeftDataType = std::decay_t<decltype(left)>;
|
||||
using RightDataType = std::decay_t<decltype(right)>;
|
||||
@ -944,8 +944,8 @@ public:
|
||||
using ColVecT1 = std::conditional_t<IsDecimalNumber<T1>, ColumnDecimal<T1>, ColumnVector<T1>>;
|
||||
using ColVecResult = std::conditional_t<IsDecimalNumber<ResultType>, ColumnDecimal<ResultType>, ColumnVector<ResultType>>;
|
||||
|
||||
auto col_left_raw = block[arguments[0]].column.get();
|
||||
auto col_right_raw = block[arguments[1]].column.get();
|
||||
auto col_left_raw = columns[arguments[0]].column.get();
|
||||
auto col_right_raw = columns[arguments[1]].column.get();
|
||||
|
||||
auto col_left_const = checkAndGetColumnConst<ColVecT0>(col_left_raw);
|
||||
auto col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw);
|
||||
@ -981,7 +981,7 @@ public:
|
||||
OpImplCheck::template constantConstant<dec_a, dec_b>(const_a, const_b, scale_a, scale_b) :
|
||||
OpImpl::template constantConstant<dec_a, dec_b>(const_a, const_b, scale_a, scale_b);
|
||||
|
||||
block[result].column = ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
|
||||
columns[result].column = ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
|
||||
col_left_const->size(), toField(res, type.getScale()));
|
||||
return true;
|
||||
}
|
||||
@ -1026,7 +1026,7 @@ public:
|
||||
if (col_left_const && col_right_const)
|
||||
{
|
||||
auto res = OpImpl::constantConstant(col_left_const->template getValue<T0>(), col_right_const->template getValue<T1>());
|
||||
block[result].column = ResultDataType().createColumnConst(col_left_const->size(), toField(res));
|
||||
columns[result].column = ResultDataType().createColumnConst(col_left_const->size(), toField(res));
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -1050,38 +1050,38 @@ public:
|
||||
return false;
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
/// Special case when multiply aggregate function state
|
||||
if (isAggregateMultiply(block[arguments[0]].type, block[arguments[1]].type))
|
||||
if (isAggregateMultiply(columns[arguments[0]].type, columns[arguments[1]].type))
|
||||
{
|
||||
executeAggregateMultiply(block, arguments, result, input_rows_count);
|
||||
executeAggregateMultiply(columns, arguments, result, input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
/// Special case - addition of two aggregate functions states
|
||||
if (isAggregateAddition(block[arguments[0]].type, block[arguments[1]].type))
|
||||
if (isAggregateAddition(columns[arguments[0]].type, columns[arguments[1]].type))
|
||||
{
|
||||
executeAggregateAddition(block, arguments, result, input_rows_count);
|
||||
executeAggregateAddition(columns, arguments, result, input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
/// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval.
|
||||
if (auto function_builder
|
||||
= getFunctionForIntervalArithmetic(block[arguments[0]].type, block[arguments[1]].type, context))
|
||||
= getFunctionForIntervalArithmetic(columns[arguments[0]].type, columns[arguments[1]].type, context))
|
||||
{
|
||||
executeDateTimeIntervalPlusMinus(block, arguments, result, input_rows_count, function_builder);
|
||||
executeDateTimeIntervalPlusMinus(columns, arguments, result, input_rows_count, function_builder);
|
||||
return;
|
||||
}
|
||||
|
||||
const auto & left_argument = block[arguments[0]];
|
||||
const auto & right_argument = block[arguments[1]];
|
||||
const auto & left_argument = columns[arguments[0]];
|
||||
const auto & right_argument = columns[arguments[1]];
|
||||
auto * left_generic = left_argument.type.get();
|
||||
auto * right_generic = right_argument.type.get();
|
||||
bool valid = castBothTypes(left_generic, right_generic, [&](const auto & left, const auto & right)
|
||||
@ -1093,10 +1093,10 @@ public:
|
||||
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
|
||||
return false;
|
||||
else
|
||||
return executeFixedString(block, arguments, result);
|
||||
return executeFixedString(columns, arguments, result);
|
||||
}
|
||||
else
|
||||
return executeNumeric(block, arguments, result, left, right);
|
||||
return executeNumeric(columns, arguments, result, left, right);
|
||||
});
|
||||
|
||||
if (!valid)
|
||||
@ -1190,30 +1190,30 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
if (left.column && isColumnConst(*left.column) && arguments.size() == 1)
|
||||
{
|
||||
ColumnsWithTypeAndName block_with_constant
|
||||
ColumnsWithTypeAndName columns_with_constant
|
||||
= {{left.column->cloneResized(input_rows_count), left.type, left.name},
|
||||
block[arguments[0]],
|
||||
block[result]};
|
||||
columns[arguments[0]],
|
||||
columns[result]};
|
||||
|
||||
Base::executeImpl(block_with_constant, {0, 1}, 2, input_rows_count);
|
||||
block[result] = block_with_constant[2];
|
||||
Base::executeImpl(columns_with_constant, {0, 1}, 2, input_rows_count);
|
||||
columns[result] = columns_with_constant[2];
|
||||
}
|
||||
else if (right.column && isColumnConst(*right.column) && arguments.size() == 1)
|
||||
{
|
||||
ColumnsWithTypeAndName block_with_constant
|
||||
= {block[arguments[0]],
|
||||
ColumnsWithTypeAndName columns_with_constant
|
||||
= {columns[arguments[0]],
|
||||
{right.column->cloneResized(input_rows_count), right.type, right.name},
|
||||
block[result]};
|
||||
columns[result]};
|
||||
|
||||
Base::executeImpl(block_with_constant, {0, 1}, 2, input_rows_count);
|
||||
block[result] = block_with_constant[2];
|
||||
Base::executeImpl(columns_with_constant, {0, 1}, 2, input_rows_count);
|
||||
columns[result] = columns_with_constant[2];
|
||||
}
|
||||
else
|
||||
Base::executeImpl(block, arguments, result, input_rows_count);
|
||||
Base::executeImpl(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override
|
||||
@ -1244,14 +1244,14 @@ public:
|
||||
{
|
||||
auto transform = [&](const Field & point)
|
||||
{
|
||||
ColumnsWithTypeAndName block_with_constant
|
||||
ColumnsWithTypeAndName columns_with_constant
|
||||
= {{left.column->cloneResized(1), left.type, left.name},
|
||||
{right.type->createColumnConst(1, point), right.type, right.name},
|
||||
{nullptr, return_type, ""}};
|
||||
|
||||
Base::executeImpl(block_with_constant, {0, 1}, 2, 1);
|
||||
Base::executeImpl(columns_with_constant, {0, 1}, 2, 1);
|
||||
Field point_transformed;
|
||||
block_with_constant[2].column->get(0, point_transformed);
|
||||
columns_with_constant[2].column->get(0, point_transformed);
|
||||
return point_transformed;
|
||||
};
|
||||
transform(left_point);
|
||||
@ -1280,14 +1280,14 @@ public:
|
||||
{
|
||||
auto transform = [&](const Field & point)
|
||||
{
|
||||
ColumnsWithTypeAndName block_with_constant
|
||||
ColumnsWithTypeAndName columns_with_constant
|
||||
= {{left.type->createColumnConst(1, point), left.type, left.name},
|
||||
{right.column->cloneResized(1), right.type, right.name},
|
||||
{nullptr, return_type, ""}};
|
||||
|
||||
Base::executeImpl(block_with_constant, {0, 1}, 2, 1);
|
||||
Base::executeImpl(columns_with_constant, {0, 1}, 2, 1);
|
||||
Field point_transformed;
|
||||
block_with_constant[2].column->get(0, point_transformed);
|
||||
columns_with_constant[2].column->get(0, point_transformed);
|
||||
return point_transformed;
|
||||
};
|
||||
|
||||
|
@ -54,32 +54,32 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block , const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns , const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto value_col = block[arguments.front()].column.get();
|
||||
const auto value_col = columns[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 (!execute<UInt8>(columns, arguments, result, value_col)
|
||||
&& !execute<UInt16>(columns, arguments, result, value_col)
|
||||
&& !execute<UInt32>(columns, arguments, result, value_col)
|
||||
&& !execute<UInt64>(columns, arguments, result, value_col)
|
||||
&& !execute<Int8>(columns, arguments, result, value_col)
|
||||
&& !execute<Int16>(columns, arguments, result, value_col)
|
||||
&& !execute<Int32>(columns, arguments, result, value_col)
|
||||
&& !execute<Int64>(columns, arguments, result, value_col))
|
||||
throw Exception{"Illegal column " + value_col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
bool execute(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result,
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result,
|
||||
const IColumn * const value_col_untyped) const
|
||||
{
|
||||
if (const auto value_col = checkAndGetColumn<ColumnVector<T>>(value_col_untyped))
|
||||
{
|
||||
const auto size = value_col->size();
|
||||
bool is_const;
|
||||
const auto const_mask = createConstMaskIfConst<T>(block, arguments, is_const);
|
||||
const auto const_mask = createConstMaskIfConst<T>(columns, arguments, is_const);
|
||||
const auto & val = value_col->getData();
|
||||
|
||||
auto out_col = ColumnVector<UInt8>::create(size);
|
||||
@ -92,29 +92,29 @@ private:
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto mask = createMask<T>(size, block, arguments);
|
||||
const auto mask = createMask<T>(size, columns, arguments);
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::apply(val[i], mask[i]);
|
||||
}
|
||||
|
||||
block[result].column = std::move(out_col);
|
||||
columns[result].column = std::move(out_col);
|
||||
return true;
|
||||
}
|
||||
else if (const auto value_col_const = checkAndGetColumnConst<ColumnVector<T>>(value_col_untyped))
|
||||
{
|
||||
const auto size = value_col_const->size();
|
||||
bool is_const;
|
||||
const auto const_mask = createConstMaskIfConst<T>(block, arguments, is_const);
|
||||
const auto const_mask = createConstMaskIfConst<T>(columns, arguments, is_const);
|
||||
const auto val = value_col_const->template getValue<T>();
|
||||
|
||||
if (is_const)
|
||||
{
|
||||
block[result].column = block[result].type->createColumnConst(size, toField(Impl::apply(val, const_mask)));
|
||||
columns[result].column = columns[result].type->createColumnConst(size, toField(Impl::apply(val, const_mask)));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto mask = createMask<T>(size, block, arguments);
|
||||
const auto mask = createMask<T>(size, columns, arguments);
|
||||
auto out_col = ColumnVector<UInt8>::create(size);
|
||||
|
||||
auto & out = out_col->getData();
|
||||
@ -122,7 +122,7 @@ private:
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::apply(val, mask[i]);
|
||||
|
||||
block[result].column = std::move(out_col);
|
||||
columns[result].column = std::move(out_col);
|
||||
}
|
||||
|
||||
return true;
|
||||
@ -132,14 +132,14 @@ private:
|
||||
}
|
||||
|
||||
template <typename ValueType>
|
||||
ValueType createConstMaskIfConst(const ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, bool & out_is_const) const
|
||||
ValueType createConstMaskIfConst(const ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, bool & out_is_const) const
|
||||
{
|
||||
out_is_const = true;
|
||||
ValueType mask = 0;
|
||||
|
||||
for (const auto i : ext::range(1, arguments.size()))
|
||||
{
|
||||
if (auto pos_col_const = checkAndGetColumnConst<ColumnVector<ValueType>>(block[arguments[i]].column.get()))
|
||||
if (auto pos_col_const = checkAndGetColumnConst<ColumnVector<ValueType>>(columns[arguments[i]].column.get()))
|
||||
{
|
||||
const auto pos = pos_col_const->getUInt(0);
|
||||
if (pos < 8 * sizeof(ValueType))
|
||||
@ -156,13 +156,13 @@ private:
|
||||
}
|
||||
|
||||
template <typename ValueType>
|
||||
PaddedPODArray<ValueType> createMask(const size_t size, const ColumnsWithTypeAndName & block, const ColumnNumbers & arguments) const
|
||||
PaddedPODArray<ValueType> createMask(const size_t size, const ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments) const
|
||||
{
|
||||
PaddedPODArray<ValueType> mask(size, ValueType{});
|
||||
|
||||
for (const auto i : ext::range(1, arguments.size()))
|
||||
{
|
||||
const auto pos_col = block[arguments[i]].column.get();
|
||||
const auto pos_col = columns[arguments[i]].column.get();
|
||||
|
||||
if (!addToMaskImpl<UInt8>(mask, pos_col)
|
||||
&& !addToMaskImpl<UInt16>(mask, pos_col)
|
||||
|
@ -96,26 +96,26 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (which.isDate())
|
||||
CustomWeekTransformImpl<DataTypeDate, ToDataType>::execute(
|
||||
block, arguments, result, input_rows_count, Transform{});
|
||||
columns, arguments, result, input_rows_count, Transform{});
|
||||
else if (which.isDateTime())
|
||||
CustomWeekTransformImpl<DataTypeDateTime, ToDataType>::execute(
|
||||
block, arguments, result, input_rows_count, Transform{});
|
||||
columns, arguments, result, input_rows_count, Transform{});
|
||||
else if (which.isDateTime64())
|
||||
{
|
||||
CustomWeekTransformImpl<DataTypeDateTime64, ToDataType>::execute(
|
||||
block, arguments, result, input_rows_count,
|
||||
columns, arguments, result, input_rows_count,
|
||||
TransformDateTime64<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
"Illegal type " + columns[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
|
@ -305,7 +305,7 @@ private:
|
||||
template <typename FromDataType, typename ToDataType, typename Transform>
|
||||
struct DateTimeAddIntervalImpl
|
||||
{
|
||||
static void execute(Transform transform, ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result)
|
||||
static void execute(Transform transform, ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
using FromValueType = typename FromDataType::FieldType;
|
||||
using FromColumnType = typename FromDataType::ColumnType;
|
||||
@ -313,16 +313,16 @@ struct DateTimeAddIntervalImpl
|
||||
|
||||
auto op = Adder<Transform>{std::move(transform)};
|
||||
|
||||
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
|
||||
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(columns, arguments, 2, 0);
|
||||
|
||||
const ColumnPtr source_col = block[arguments[0]].column;
|
||||
const ColumnPtr source_col = columns[arguments[0]].column;
|
||||
|
||||
auto result_col = block[result].type->createColumn();
|
||||
auto result_col = columns[result].type->createColumn();
|
||||
auto col_to = assert_cast<ToColumnType *>(result_col.get());
|
||||
|
||||
if (const auto * sources = checkAndGetColumn<FromColumnType>(source_col.get()))
|
||||
{
|
||||
const IColumn & delta_column = *block[arguments[1]].column;
|
||||
const IColumn & delta_column = *columns[arguments[1]].column;
|
||||
|
||||
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_column))
|
||||
op.vectorConstant(sources->getData(), col_to->getData(), delta_const_column->getInt(0), time_zone);
|
||||
@ -334,16 +334,16 @@ struct DateTimeAddIntervalImpl
|
||||
op.constantVector(
|
||||
sources_const->template getValue<FromValueType>(),
|
||||
col_to->getData(),
|
||||
*block[arguments[1]].column, time_zone);
|
||||
*columns[arguments[1]].column, time_zone);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + Transform::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
block[result].column = std::move(result_col);
|
||||
columns[result].column = std::move(result_col);
|
||||
}
|
||||
};
|
||||
|
||||
@ -463,28 +463,28 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (which.isDate())
|
||||
{
|
||||
DateTimeAddIntervalImpl<DataTypeDate, TransformResultDataType<DataTypeDate>, Transform>::execute(
|
||||
Transform{}, block, arguments, result);
|
||||
Transform{}, columns, arguments, result);
|
||||
}
|
||||
else if (which.isDateTime())
|
||||
{
|
||||
DateTimeAddIntervalImpl<DataTypeDateTime, TransformResultDataType<DataTypeDateTime>, Transform>::execute(
|
||||
Transform{}, block, arguments, result);
|
||||
Transform{}, columns, arguments, result);
|
||||
}
|
||||
else if (const auto * datetime64_type = assert_cast<const DataTypeDateTime64 *>(from_type))
|
||||
{
|
||||
DateTimeAddIntervalImpl<DataTypeDateTime64, TransformResultDataType<DataTypeDateTime64>, Transform>::execute(
|
||||
Transform{datetime64_type->getScale()}, block, arguments, result);
|
||||
Transform{datetime64_type->getScale()}, columns, arguments, result);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of first argument of function " + getName(),
|
||||
throw Exception("Illegal type " + columns[arguments[0]].type->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
@ -95,23 +95,23 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (which.isDate())
|
||||
DateTimeTransformImpl<DataTypeDate, ToDataType, Transform>::execute(block, arguments, result, input_rows_count);
|
||||
DateTimeTransformImpl<DataTypeDate, ToDataType, Transform>::execute(columns, arguments, result, input_rows_count);
|
||||
else if (which.isDateTime())
|
||||
DateTimeTransformImpl<DataTypeDateTime, ToDataType, Transform>::execute(block, arguments, result, input_rows_count);
|
||||
DateTimeTransformImpl<DataTypeDateTime, ToDataType, Transform>::execute(columns, arguments, result, input_rows_count);
|
||||
else if (which.isDateTime64())
|
||||
{
|
||||
const auto scale = static_cast<const DataTypeDateTime64 *>(from_type)->getScale();
|
||||
const TransformDateTime64<Transform> transformer(scale);
|
||||
DateTimeTransformImpl<DataTypeDateTime64, ToDataType, decltype(transformer)>::execute(block, arguments, result, input_rows_count, transformer);
|
||||
DateTimeTransformImpl<DataTypeDateTime64, ToDataType, decltype(transformer)>::execute(columns, arguments, result, input_rows_count, transformer);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
throw Exception("Illegal type " + columns[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
|
@ -34,9 +34,9 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
block[result].column = block[result].type->createColumnConst(
|
||||
columns[result].column = columns[result].type->createColumnConst(
|
||||
input_rows_count, getFQDNOrHostName())->convertToFullColumnIfConst();
|
||||
}
|
||||
};
|
||||
|
@ -82,7 +82,7 @@ inline std::enable_if_t<IsDecimalNumber<T>, Field> toField(const T & x, UInt32 s
|
||||
Columns convertConstTupleToConstantElements(const ColumnConst & column);
|
||||
|
||||
|
||||
/// Returns the copy of a given block in which each column specified in
|
||||
/// Returns the copy of a given columns in which each column specified in
|
||||
/// the "arguments" parameter is replaced with its respective nested
|
||||
/// column if it is nullable.
|
||||
ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns, const ColumnNumbers & args);
|
||||
|
@ -17,15 +17,15 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
template <bool or_null>
|
||||
void ExecutableFunctionJoinGet<or_null>::execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t)
|
||||
void ExecutableFunctionJoinGet<or_null>::execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t)
|
||||
{
|
||||
ColumnsWithTypeAndName keys;
|
||||
for (size_t i = 2; i < arguments.size(); ++i)
|
||||
{
|
||||
auto key = block[arguments[i]];
|
||||
auto key = columns[arguments[i]];
|
||||
keys.emplace_back(std::move(key));
|
||||
}
|
||||
block[result] = join->joinGet(keys, result_block);
|
||||
columns[result] = join->joinGet(keys, result_columns);
|
||||
}
|
||||
|
||||
template <bool or_null>
|
||||
|
@ -15,8 +15,8 @@ template <bool or_null>
|
||||
class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl
|
||||
{
|
||||
public:
|
||||
ExecutableFunctionJoinGet(HashJoinPtr join_, const DB::Block & result_block_)
|
||||
: join(std::move(join_)), result_block(result_block_) {}
|
||||
ExecutableFunctionJoinGet(HashJoinPtr join_, const DB::Block & result_columns_)
|
||||
: join(std::move(join_)), result_columns(result_columns_) {}
|
||||
|
||||
static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet";
|
||||
|
||||
@ -24,13 +24,13 @@ public:
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
private:
|
||||
HashJoinPtr join;
|
||||
DB::Block result_block;
|
||||
DB::Block result_columns;
|
||||
};
|
||||
|
||||
template <bool or_null>
|
||||
@ -56,7 +56,7 @@ public:
|
||||
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & sample_block, const ColumnNumbers & arguments, size_t result) const override;
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & sample_columns, const ColumnNumbers & arguments, size_t result) const override;
|
||||
|
||||
private:
|
||||
TableLockHolder table_lock;
|
||||
|
@ -54,7 +54,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename LeftType, typename RightType>
|
||||
bool executeTyped(ColumnsWithTypeAndName & block, const size_t result, const ColumnConst * left_arg, const IColumn * right_arg) const
|
||||
bool executeTyped(ColumnsWithTypeAndName & columns, const size_t result, const ColumnConst * left_arg, const IColumn * right_arg) const
|
||||
{
|
||||
if (const auto right_arg_typed = checkAndGetColumn<ColumnVector<RightType>>(right_arg))
|
||||
{
|
||||
@ -95,7 +95,7 @@ private:
|
||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
||||
}
|
||||
|
||||
block[result].column = std::move(dst);
|
||||
columns[result].column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -103,7 +103,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename LeftType, typename RightType>
|
||||
bool executeTyped(ColumnsWithTypeAndName & block, const size_t result, const ColumnVector<LeftType> * left_arg, const IColumn * right_arg) const
|
||||
bool executeTyped(ColumnsWithTypeAndName & columns, const size_t result, const ColumnVector<LeftType> * left_arg, const IColumn * right_arg) const
|
||||
{
|
||||
if (const auto right_arg_typed = checkAndGetColumn<ColumnVector<RightType>>(right_arg))
|
||||
{
|
||||
@ -157,7 +157,7 @@ private:
|
||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
||||
}
|
||||
|
||||
block[result].column = std::move(dst);
|
||||
columns[result].column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
if (const auto right_arg_typed = checkAndGetColumnConst<ColumnVector<RightType>>(right_arg))
|
||||
@ -200,17 +200,17 @@ private:
|
||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
||||
}
|
||||
|
||||
block[result].column = std::move(dst);
|
||||
columns[result].column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_left = block[arguments[0]];
|
||||
const ColumnWithTypeAndName & col_right = block[arguments[1]];
|
||||
const ColumnWithTypeAndName & col_left = columns[arguments[0]];
|
||||
const ColumnWithTypeAndName & col_right = columns[arguments[1]];
|
||||
|
||||
auto call = [&](const auto & types) -> bool
|
||||
{
|
||||
@ -224,7 +224,7 @@ private:
|
||||
|
||||
if (const auto left_arg_typed = checkAndGetColumn<ColVecLeft>(left_arg))
|
||||
{
|
||||
if (executeTyped<LeftType, RightType>(block, result, left_arg_typed, right_arg))
|
||||
if (executeTyped<LeftType, RightType>(columns, result, left_arg_typed, right_arg))
|
||||
return true;
|
||||
|
||||
throw Exception{"Illegal column " + right_arg->getName() + " of second argument of function " + getName(),
|
||||
@ -232,7 +232,7 @@ private:
|
||||
}
|
||||
if (const auto left_arg_typed = checkAndGetColumnConst<ColVecLeft>(left_arg))
|
||||
{
|
||||
if (executeTyped<LeftType, RightType>(block, result, left_arg_typed, right_arg))
|
||||
if (executeTyped<LeftType, RightType>(columns, result, left_arg_typed, right_arg))
|
||||
return true;
|
||||
|
||||
throw Exception{"Illegal column " + right_arg->getName() + " of second argument of function " + getName(),
|
||||
|
@ -25,9 +25,9 @@ private:
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
block[result].column = block[result].type->createColumnConst(input_rows_count, Impl::value);
|
||||
columns[result].column = columns[result].type->createColumnConst(input_rows_count, Impl::value);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -113,7 +113,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename T, typename ReturnType>
|
||||
static bool execute(ColumnsWithTypeAndName & block, const ColumnVector<T> * col, const size_t result)
|
||||
static bool execute(ColumnsWithTypeAndName & columns, const ColumnVector<T> * col, const size_t result)
|
||||
{
|
||||
const auto & src_data = col->getData();
|
||||
const size_t size = src_data.size();
|
||||
@ -124,12 +124,12 @@ private:
|
||||
|
||||
executeInIterations(src_data.data(), dst_data.data(), size);
|
||||
|
||||
block[result].column = std::move(dst);
|
||||
columns[result].column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T, typename ReturnType>
|
||||
static bool execute(ColumnsWithTypeAndName & block, const ColumnDecimal<T> * col, const size_t result)
|
||||
static bool execute(ColumnsWithTypeAndName & columns, const ColumnDecimal<T> * col, const size_t result)
|
||||
{
|
||||
const auto & src_data = col->getData();
|
||||
const size_t size = src_data.size();
|
||||
@ -144,15 +144,15 @@ private:
|
||||
|
||||
executeInIterations(dst_data.data(), dst_data.data(), size);
|
||||
|
||||
block[result].column = std::move(dst);
|
||||
columns[result].column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & col = block[arguments[0]];
|
||||
const ColumnWithTypeAndName & col = columns[arguments[0]];
|
||||
|
||||
auto call = [&](const auto & types) -> bool
|
||||
{
|
||||
@ -162,7 +162,7 @@ private:
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
||||
|
||||
const auto col_vec = checkAndGetColumn<ColVecType>(col.column.get());
|
||||
return execute<Type, ReturnType>(block, col_vec, result);
|
||||
return execute<Type, ReturnType>(columns, col_vec, result);
|
||||
};
|
||||
|
||||
if (!callOnBasicType<void, true, true, true, false>(col.type->getTypeId(), call))
|
||||
|
@ -46,25 +46,25 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto in = block[arguments.front()].column.get();
|
||||
const auto in = columns[arguments.front()].column.get();
|
||||
|
||||
if ( !execute<UInt8>(block, in, result)
|
||||
&& !execute<UInt16>(block, in, result)
|
||||
&& !execute<UInt32>(block, in, result)
|
||||
&& !execute<UInt64>(block, in, result)
|
||||
&& !execute<Int8>(block, in, result)
|
||||
&& !execute<Int16>(block, in, result)
|
||||
&& !execute<Int32>(block, in, result)
|
||||
&& !execute<Int64>(block, in, result)
|
||||
&& !execute<Float32>(block, in, result)
|
||||
&& !execute<Float64>(block, in, result))
|
||||
if ( !execute<UInt8>(columns, in, result)
|
||||
&& !execute<UInt16>(columns, in, result)
|
||||
&& !execute<UInt32>(columns, in, result)
|
||||
&& !execute<UInt64>(columns, in, result)
|
||||
&& !execute<Int8>(columns, in, result)
|
||||
&& !execute<Int16>(columns, in, result)
|
||||
&& !execute<Int32>(columns, in, result)
|
||||
&& !execute<Int64>(columns, in, result)
|
||||
&& !execute<Float32>(columns, in, result)
|
||||
&& !execute<Float64>(columns, in, result))
|
||||
throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool execute(ColumnsWithTypeAndName & block, const IColumn * in_untyped, const size_t result) const
|
||||
bool execute(ColumnsWithTypeAndName & columns, const IColumn * in_untyped, const size_t result) const
|
||||
{
|
||||
if (const auto in = checkAndGetColumn<ColumnVector<T>>(in_untyped))
|
||||
{
|
||||
@ -78,7 +78,7 @@ public:
|
||||
for (const auto i : ext::range(0, size))
|
||||
out_data[i] = Impl::execute(in_data[i]);
|
||||
|
||||
block[result].column = std::move(out);
|
||||
columns[result].column = std::move(out);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -63,10 +63,10 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const IColumn * haystack_column = block[arguments[0]].column.get();
|
||||
const IColumn * needle_column = block[arguments[1]].column.get();
|
||||
const IColumn * haystack_column = columns[arguments[0]].column.get();
|
||||
const IColumn * needle_column = columns[arguments[1]].column.get();
|
||||
|
||||
auto col_res = ColumnVector<UInt8>::create();
|
||||
typename ColumnVector<UInt8>::Container & vec_res = col_res->getData();
|
||||
@ -84,7 +84,7 @@ public:
|
||||
else
|
||||
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -159,9 +159,9 @@ public:
|
||||
#endif
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
selector.selectAndExecute(block, arguments, result, input_rows_count);
|
||||
selector.selectAndExecute(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
|
@ -50,9 +50,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column = block[arguments[0]].column;
|
||||
const ColumnPtr column = columns[arguments[0]].column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
@ -61,7 +61,7 @@ public:
|
||||
vec_res.resize(col->size());
|
||||
Impl::vector(col->getChars(), col->getOffsets(), vec_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
@ -70,7 +70,7 @@ public:
|
||||
ResultType res = 0;
|
||||
Impl::vectorFixedToConstant(col_fixed->getChars(), col_fixed->getN(), res);
|
||||
|
||||
block[result].column = block[result].type->createColumnConst(col_fixed->size(), toField(res));
|
||||
columns[result].column = columns[result].type->createColumnConst(col_fixed->size(), toField(res));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -80,7 +80,7 @@ public:
|
||||
vec_res.resize(col_fixed->size());
|
||||
Impl::vectorFixedToVector(col_fixed->getChars(), col_fixed->getN(), vec_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
}
|
||||
else if (const ColumnArray * col_arr = checkAndGetColumn<ColumnArray>(column.get()))
|
||||
@ -91,10 +91,10 @@ public:
|
||||
vec_res.resize(col_arr->size());
|
||||
Impl::array(col_arr->getOffsets(), vec_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
@ -52,17 +52,17 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column_src = block[arguments[0]].column;
|
||||
const ColumnPtr column_needle = block[arguments[1]].column;
|
||||
const ColumnPtr column_replacement = block[arguments[2]].column;
|
||||
const ColumnPtr column_src = columns[arguments[0]].column;
|
||||
const ColumnPtr column_needle = columns[arguments[1]].column;
|
||||
const ColumnPtr column_replacement = columns[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 = block[arguments[1]].column.get();
|
||||
const IColumn * c2 = block[arguments[2]].column.get();
|
||||
const IColumn * c1 = columns[arguments[1]].column.get();
|
||||
const IColumn * c2 = columns[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());
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(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());
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
@ -52,24 +52,24 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column = block[arguments[0]].column;
|
||||
const ColumnPtr column = columns[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());
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(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());
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
@ -154,9 +154,9 @@ public:
|
||||
return result;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
bool valid = castType(block[arguments[0]].type.get(), [&](const auto & type)
|
||||
bool valid = castType(columns[arguments[0]].type.get(), [&](const auto & type)
|
||||
{
|
||||
using DataType = std::decay_t<decltype(type)>;
|
||||
|
||||
@ -164,13 +164,13 @@ public:
|
||||
{
|
||||
if constexpr (allow_fixed_string)
|
||||
{
|
||||
if (auto col = checkAndGetColumn<ColumnFixedString>(block[arguments[0]].column.get()))
|
||||
if (auto col = checkAndGetColumn<ColumnFixedString>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_res = ColumnFixedString::create(col->getN());
|
||||
auto & vec_res = col_res->getChars();
|
||||
vec_res.resize(col->size() * col->getN());
|
||||
FixedStringUnaryOperationImpl<Op<UInt8>>::vector(col->getChars(), vec_res);
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@ -180,13 +180,13 @@ public:
|
||||
using T0 = typename DataType::FieldType;
|
||||
if constexpr (allow_decimal)
|
||||
{
|
||||
if (auto col = checkAndGetColumn<ColumnDecimal<T0>>(block[arguments[0]].column.get()))
|
||||
if (auto col = checkAndGetColumn<ColumnDecimal<T0>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_res = ColumnDecimal<typename Op<T0>::ResultType>::create(0, type.getScale());
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(col->getData().size());
|
||||
UnaryOperationImpl<T0, Op<T0>>::vector(col->getData(), vec_res);
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@ -194,13 +194,13 @@ public:
|
||||
else
|
||||
{
|
||||
using T0 = typename DataType::FieldType;
|
||||
if (auto col = checkAndGetColumn<ColumnVector<T0>>(block[arguments[0]].column.get()))
|
||||
if (auto col = checkAndGetColumn<ColumnVector<T0>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<typename Op<T0>::ResultType>::create();
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(col->getData().size());
|
||||
UnaryOperationImpl<T0, Op<T0>>::vector(col->getData(), vec_res);
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -65,13 +65,13 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
using SourceColumnType = typename SourceDataType::ColumnType;
|
||||
using ResultColumnType = typename ResultDataType::ColumnType;
|
||||
|
||||
const auto & src = block[arguments[0]];
|
||||
auto & res = block[result];
|
||||
const auto & src = columns[arguments[0]];
|
||||
auto & res = columns[result];
|
||||
const auto & col = *src.column;
|
||||
|
||||
const SourceColumnType * source_col_typed = checkAndGetColumn<SourceColumnType>(col);
|
||||
|
@ -122,9 +122,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(from_type);
|
||||
auto nested_type = array_type->getNestedType();
|
||||
|
||||
@ -132,13 +132,13 @@ public:
|
||||
|
||||
WhichDataType which(nested_type);
|
||||
if (which.isUInt8())
|
||||
executeBitmapData<UInt8>(block, argument_types, arguments, result);
|
||||
executeBitmapData<UInt8>(columns, argument_types, arguments, result);
|
||||
else if (which.isUInt16())
|
||||
executeBitmapData<UInt16>(block, argument_types, arguments, result);
|
||||
executeBitmapData<UInt16>(columns, argument_types, arguments, result);
|
||||
else if (which.isUInt32())
|
||||
executeBitmapData<UInt32>(block, argument_types, arguments, result);
|
||||
executeBitmapData<UInt32>(columns, argument_types, arguments, result);
|
||||
else if (which.isUInt64())
|
||||
executeBitmapData<UInt64>(block, argument_types, arguments, result);
|
||||
executeBitmapData<UInt64>(columns, argument_types, arguments, result);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -146,10 +146,10 @@ public:
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
void executeBitmapData(ColumnsWithTypeAndName & block, DataTypes & argument_types, const ColumnNumbers & arguments, size_t result) const
|
||||
void executeBitmapData(ColumnsWithTypeAndName & columns, DataTypes & argument_types, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
// input data
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(block[arguments[0]].column.get());
|
||||
const ColumnArray * array = typeid_cast<const ColumnArray *>(columns[arguments[0]].column.get());
|
||||
ColumnPtr mapped = array->getDataPtr();
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(&*mapped);
|
||||
@ -174,7 +174,7 @@ private:
|
||||
bitmap_data.rbs.add(input_data[pos]);
|
||||
}
|
||||
}
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -207,32 +207,32 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
// input data
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
auto res_ptr = return_type->createColumn();
|
||||
ColumnArray & res = assert_cast<ColumnArray &>(*res_ptr);
|
||||
|
||||
IColumn & res_data = res.getData();
|
||||
ColumnArray::Offsets & res_offsets = res.getOffsets();
|
||||
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
|
||||
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
|
||||
if (which.isUInt8())
|
||||
executeIntType<UInt8>(block, arguments, input_rows_count, res_data, res_offsets);
|
||||
executeIntType<UInt8>(columns, arguments, input_rows_count, res_data, res_offsets);
|
||||
else if (which.isUInt16())
|
||||
executeIntType<UInt16>(block, arguments, input_rows_count, res_data, res_offsets);
|
||||
executeIntType<UInt16>(columns, arguments, input_rows_count, res_data, res_offsets);
|
||||
else if (which.isUInt32())
|
||||
executeIntType<UInt32>(block, arguments, input_rows_count, res_data, res_offsets);
|
||||
executeIntType<UInt32>(columns, arguments, input_rows_count, res_data, res_offsets);
|
||||
else if (which.isUInt64())
|
||||
executeIntType<UInt64>(block, arguments, input_rows_count, res_data, res_offsets);
|
||||
executeIntType<UInt64>(columns, arguments, input_rows_count, res_data, res_offsets);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
block[result].column = std::move(res_ptr);
|
||||
columns[result].column = std::move(res_ptr);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -240,11 +240,11 @@ private:
|
||||
|
||||
template <typename T>
|
||||
void executeIntType(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t input_rows_count, IColumn & res_data_col, ColumnArray::Offsets & res_offsets)
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t input_rows_count, IColumn & res_data_col, ColumnArray::Offsets & res_offsets)
|
||||
const
|
||||
{
|
||||
const ColumnAggregateFunction * column
|
||||
= typeid_cast<const ColumnAggregateFunction *>(block[arguments[0]].column.get());
|
||||
= typeid_cast<const ColumnAggregateFunction *>(columns[arguments[0]].column.get());
|
||||
|
||||
PaddedPODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
|
||||
ColumnArray::Offset res_offset = 0;
|
||||
@ -299,19 +299,19 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
|
||||
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
|
||||
if (which.isUInt8())
|
||||
executeIntType<UInt8>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt8>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt16())
|
||||
executeIntType<UInt16>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt16>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt32())
|
||||
executeIntType<UInt32>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt32>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt64())
|
||||
executeIntType<UInt64>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt64>(columns, arguments, result, input_rows_count);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -322,7 +322,7 @@ private:
|
||||
|
||||
template <typename T>
|
||||
void executeIntType(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
const
|
||||
{
|
||||
const IColumn * columns[3];
|
||||
@ -333,7 +333,7 @@ private:
|
||||
|
||||
for (size_t i = 0; i < 3; ++i)
|
||||
{
|
||||
columns[i] = block[arguments[i]].column.get();
|
||||
columns[i] = columns[arguments[i]].column.get();
|
||||
is_column_const[i] = isColumnConst(*columns[i]);
|
||||
}
|
||||
if (is_column_const[0])
|
||||
@ -367,7 +367,7 @@ private:
|
||||
= *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
|
||||
Impl::apply(bitmap_data_0, range_start, range_end, bitmap_data_2);
|
||||
}
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -435,19 +435,19 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
|
||||
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
|
||||
if (which.isUInt8())
|
||||
executeIntType<UInt8>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt8>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt16())
|
||||
executeIntType<UInt16>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt16>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt32())
|
||||
executeIntType<UInt32>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt32>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt64())
|
||||
executeIntType<UInt64>(block, arguments, result, input_rows_count);
|
||||
executeIntType<UInt64>(columns, arguments, result, input_rows_count);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -458,7 +458,7 @@ private:
|
||||
|
||||
template <typename T>
|
||||
void executeIntType(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
const IColumn * columns[3];
|
||||
bool is_column_const[3];
|
||||
@ -468,7 +468,7 @@ private:
|
||||
|
||||
for (size_t i = 0; i < 3; ++i)
|
||||
{
|
||||
columns[i] = block[arguments[i]].column.get();
|
||||
columns[i] = columns[arguments[i]].column.get();
|
||||
is_column_const[i] = isColumnConst(*columns[i]);
|
||||
}
|
||||
if (is_column_const[0])
|
||||
@ -485,7 +485,7 @@ private:
|
||||
array = typeid_cast<const ColumnArray*>(typeid_cast<const ColumnConst*>(columns[1])->getDataColumnPtr().get());
|
||||
else
|
||||
{
|
||||
array = typeid_cast<const ColumnArray *>(block[arguments[1]].column.get());
|
||||
array = typeid_cast<const ColumnArray *>(columns[arguments[1]].column.get());
|
||||
}
|
||||
const ColumnArray::Offsets & from_offsets = array->getOffsets();
|
||||
const ColumnVector<UInt32>::Container & from_container = typeid_cast<const ColumnVector<UInt32> *>(&array->getData())->getData();
|
||||
@ -493,7 +493,7 @@ private:
|
||||
if (is_column_const[2])
|
||||
array = typeid_cast<const ColumnArray*>(typeid_cast<const ColumnConst*>(columns[2])->getDataColumnPtr().get());
|
||||
else
|
||||
array = typeid_cast<const ColumnArray *>(block[arguments[2]].column.get());
|
||||
array = typeid_cast<const ColumnArray *>(columns[arguments[2]].column.get());
|
||||
|
||||
const ColumnArray::Offsets & to_offsets = array->getOffsets();
|
||||
const ColumnVector<UInt32>::Container & to_container = typeid_cast<const ColumnVector<UInt32> *>(&array->getData())->getData();
|
||||
@ -538,7 +538,7 @@ private:
|
||||
bitmap_data_2.rbs.merge(bitmap_data_0.rbs);
|
||||
bitmap_data_2.rbs.rb_replace(&from_container[from_start], &to_container[to_start], from_end - from_start);
|
||||
}
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -568,27 +568,27 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create(input_rows_count);
|
||||
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
|
||||
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
|
||||
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
|
||||
if (which.isUInt8())
|
||||
executeIntType<UInt8>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt8>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt16())
|
||||
executeIntType<UInt16>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt16>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt32())
|
||||
executeIntType<UInt32>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt32>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt64())
|
||||
executeIntType<UInt64>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt64>(columns, arguments, input_rows_count, vec_to);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -596,10 +596,10 @@ private:
|
||||
|
||||
template <typename T>
|
||||
void executeIntType(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to) const
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to) const
|
||||
{
|
||||
const ColumnAggregateFunction * column
|
||||
= typeid_cast<const ColumnAggregateFunction *>(block[arguments[0]].column.get());
|
||||
= typeid_cast<const ColumnAggregateFunction *>(columns[arguments[0]].column.get());
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
const AggregateFunctionGroupBitmapData<T> & bitmap_data
|
||||
@ -738,33 +738,33 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_to = ColumnVector<UInt8>::create(input_rows_count);
|
||||
typename ColumnVector<UInt8>::Container & vec_to = col_to->getData();
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
|
||||
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
|
||||
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
|
||||
if (which.isUInt8())
|
||||
executeIntType<UInt8>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt8>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt16())
|
||||
executeIntType<UInt16>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt16>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt32())
|
||||
executeIntType<UInt32>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt32>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt64())
|
||||
executeIntType<UInt64>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt64>(columns, arguments, input_rows_count, vec_to);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
void executeIntType(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<UInt8>::Container & vec_to) const
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<UInt8>::Container & vec_to) const
|
||||
{
|
||||
const IColumn * columns[2];
|
||||
bool is_column_const[2];
|
||||
@ -773,7 +773,7 @@ private:
|
||||
|
||||
for (size_t i = 0; i < 2; ++i)
|
||||
{
|
||||
columns[i] = block[arguments[i]].column.get();
|
||||
columns[i] = columns[arguments[i]].column.get();
|
||||
is_column_const[i] = isColumnConst(*columns[i]);
|
||||
}
|
||||
if (is_column_const[0])
|
||||
@ -835,46 +835,46 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create(input_rows_count);
|
||||
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
|
||||
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
|
||||
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
|
||||
if (which.isUInt8())
|
||||
executeIntType<UInt8>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt8>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt16())
|
||||
executeIntType<UInt16>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt16>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt32())
|
||||
executeIntType<UInt32>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt32>(columns, arguments, input_rows_count, vec_to);
|
||||
else if (which.isUInt64())
|
||||
executeIntType<UInt64>(block, arguments, input_rows_count, vec_to);
|
||||
executeIntType<UInt64>(columns, arguments, input_rows_count, vec_to);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
void executeIntType(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to) const
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to) const
|
||||
{
|
||||
const ColumnAggregateFunction * columns[2];
|
||||
bool is_column_const[2];
|
||||
for (size_t i = 0; i < 2; ++i)
|
||||
{
|
||||
if (auto argument_column_const = checkAndGetColumn<ColumnConst>(block[arguments[i]].column.get()))
|
||||
if (auto argument_column_const = checkAndGetColumn<ColumnConst>(columns[arguments[i]].column.get()))
|
||||
{
|
||||
columns[i] = typeid_cast<const ColumnAggregateFunction*>(argument_column_const->getDataColumnPtr().get());
|
||||
is_column_const[i] = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
columns[i] = typeid_cast<const ColumnAggregateFunction*>(block[arguments[i]].column.get());
|
||||
columns[i] = typeid_cast<const ColumnAggregateFunction*>(columns[arguments[i]].column.get());
|
||||
is_column_const[i] = false;
|
||||
}
|
||||
}
|
||||
@ -970,19 +970,19 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
|
||||
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
|
||||
if (which.isUInt8())
|
||||
executeBitmapData<UInt8>(block, arguments, result, input_rows_count);
|
||||
executeBitmapData<UInt8>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt16())
|
||||
executeBitmapData<UInt16>(block, arguments, result, input_rows_count);
|
||||
executeBitmapData<UInt16>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt32())
|
||||
executeBitmapData<UInt32>(block, arguments, result, input_rows_count);
|
||||
executeBitmapData<UInt32>(columns, arguments, result, input_rows_count);
|
||||
else if (which.isUInt64())
|
||||
executeBitmapData<UInt64>(block, arguments, result, input_rows_count);
|
||||
executeBitmapData<UInt64>(columns, arguments, result, input_rows_count);
|
||||
else
|
||||
throw Exception(
|
||||
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -990,20 +990,20 @@ public:
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
void executeBitmapData(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void executeBitmapData(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
const ColumnAggregateFunction * columns[2];
|
||||
bool is_column_const[2];
|
||||
for (size_t i = 0; i < 2; ++i)
|
||||
{
|
||||
if (auto argument_column_const = typeid_cast<const ColumnConst *>(block[arguments[i]].column.get()))
|
||||
if (auto argument_column_const = typeid_cast<const ColumnConst *>(columns[arguments[i]].column.get()))
|
||||
{
|
||||
columns[i] = typeid_cast<const ColumnAggregateFunction *>(argument_column_const->getDataColumnPtr().get());
|
||||
is_column_const[i] = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
columns[i] = typeid_cast<const ColumnAggregateFunction *>(block[arguments[i]].column.get());
|
||||
columns[i] = typeid_cast<const ColumnAggregateFunction *>(columns[arguments[i]].column.get());
|
||||
is_column_const[i] = false;
|
||||
}
|
||||
}
|
||||
@ -1026,7 +1026,7 @@ private:
|
||||
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(data_ptr_1);
|
||||
Impl<T>::apply(bitmap_data_1, bitmap_data_2);
|
||||
}
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -88,9 +88,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto & col_type_name = block[arguments[0]];
|
||||
const auto & col_type_name = columns[arguments[0]];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
@ -123,10 +123,10 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -168,14 +168,14 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto & col_type_name = block[arguments[0]];
|
||||
const auto & col_type_name = columns[arguments[0]];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
const auto & col_ipv6_zeroed_tail_bytes_type = block[arguments[1]];
|
||||
const auto & col_ipv6_zeroed_tail_bytes_type = columns[arguments[1]];
|
||||
const auto & col_ipv6_zeroed_tail_bytes = col_ipv6_zeroed_tail_bytes_type.column;
|
||||
const auto & col_ipv4_zeroed_tail_bytes_type = block[arguments[2]];
|
||||
const auto & col_ipv4_zeroed_tail_bytes_type = columns[arguments[2]];
|
||||
const auto & col_ipv4_zeroed_tail_bytes = col_ipv4_zeroed_tail_bytes_type.column;
|
||||
|
||||
if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
@ -234,10 +234,10 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -277,9 +277,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
|
||||
if (const auto col_in = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
@ -301,10 +301,10 @@ public:
|
||||
src_offset = offsets_src[i];
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -339,9 +339,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
|
||||
if (const ColumnUInt32 * col = typeid_cast<const ColumnUInt32 *>(column.get()))
|
||||
{
|
||||
@ -365,10 +365,10 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -407,9 +407,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
@ -428,10 +428,10 @@ public:
|
||||
prev_offset = offsets_src[i];
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -460,9 +460,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto & col_type_name = block[arguments[0]];
|
||||
const auto & col_type_name = columns[arguments[0]];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnUInt32 *>(column.get()))
|
||||
@ -477,10 +477,10 @@ public:
|
||||
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += IPV6_BINARY_LENGTH, ++i)
|
||||
mapIPv4ToIPv6(vec_in[i], &vec_res[out_offset]);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -578,9 +578,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
|
||||
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(column.get()))
|
||||
{
|
||||
@ -602,10 +602,10 @@ public:
|
||||
offsets_res[i] = current_offset;
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -688,9 +688,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
@ -716,10 +716,10 @@ public:
|
||||
prev_offset = current_offset;
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -755,9 +755,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = block[arguments[0]];
|
||||
const ColumnWithTypeAndName & col_type_name = columns[arguments[0]];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
@ -792,10 +792,10 @@ public:
|
||||
offsets_res[i] = dst_offset;
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -857,9 +857,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = block[arguments[0]];
|
||||
const ColumnWithTypeAndName & col_type_name = columns[arguments[0]];
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = checkAndGetColumn<ColumnString>(column.get()))
|
||||
@ -891,7 +891,7 @@ public:
|
||||
src_offset += string_size;
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else if (const auto col_in_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
@ -920,10 +920,10 @@ public:
|
||||
dst_offset += uuid_bytes_length;
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
@ -1187,10 +1187,10 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IColumn * column = block[arguments[0]].column.get();
|
||||
ColumnPtr & res_column = block[result].column;
|
||||
const IColumn * column = columns[arguments[0]].column.get();
|
||||
ColumnPtr & res_column = columns[result].column;
|
||||
|
||||
if (tryExecuteUInt<UInt8>(column, res_column) ||
|
||||
tryExecuteUInt<UInt16>(column, res_column) ||
|
||||
@ -1205,7 +1205,7 @@ public:
|
||||
tryExecuteDecimal<Decimal128>(column, res_column))
|
||||
return;
|
||||
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -1255,9 +1255,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
@ -1290,11 +1290,11 @@ public:
|
||||
|
||||
out_vec.resize(pos - begin);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -1335,7 +1335,7 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_str = ColumnString::create();
|
||||
ColumnString::Chars & out_vec = col_str->getChars();
|
||||
@ -1355,7 +1355,7 @@ public:
|
||||
for (size_t idx = 0; idx < arguments.size(); ++idx)
|
||||
{
|
||||
//partial const column
|
||||
columns_holder[idx] = block[arguments[idx]].column->convertToFullColumnIfConst();
|
||||
columns_holder[idx] = columns[arguments[idx]].column->convertToFullColumnIfConst();
|
||||
const IColumn * column = columns_holder[idx].get();
|
||||
|
||||
if (!(executeNumber<UInt8>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
@ -1369,12 +1369,12 @@ public:
|
||||
|| executeNumber<Float32>(*column, out_vec, idx, input_rows_count, size_per_row)
|
||||
|| executeNumber<Float64>(*column, out_vec, idx, input_rows_count, size_per_row)))
|
||||
{
|
||||
throw Exception{"Illegal column " + block[arguments[idx]].column->getName()
|
||||
throw Exception{"Illegal column " + columns[arguments[idx]].column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_str);
|
||||
columns[result].column = std::move(col_str);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -1461,10 +1461,10 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IColumn * in_column = block[arguments[0]].column.get();
|
||||
ColumnPtr & out_column = block[result].column;
|
||||
const IColumn * in_column = columns[arguments[0]].column.get();
|
||||
ColumnPtr & out_column = columns[result].column;
|
||||
|
||||
if (tryExecute<UInt8>(in_column, out_column) ||
|
||||
tryExecute<UInt16>(in_column, out_column) ||
|
||||
@ -1476,7 +1476,7 @@ public:
|
||||
tryExecute<Int64>(in_column, out_column))
|
||||
return;
|
||||
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -1599,15 +1599,15 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IColumn * column = block[arguments[0]].column.get();
|
||||
ColumnPtr & res_column = block[result].column;
|
||||
const IColumn * column = columns[arguments[0]].column.get();
|
||||
ColumnPtr & res_column = columns[result].column;
|
||||
|
||||
if (tryExecuteFixedString(column, res_column) || tryExecuteString(column, res_column))
|
||||
return;
|
||||
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -1668,15 +1668,15 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & col_type_name_ip = block[arguments[0]];
|
||||
const auto & col_type_name_ip = columns[arguments[0]];
|
||||
const ColumnPtr & column_ip = col_type_name_ip.column;
|
||||
|
||||
const auto col_ip_in = checkAndGetColumn<ColumnFixedString>(column_ip.get());
|
||||
|
||||
if (!col_ip_in)
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -1687,14 +1687,14 @@ public:
|
||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto & col_type_name_cidr = block[arguments[1]];
|
||||
const auto & col_type_name_cidr = columns[arguments[1]];
|
||||
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
||||
|
||||
const auto col_const_cidr_in = checkAndGetColumnConst<ColumnUInt8>(column_cidr.get());
|
||||
const auto col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
||||
|
||||
if (!col_const_cidr_in && !col_cidr_in)
|
||||
throw Exception("Illegal column " + block[arguments[1]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[1]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -1721,7 +1721,7 @@ public:
|
||||
applyCIDRMask(&vec_in[offset_ipv6], &vec_res_lower_range[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
|
||||
}
|
||||
|
||||
block[result].column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||
columns[result].column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||
}
|
||||
};
|
||||
|
||||
@ -1772,25 +1772,25 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & col_type_name_ip = block[arguments[0]];
|
||||
const auto & col_type_name_ip = columns[arguments[0]];
|
||||
const ColumnPtr & column_ip = col_type_name_ip.column;
|
||||
|
||||
const auto col_ip_in = checkAndGetColumn<ColumnUInt32>(column_ip.get());
|
||||
if (!col_ip_in)
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
const auto & col_type_name_cidr = block[arguments[1]];
|
||||
const auto & col_type_name_cidr = columns[arguments[1]];
|
||||
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
||||
|
||||
const auto col_const_cidr_in = checkAndGetColumnConst<ColumnUInt8>(column_cidr.get());
|
||||
const auto col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
||||
|
||||
if (!col_const_cidr_in && !col_cidr_in)
|
||||
throw Exception("Illegal column " + block[arguments[1]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[1]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -1814,7 +1814,7 @@ public:
|
||||
std::tie(vec_res_lower_range[i], vec_res_upper_range[i]) = applyCIDRMask(vec_in[i], cidr);
|
||||
}
|
||||
|
||||
block[result].column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||
columns[result].column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -571,7 +571,7 @@ private:
|
||||
bool check_decimal_overflow = true;
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumRightType(ColumnsWithTypeAndName & block, size_t result, const ColumnVector<T0> * col_left, const IColumn * col_right_untyped) const
|
||||
bool executeNumRightType(ColumnsWithTypeAndName & columns, size_t result, const ColumnVector<T0> * col_left, const IColumn * col_right_untyped) const
|
||||
{
|
||||
if (const ColumnVector<T1> * col_right = checkAndGetColumn<ColumnVector<T1>>(col_right_untyped))
|
||||
{
|
||||
@ -581,7 +581,7 @@ private:
|
||||
vec_res.resize(col_left->getData().size());
|
||||
NumComparisonImpl<T0, T1, Op<T0, T1>>::vectorVector(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
else if (auto col_right_const = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
|
||||
@ -592,7 +592,7 @@ private:
|
||||
vec_res.resize(col_left->size());
|
||||
NumComparisonImpl<T0, T1, Op<T0, T1>>::vectorConstant(col_left->getData(), col_right_const->template getValue<T1>(), vec_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -600,7 +600,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumConstRightType(ColumnsWithTypeAndName & block, size_t result, const ColumnConst * col_left, const IColumn * col_right_untyped) const
|
||||
bool executeNumConstRightType(ColumnsWithTypeAndName & columns, size_t result, const ColumnConst * col_left, const IColumn * col_right_untyped) const
|
||||
{
|
||||
if (const ColumnVector<T1> * col_right = checkAndGetColumn<ColumnVector<T1>>(col_right_untyped))
|
||||
{
|
||||
@ -610,7 +610,7 @@ private:
|
||||
vec_res.resize(col_left->size());
|
||||
NumComparisonImpl<T0, T1, Op<T0, T1>>::constantVector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
else if (auto col_right_const = checkAndGetColumnConst<ColumnVector<T1>>(col_right_untyped))
|
||||
@ -618,7 +618,7 @@ private:
|
||||
UInt8 res = 0;
|
||||
NumComparisonImpl<T0, T1, Op<T0, T1>>::constantConstant(col_left->template getValue<T0>(), col_right_const->template getValue<T1>(), res);
|
||||
|
||||
block[result].column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
|
||||
columns[result].column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -626,24 +626,24 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeNumLeftType(ColumnsWithTypeAndName & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped) const
|
||||
bool executeNumLeftType(ColumnsWithTypeAndName & columns, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped) const
|
||||
{
|
||||
if (const ColumnVector<T0> * col_left = checkAndGetColumn<ColumnVector<T0>>(col_left_untyped))
|
||||
{
|
||||
if ( executeNumRightType<T0, UInt8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt128>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt256>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int8>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int16>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int64>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int128>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int256>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float32>(block, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float64>(block, result, col_left, col_right_untyped))
|
||||
if ( executeNumRightType<T0, UInt8>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt16>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt32>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt64>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt128>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, UInt256>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int8>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int16>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int32>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int64>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int128>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Int256>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float32>(columns, result, col_left, col_right_untyped)
|
||||
|| executeNumRightType<T0, Float64>(columns, result, col_left, col_right_untyped))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + col_right_untyped->getName()
|
||||
@ -652,20 +652,20 @@ private:
|
||||
}
|
||||
else if (auto col_left_const = checkAndGetColumnConst<ColumnVector<T0>>(col_left_untyped))
|
||||
{
|
||||
if ( executeNumConstRightType<T0, UInt8>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt16>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt32>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt64>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt128>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt256>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int8>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int16>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int32>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int64>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int128>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int256>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float32>(block, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float64>(block, result, col_left_const, col_right_untyped))
|
||||
if ( executeNumConstRightType<T0, UInt8>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt16>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt32>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt64>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt128>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, UInt256>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int8>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int16>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int32>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int64>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int128>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Int256>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float32>(columns, result, col_left_const, col_right_untyped)
|
||||
|| executeNumConstRightType<T0, Float64>(columns, result, col_left_const, col_right_untyped))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column " + col_right_untyped->getName()
|
||||
@ -676,7 +676,7 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeDecimal(ColumnsWithTypeAndName & block, size_t result, const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right) const
|
||||
void executeDecimal(ColumnsWithTypeAndName & columns, size_t result, const ColumnWithTypeAndName & col_left, const ColumnWithTypeAndName & col_right) const
|
||||
{
|
||||
TypeIndex left_number = col_left.type->getTypeId();
|
||||
TypeIndex right_number = col_right.type->getTypeId();
|
||||
@ -688,9 +688,9 @@ private:
|
||||
using RightDataType = typename Types::RightType;
|
||||
|
||||
if (check_decimal_overflow)
|
||||
DecimalComparison<LeftDataType, RightDataType, Op, true>(block, result, col_left, col_right);
|
||||
DecimalComparison<LeftDataType, RightDataType, Op, true>(columns, result, col_left, col_right);
|
||||
else
|
||||
DecimalComparison<LeftDataType, RightDataType, Op, false>(block, result, col_left, col_right);
|
||||
DecimalComparison<LeftDataType, RightDataType, Op, false>(columns, result, col_left, col_right);
|
||||
return true;
|
||||
};
|
||||
|
||||
@ -699,7 +699,7 @@ private:
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
bool executeString(ColumnsWithTypeAndName & block, size_t result, const IColumn * c0, const IColumn * c1) const
|
||||
bool executeString(ColumnsWithTypeAndName & columns, size_t result, const IColumn * c0, const IColumn * c1) const
|
||||
{
|
||||
const ColumnString * c0_string = checkAndGetColumn<ColumnString>(c0);
|
||||
const ColumnString * c1_string = checkAndGetColumn<ColumnString>(c1);
|
||||
@ -759,11 +759,11 @@ private:
|
||||
|
||||
if (c0_const && c1_const)
|
||||
{
|
||||
auto res = executeString(block, result, &c0_const->getDataColumn(), &c1_const->getDataColumn());
|
||||
auto res = executeString(columns, result, &c0_const->getDataColumn(), &c1_const->getDataColumn());
|
||||
if (!res)
|
||||
return false;
|
||||
|
||||
block[result].column = ColumnConst::create(block[result].column, c0_const->size());
|
||||
columns[result].column = ColumnConst::create(columns[result].column, c0_const->size());
|
||||
return true;
|
||||
}
|
||||
else
|
||||
@ -818,13 +818,13 @@ private:
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block[result].column = std::move(c_res);
|
||||
columns[result].column = std::move(c_res);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
bool executeWithConstString(
|
||||
ColumnsWithTypeAndName & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped,
|
||||
ColumnsWithTypeAndName & columns, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped,
|
||||
const DataTypePtr & left_type, const DataTypePtr & right_type, size_t input_rows_count) const
|
||||
{
|
||||
/// To compare something with const string, we cast constant to appropriate type and compare as usual.
|
||||
@ -846,28 +846,28 @@ private:
|
||||
/// If not possible to convert, comparison with =, <, >, <=, >= yields to false and comparison with != yields to true.
|
||||
if (converted.isNull())
|
||||
{
|
||||
block[result].column = DataTypeUInt8().createColumnConst(input_rows_count, IsOperation<Op>::not_equals);
|
||||
columns[result].column = DataTypeUInt8().createColumnConst(input_rows_count, IsOperation<Op>::not_equals);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto column_converted = type_to_compare->createColumnConst(input_rows_count, converted);
|
||||
|
||||
ColumnsWithTypeAndName tmp_block_columns
|
||||
ColumnsWithTypeAndName tmp_columns_columns
|
||||
{
|
||||
{ left_const ? column_converted : col_left_untyped->getPtr(), type_to_compare, "" },
|
||||
{ !left_const ? column_converted : col_right_untyped->getPtr(), type_to_compare, "" },
|
||||
block[result]
|
||||
columns[result]
|
||||
};
|
||||
|
||||
executeImpl(tmp_block_columns, {0, 1}, 2, input_rows_count);
|
||||
executeImpl(tmp_columns_columns, {0, 1}, 2, input_rows_count);
|
||||
|
||||
block[result].column = std::move(tmp_block_columns[2].column);
|
||||
columns[result].column = std::move(tmp_columns_columns[2].column);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void executeTuple(ColumnsWithTypeAndName & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1,
|
||||
void executeTuple(ColumnsWithTypeAndName & columns, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1,
|
||||
size_t input_rows_count) const
|
||||
{
|
||||
/** We will lexicographically compare the tuples. This is done as follows:
|
||||
@ -892,7 +892,7 @@ private:
|
||||
if (tuple_size != typeid_cast<const DataTypeTuple &>(*c1.type).getElements().size())
|
||||
throw Exception("Cannot compare tuples of different sizes.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto & res = block[result];
|
||||
auto & res = columns[result];
|
||||
if (res.type->onlyNull())
|
||||
{
|
||||
res.column = res.type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
@ -927,17 +927,17 @@ private:
|
||||
y[i].column = y_columns[i];
|
||||
}
|
||||
|
||||
executeTupleImpl(block, result, x, y, tuple_size, input_rows_count);
|
||||
executeTupleImpl(columns, result, x, y, tuple_size, input_rows_count);
|
||||
}
|
||||
|
||||
void executeTupleImpl(ColumnsWithTypeAndName & block, size_t result, const ColumnsWithTypeAndName & x,
|
||||
void executeTupleImpl(ColumnsWithTypeAndName & columns, size_t result, const ColumnsWithTypeAndName & x,
|
||||
const ColumnsWithTypeAndName & y, size_t tuple_size,
|
||||
size_t input_rows_count) const;
|
||||
|
||||
void executeTupleEqualityImpl(
|
||||
std::shared_ptr<IFunctionOverloadResolver> func_compare,
|
||||
std::shared_ptr<IFunctionOverloadResolver> func_convolution,
|
||||
ColumnsWithTypeAndName & block,
|
||||
ColumnsWithTypeAndName & columns,
|
||||
size_t result,
|
||||
const ColumnsWithTypeAndName & x,
|
||||
const ColumnsWithTypeAndName & y,
|
||||
@ -949,24 +949,24 @@ private:
|
||||
|
||||
ColumnsWithTypeAndName convolution_types(tuple_size);
|
||||
|
||||
ColumnsWithTypeAndName tmp_block;
|
||||
ColumnsWithTypeAndName tmp_columns;
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
tmp_block.emplace_back(x[i]);
|
||||
tmp_block.emplace_back(y[i]);
|
||||
tmp_columns.emplace_back(x[i]);
|
||||
tmp_columns.emplace_back(y[i]);
|
||||
|
||||
auto impl = func_compare->build({x[i], y[i]});
|
||||
convolution_types[i].type = impl->getReturnType();
|
||||
|
||||
/// Comparison of the elements.
|
||||
tmp_block.emplace_back(ColumnWithTypeAndName{ nullptr, impl->getReturnType(), "" });
|
||||
impl->execute(tmp_block, {i * 3, i * 3 + 1}, i * 3 + 2, input_rows_count);
|
||||
tmp_columns.emplace_back(ColumnWithTypeAndName{ nullptr, impl->getReturnType(), "" });
|
||||
impl->execute(tmp_columns, {i * 3, i * 3 + 1}, i * 3 + 2, input_rows_count);
|
||||
}
|
||||
|
||||
if (tuple_size == 1)
|
||||
{
|
||||
/// Do not call AND for single-element tuple.
|
||||
block[result].column = tmp_block[2].column;
|
||||
columns[result].column = tmp_columns[2].column;
|
||||
return;
|
||||
}
|
||||
|
||||
@ -977,10 +977,10 @@ private:
|
||||
convolution_args[i] = i * 3 + 2;
|
||||
|
||||
auto impl = func_convolution->build(convolution_types);
|
||||
tmp_block.emplace_back(ColumnWithTypeAndName{ nullptr, impl->getReturnType(), "" });
|
||||
tmp_columns.emplace_back(ColumnWithTypeAndName{ nullptr, impl->getReturnType(), "" });
|
||||
|
||||
impl->execute(tmp_block, convolution_args, tuple_size * 3, input_rows_count);
|
||||
block[result].column = tmp_block[tuple_size * 3].column;
|
||||
impl->execute(tmp_columns, convolution_args, tuple_size * 3, input_rows_count);
|
||||
columns[result].column = tmp_columns[tuple_size * 3].column;
|
||||
}
|
||||
|
||||
void executeTupleLessGreaterImpl(
|
||||
@ -989,42 +989,42 @@ private:
|
||||
std::shared_ptr<IFunctionOverloadResolver> func_and,
|
||||
std::shared_ptr<IFunctionOverloadResolver> func_or,
|
||||
std::shared_ptr<IFunctionOverloadResolver> func_equals,
|
||||
ColumnsWithTypeAndName & block,
|
||||
ColumnsWithTypeAndName & columns,
|
||||
size_t result,
|
||||
const ColumnsWithTypeAndName & x,
|
||||
const ColumnsWithTypeAndName & y,
|
||||
size_t tuple_size,
|
||||
size_t input_rows_count) const
|
||||
{
|
||||
ColumnsWithTypeAndName tmp_block;
|
||||
ColumnsWithTypeAndName tmp_columns;
|
||||
|
||||
/// Pairwise comparison of the inequality of all elements; on the equality of all elements except the last.
|
||||
/// (x[i], y[i], x[i] < y[i], x[i] == y[i])
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
tmp_block.emplace_back(x[i]);
|
||||
tmp_block.emplace_back(y[i]);
|
||||
tmp_columns.emplace_back(x[i]);
|
||||
tmp_columns.emplace_back(y[i]);
|
||||
|
||||
tmp_block.emplace_back(ColumnWithTypeAndName()); // pos == i * 4 + 2
|
||||
tmp_columns.emplace_back(ColumnWithTypeAndName()); // pos == i * 4 + 2
|
||||
|
||||
if (i + 1 != tuple_size)
|
||||
{
|
||||
auto impl_head = func_compare_head->build({x[i], y[i]});
|
||||
tmp_block[i * 4 + 2].type = impl_head->getReturnType();
|
||||
impl_head->execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count);
|
||||
tmp_columns[i * 4 + 2].type = impl_head->getReturnType();
|
||||
impl_head->execute(tmp_columns, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count);
|
||||
|
||||
tmp_block.emplace_back(ColumnWithTypeAndName()); // i * 4 + 3
|
||||
tmp_columns.emplace_back(ColumnWithTypeAndName()); // i * 4 + 3
|
||||
|
||||
auto impl_equals = func_equals->build({x[i], y[i]});
|
||||
tmp_block[i * 4 + 3].type = impl_equals->getReturnType();
|
||||
impl_equals->execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 3, input_rows_count);
|
||||
tmp_columns[i * 4 + 3].type = impl_equals->getReturnType();
|
||||
impl_equals->execute(tmp_columns, {i * 4, i * 4 + 1}, i * 4 + 3, input_rows_count);
|
||||
|
||||
}
|
||||
else
|
||||
{
|
||||
auto impl_tail = func_compare_tail->build({x[i], y[i]});
|
||||
tmp_block[i * 4 + 2].type = impl_tail->getReturnType();
|
||||
impl_tail->execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count);
|
||||
tmp_columns[i * 4 + 2].type = impl_tail->getReturnType();
|
||||
impl_tail->execute(tmp_columns, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1039,34 +1039,34 @@ private:
|
||||
{
|
||||
--i;
|
||||
|
||||
size_t and_lhs_pos = tmp_block.size() - 1; // res
|
||||
size_t and_lhs_pos = tmp_columns.size() - 1; // res
|
||||
size_t and_rhs_pos = i * 4 + 3; // `x == y`[i]
|
||||
tmp_block.emplace_back(ColumnWithTypeAndName());
|
||||
tmp_columns.emplace_back(ColumnWithTypeAndName());
|
||||
|
||||
ColumnsWithTypeAndName and_args = {{ nullptr, tmp_block[and_lhs_pos].type, "" },
|
||||
{ nullptr, tmp_block[and_rhs_pos].type, "" }};
|
||||
ColumnsWithTypeAndName and_args = {{ nullptr, tmp_columns[and_lhs_pos].type, "" },
|
||||
{ nullptr, tmp_columns[and_rhs_pos].type, "" }};
|
||||
|
||||
auto func_and_adaptor = func_and->build(and_args);
|
||||
tmp_block[tmp_block.size() - 1].type = func_and_adaptor->getReturnType();
|
||||
func_and_adaptor->execute(tmp_block, {and_lhs_pos, and_rhs_pos}, tmp_block.size() - 1, input_rows_count);
|
||||
tmp_columns[tmp_columns.size() - 1].type = func_and_adaptor->getReturnType();
|
||||
func_and_adaptor->execute(tmp_columns, {and_lhs_pos, and_rhs_pos}, tmp_columns.size() - 1, input_rows_count);
|
||||
|
||||
size_t or_lhs_pos = tmp_block.size() - 1; // (res && `x == y`[i])
|
||||
size_t or_lhs_pos = tmp_columns.size() - 1; // (res && `x == y`[i])
|
||||
size_t or_rhs_pos = i * 4 + 2; // `x < y`[i]
|
||||
tmp_block.emplace_back(ColumnWithTypeAndName());
|
||||
tmp_columns.emplace_back(ColumnWithTypeAndName());
|
||||
|
||||
ColumnsWithTypeAndName or_args = {{ nullptr, tmp_block[or_lhs_pos].type, "" },
|
||||
{ nullptr, tmp_block[or_rhs_pos].type, "" }};
|
||||
ColumnsWithTypeAndName or_args = {{ nullptr, tmp_columns[or_lhs_pos].type, "" },
|
||||
{ nullptr, tmp_columns[or_rhs_pos].type, "" }};
|
||||
|
||||
auto func_or_adaptor = func_or->build(or_args);
|
||||
tmp_block[tmp_block.size() - 1].type = func_or_adaptor->getReturnType();
|
||||
func_or_adaptor->execute(tmp_block, {or_lhs_pos, or_rhs_pos}, tmp_block.size() - 1, input_rows_count);
|
||||
tmp_columns[tmp_columns.size() - 1].type = func_or_adaptor->getReturnType();
|
||||
func_or_adaptor->execute(tmp_columns, {or_lhs_pos, or_rhs_pos}, tmp_columns.size() - 1, input_rows_count);
|
||||
|
||||
}
|
||||
|
||||
block[result].column = tmp_block[tmp_block.size() - 1].column;
|
||||
columns[result].column = tmp_columns[tmp_columns.size() - 1].column;
|
||||
}
|
||||
|
||||
void executeGenericIdenticalTypes(ColumnsWithTypeAndName & block, size_t result, const IColumn * c0, const IColumn * c1) const
|
||||
void executeGenericIdenticalTypes(ColumnsWithTypeAndName & columns, size_t result, const IColumn * c0, const IColumn * c1) const
|
||||
{
|
||||
bool c0_const = isColumnConst(*c0);
|
||||
bool c1_const = isColumnConst(*c1);
|
||||
@ -1075,7 +1075,7 @@ private:
|
||||
{
|
||||
UInt8 res = 0;
|
||||
GenericComparisonImpl<Op<int, int>>::constantConstant(*c0, *c1, res);
|
||||
block[result].column = DataTypeUInt8().createColumnConst(c0->size(), toField(res));
|
||||
columns[result].column = DataTypeUInt8().createColumnConst(c0->size(), toField(res));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1090,18 +1090,18 @@ private:
|
||||
else
|
||||
GenericComparisonImpl<Op<int, int>>::vectorVector(*c0, *c1, vec_res);
|
||||
|
||||
block[result].column = std::move(c_res);
|
||||
columns[result].column = std::move(c_res);
|
||||
}
|
||||
}
|
||||
|
||||
void executeGeneric(ColumnsWithTypeAndName & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1) const
|
||||
void executeGeneric(ColumnsWithTypeAndName & columns, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1) const
|
||||
{
|
||||
DataTypePtr common_type = getLeastSupertype({c0.type, c1.type});
|
||||
|
||||
ColumnPtr c0_converted = castColumn(c0, common_type);
|
||||
ColumnPtr c1_converted = castColumn(c1, common_type);
|
||||
|
||||
executeGenericIdenticalTypes(block, result, c0_converted.get(), c1_converted.get());
|
||||
executeGenericIdenticalTypes(columns, result, c0_converted.get(), c1_converted.get());
|
||||
}
|
||||
|
||||
public:
|
||||
@ -1173,10 +1173,10 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & col_with_type_and_name_left = block[arguments[0]];
|
||||
const auto & col_with_type_and_name_right = block[arguments[1]];
|
||||
const auto & col_with_type_and_name_left = columns[arguments[0]];
|
||||
const auto & col_with_type_and_name_right = columns[arguments[1]];
|
||||
const IColumn * col_left_untyped = col_with_type_and_name_left.column.get();
|
||||
const IColumn * col_right_untyped = col_with_type_and_name_right.column.get();
|
||||
|
||||
@ -1194,12 +1194,12 @@ public:
|
||||
|| IsOperation<Op>::less_or_equals
|
||||
|| IsOperation<Op>::greater_or_equals)
|
||||
{
|
||||
block[result].column = DataTypeUInt8().createColumnConst(input_rows_count, 1u);
|
||||
columns[result].column = DataTypeUInt8().createColumnConst(input_rows_count, 1u);
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
block[result].column = DataTypeUInt8().createColumnConst(input_rows_count, 0u);
|
||||
columns[result].column = DataTypeUInt8().createColumnConst(input_rows_count, 0u);
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -1218,20 +1218,20 @@ public:
|
||||
|
||||
if (left_is_num && right_is_num && !date_and_datetime)
|
||||
{
|
||||
if (!(executeNumLeftType<UInt8>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt16>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt64>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt128>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt256>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int8>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int16>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int64>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int128>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int256>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float32>(block, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float64>(block, result, col_left_untyped, col_right_untyped)))
|
||||
if (!(executeNumLeftType<UInt8>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt16>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt32>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt64>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt128>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<UInt256>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int8>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int16>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int32>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int64>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int128>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Int256>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float32>(columns, result, col_left_untyped, col_right_untyped)
|
||||
|| executeNumLeftType<Float64>(columns, result, col_left_untyped, col_right_untyped)))
|
||||
throw Exception("Illegal column " + col_left_untyped->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -1239,13 +1239,13 @@ public:
|
||||
else if (checkAndGetDataType<DataTypeTuple>(left_type.get())
|
||||
&& checkAndGetDataType<DataTypeTuple>(right_type.get()))
|
||||
{
|
||||
executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
|
||||
executeTuple(columns, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
|
||||
}
|
||||
else if (left_is_string && right_is_string && executeString(block, result, col_left_untyped, col_right_untyped))
|
||||
else if (left_is_string && right_is_string && executeString(columns, result, col_left_untyped, col_right_untyped))
|
||||
{
|
||||
}
|
||||
else if (executeWithConstString(
|
||||
block, result, col_left_untyped, col_right_untyped,
|
||||
columns, result, col_left_untyped, col_right_untyped,
|
||||
left_type, right_type,
|
||||
input_rows_count))
|
||||
{
|
||||
@ -1257,15 +1257,15 @@ public:
|
||||
throw Exception("No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
executeDecimal(block, result, col_with_type_and_name_left, col_with_type_and_name_right);
|
||||
executeDecimal(columns, result, col_with_type_and_name_left, col_with_type_and_name_right);
|
||||
}
|
||||
else if (left_type->equals(*right_type))
|
||||
{
|
||||
executeGenericIdenticalTypes(block, result, col_left_untyped, col_right_untyped);
|
||||
executeGenericIdenticalTypes(columns, result, col_left_untyped, col_right_untyped);
|
||||
}
|
||||
else
|
||||
{
|
||||
executeGeneric(block, result, col_with_type_and_name_left, col_with_type_and_name_right);
|
||||
executeGeneric(columns, result, col_with_type_and_name_left, col_with_type_and_name_right);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -65,10 +65,10 @@ public:
|
||||
return {1};
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
if (isColumnConst(*block[arguments[1]].column))
|
||||
executeConstBuckets(block, arguments, result);
|
||||
if (isColumnConst(*columns[arguments[1]].column))
|
||||
executeConstBuckets(columns, arguments, result);
|
||||
else
|
||||
throw Exception(
|
||||
"The second argument of function " + getName() + " (number of buckets) must be constant", ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -93,9 +93,9 @@ private:
|
||||
return static_cast<BucketsType>(buckets);
|
||||
}
|
||||
|
||||
void executeConstBuckets(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
void executeConstBuckets(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
Field buckets_field = (*block[arguments[1]].column)[0];
|
||||
Field buckets_field = (*columns[arguments[1]].column)[0];
|
||||
BucketsType num_buckets;
|
||||
|
||||
if (buckets_field.getType() == Field::Types::Int64)
|
||||
@ -106,8 +106,8 @@ private:
|
||||
throw Exception("Illegal type " + String(buckets_field.getTypeName()) + " of the second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto & hash_col = block[arguments[0]].column;
|
||||
const IDataType * hash_type = block[arguments[0]].type.get();
|
||||
const auto & hash_col = columns[arguments[0]].column;
|
||||
const IDataType * hash_type = columns[arguments[0]].type.get();
|
||||
auto res_col = ColumnVector<ResultType>::create();
|
||||
|
||||
WhichDataType which(hash_type);
|
||||
@ -132,7 +132,7 @@ private:
|
||||
throw Exception("Illegal type " + hash_type->getName() + " of the first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
block[result].column = std::move(res_col);
|
||||
columns[result].column = std::move(res_col);
|
||||
}
|
||||
|
||||
template <typename CurrentHashType>
|
||||
|
@ -101,10 +101,10 @@ struct ConvertImpl
|
||||
using ToFieldType = typename ToDataType::FieldType;
|
||||
|
||||
template <typename Additions = void *>
|
||||
static void NO_SANITIZE_UNDEFINED execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/,
|
||||
static void NO_SANITIZE_UNDEFINED execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/,
|
||||
Additions additions [[maybe_unused]] = Additions())
|
||||
{
|
||||
const ColumnWithTypeAndName & named_from = block[arguments[0]];
|
||||
const ColumnWithTypeAndName & named_from = columns[arguments[0]];
|
||||
|
||||
using ColVecFrom = typename FromDataType::ColumnType;
|
||||
using ColVecTo = typename ToDataType::ColumnType;
|
||||
@ -161,7 +161,7 @@ struct ConvertImpl
|
||||
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
|
||||
@ -441,9 +441,9 @@ struct FormatImpl<DataTypeDecimal<FieldType>>
|
||||
template <typename FieldType, typename Name>
|
||||
struct ConvertImpl<DataTypeEnum<FieldType>, DataTypeNumber<FieldType>, Name>
|
||||
{
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
block[result].column = block[arguments[0]].column;
|
||||
columns[result].column = columns[arguments[0]].column;
|
||||
}
|
||||
};
|
||||
|
||||
@ -454,16 +454,16 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
|
||||
using FromFieldType = typename FromDataType::FieldType;
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>, ColumnVector<FromFieldType>>;
|
||||
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
const auto & col_with_type_and_name = block[arguments[0]];
|
||||
const auto & col_with_type_and_name = columns[arguments[0]];
|
||||
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
|
||||
|
||||
const DateLUTImpl * time_zone = nullptr;
|
||||
|
||||
/// For argument of DateTime type, second argument with time zone could be specified.
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime> || std::is_same_v<FromDataType, DataTypeDateTime64>)
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(columns, arguments, 1, 0);
|
||||
|
||||
if (const auto col_from = checkAndGetColumn<ColVecType>(col_with_type_and_name.column.get()))
|
||||
{
|
||||
@ -495,10 +495,10 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
|
||||
}
|
||||
|
||||
write_buffer.finalize();
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -508,9 +508,9 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
|
||||
/// Generic conversion of any type to String.
|
||||
struct ConvertImplGenericToString
|
||||
{
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result)
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const auto & col_with_type_and_name = block[arguments[0]];
|
||||
const auto & col_with_type_and_name = columns[arguments[0]];
|
||||
const IDataType & type = *col_with_type_and_name.type;
|
||||
const IColumn & col_from = *col_with_type_and_name.column;
|
||||
|
||||
@ -535,7 +535,7 @@ struct ConvertImplGenericToString
|
||||
}
|
||||
|
||||
write_buffer.finalize();
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -605,9 +605,9 @@ inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, Read
|
||||
|
||||
/** Throw exception with verbose message when string value is not parsed completely.
|
||||
*/
|
||||
[[noreturn]] inline void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, ColumnsWithTypeAndName & block, size_t result)
|
||||
[[noreturn]] inline void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, ColumnsWithTypeAndName & columns, size_t result)
|
||||
{
|
||||
const IDataType & to_type = *block[result].type;
|
||||
const IDataType & to_type = *columns[result].type;
|
||||
|
||||
WriteBufferFromOwnString message_buf;
|
||||
message_buf << "Cannot parse string " << quote << String(read_buffer.buffer().begin(), read_buffer.buffer().size())
|
||||
@ -670,7 +670,7 @@ struct ConvertThroughParsing
|
||||
}
|
||||
|
||||
template <typename Additions = void *>
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count,
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count,
|
||||
Additions additions [[maybe_unused]] = Additions())
|
||||
{
|
||||
using ColVecTo = typename ToDataType::ColumnType;
|
||||
@ -681,20 +681,20 @@ struct ConvertThroughParsing
|
||||
/// For conversion to DateTime type, second argument with time zone could be specified.
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime> || to_datetime64)
|
||||
{
|
||||
const auto result_type = removeNullable(block[result].type);
|
||||
const auto result_type = removeNullable(columns[result].type);
|
||||
// Time zone is already figured out during result type resolution, no need to do it here.
|
||||
if (const auto dt_col = checkAndGetDataType<ToDataType>(result_type.get()))
|
||||
local_time_zone = &dt_col->getTimeZone();
|
||||
else
|
||||
{
|
||||
local_time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
|
||||
local_time_zone = &extractTimeZoneFromFunctionArguments(columns, arguments, 1, 0);
|
||||
}
|
||||
|
||||
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort || parsing_mode == ConvertFromStringParsingMode::BestEffortUS)
|
||||
utc_time_zone = &DateLUT::instance("UTC");
|
||||
}
|
||||
|
||||
const IColumn * col_from = block[arguments[0]].column.get();
|
||||
const IColumn * col_from = columns[arguments[0]].column.get();
|
||||
const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(col_from);
|
||||
const ColumnFixedString * col_from_fixed_string = checkAndGetColumn<ColumnFixedString>(col_from);
|
||||
|
||||
@ -799,7 +799,7 @@ struct ConvertThroughParsing
|
||||
}
|
||||
|
||||
if (!isAllRead(read_buffer))
|
||||
throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
|
||||
throwExceptionForIncompletelyParsedValue(read_buffer, columns, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -847,9 +847,9 @@ struct ConvertThroughParsing
|
||||
}
|
||||
|
||||
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
|
||||
block[result].column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
|
||||
columns[result].column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
|
||||
else
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -865,12 +865,12 @@ struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeFixedStr
|
||||
/// Generic conversion of any type from String. Used for complex types: Array and Tuple.
|
||||
struct ConvertImplGenericFromString
|
||||
{
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result)
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const IColumn & col_from = *block[arguments[0]].column;
|
||||
const IColumn & col_from = *columns[arguments[0]].column;
|
||||
size_t size = col_from.size();
|
||||
|
||||
const IDataType & data_type_to = *block[result].type;
|
||||
const IDataType & data_type_to = *columns[result].type;
|
||||
|
||||
if (const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(&col_from))
|
||||
{
|
||||
@ -892,15 +892,15 @@ struct ConvertImplGenericFromString
|
||||
data_type_to.deserializeAsWholeText(column_to, read_buffer, format_settings);
|
||||
|
||||
if (!read_buffer.eof())
|
||||
throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
|
||||
throwExceptionForIncompletelyParsedValue(read_buffer, columns, result);
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
block[result].column = std::move(res);
|
||||
columns[result].column = std::move(res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of conversion function from string",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -920,9 +920,9 @@ struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp>
|
||||
template <typename T, typename Name>
|
||||
struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
|
||||
{
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
block[result].column = block[arguments[0]].column;
|
||||
columns[result].column = columns[arguments[0]].column;
|
||||
}
|
||||
};
|
||||
|
||||
@ -933,9 +933,9 @@ struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
{
|
||||
static void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
static void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(block[arguments[0]].column.get()))
|
||||
if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
@ -964,10 +964,10 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
}
|
||||
|
||||
data_to.resize(offset_to);
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -1132,11 +1132,11 @@ public:
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
bool canBeExecutedOnDefaultArguments() const override { return false; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
try
|
||||
{
|
||||
executeInternal(block, arguments, result, input_rows_count);
|
||||
executeInternal(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -1144,8 +1144,8 @@ public:
|
||||
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
||||
{
|
||||
e.addMessage("Cannot parse "
|
||||
+ block[result].type->getName() + " from "
|
||||
+ block[arguments[0]].type->getName()
|
||||
+ columns[result].type->getName() + " from "
|
||||
+ columns[arguments[0]].type->getName()
|
||||
+ ", because value is too short");
|
||||
}
|
||||
else if (e.code() == ErrorCodes::CANNOT_PARSE_NUMBER
|
||||
@ -1158,8 +1158,8 @@ public:
|
||||
|| e.code() == ErrorCodes::CANNOT_PARSE_UUID)
|
||||
{
|
||||
e.addMessage("Cannot parse "
|
||||
+ block[result].type->getName() + " from "
|
||||
+ block[arguments[0]].type->getName());
|
||||
+ columns[result].type->getName() + " from "
|
||||
+ columns[arguments[0]].type->getName());
|
||||
}
|
||||
|
||||
throw;
|
||||
@ -1177,13 +1177,13 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
void executeInternal(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void executeInternal(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
if (!arguments.size())
|
||||
throw Exception{"Function " + getName() + " expects at least 1 arguments",
|
||||
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
|
||||
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
|
||||
auto call = [&](const auto & types) -> bool
|
||||
{
|
||||
@ -1206,15 +1206,15 @@ private:
|
||||
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
|
||||
}
|
||||
|
||||
const ColumnWithTypeAndName & scale_column = block[arguments[1]];
|
||||
const ColumnWithTypeAndName & scale_column = columns[arguments[1]];
|
||||
UInt32 scale = extractToDecimalScale(scale_column);
|
||||
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, scale);
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(columns, arguments, result, input_rows_count, scale);
|
||||
}
|
||||
else if constexpr (IsDataTypeDateOrDateTime<RightDataType> && std::is_same_v<LeftDataType, DataTypeDateTime64>)
|
||||
{
|
||||
const auto * dt64 = assert_cast<const DataTypeDateTime64 *>(block[arguments[0]].type.get());
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, dt64->getScale());
|
||||
const auto * dt64 = assert_cast<const DataTypeDateTime64 *>(columns[arguments[0]].type.get());
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(columns, arguments, result, input_rows_count, dt64->getScale());
|
||||
}
|
||||
else if constexpr (IsDataTypeDecimalOrNumber<LeftDataType> && IsDataTypeDecimalOrNumber<RightDataType>)
|
||||
{
|
||||
@ -1233,24 +1233,24 @@ private:
|
||||
throw Exception("Wrong UUID conversion", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
else
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
else
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
|
||||
ConvertImpl<LeftDataType, RightDataType, Name>::execute(columns, arguments, result, input_rows_count);
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
if (isDateTime64<Name, ToDataType>(block, arguments))
|
||||
if (isDateTime64<Name, ToDataType>(columns, arguments))
|
||||
{
|
||||
/// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64
|
||||
const ColumnWithTypeAndName & scale_column = block[arguments[1]];
|
||||
const ColumnWithTypeAndName & scale_column = columns[arguments[1]];
|
||||
UInt32 scale = extractToDecimalScale(scale_column);
|
||||
|
||||
if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64
|
||||
{
|
||||
if (!callOnIndexAndDataType<DataTypeDateTime64>(from_type->getTypeId(), call))
|
||||
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
throw Exception("Illegal type " + columns[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return;
|
||||
@ -1263,10 +1263,10 @@ private:
|
||||
/// Generic conversion of any type to String.
|
||||
if (std::is_same_v<ToDataType, DataTypeString>)
|
||||
{
|
||||
ConvertImplGenericToString::execute(block, arguments, result);
|
||||
ConvertImplGenericToString::execute(columns, arguments, result);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
throw Exception("Illegal type " + columns[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
}
|
||||
@ -1394,56 +1394,56 @@ public:
|
||||
}
|
||||
|
||||
template <typename ConvertToDataType>
|
||||
bool executeInternal(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, UInt32 scale = 0) const
|
||||
bool executeInternal(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, UInt32 scale = 0) const
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
|
||||
if (checkAndGetDataType<DataTypeString>(from_type))
|
||||
{
|
||||
ConvertThroughParsing<DataTypeString, ConvertToDataType, Name, exception_mode, parsing_mode>::execute(
|
||||
block, arguments, result, input_rows_count, scale);
|
||||
columns, arguments, result, input_rows_count, scale);
|
||||
return true;
|
||||
}
|
||||
else if (checkAndGetDataType<DataTypeFixedString>(from_type))
|
||||
{
|
||||
ConvertThroughParsing<DataTypeFixedString, ConvertToDataType, Name, exception_mode, parsing_mode>::execute(
|
||||
block, arguments, result, input_rows_count, scale);
|
||||
columns, arguments, result, input_rows_count, scale);
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
bool ok = true;
|
||||
|
||||
if constexpr (to_decimal)
|
||||
ok = executeInternal<ToDataType>(block, arguments, result, input_rows_count,
|
||||
assert_cast<const ToDataType &>(*removeNullable(block[result].type)).getScale());
|
||||
ok = executeInternal<ToDataType>(columns, arguments, result, input_rows_count,
|
||||
assert_cast<const ToDataType &>(*removeNullable(columns[result].type)).getScale());
|
||||
else
|
||||
{
|
||||
if (isDateTime64<Name, ToDataType>(block, arguments))
|
||||
if (isDateTime64<Name, ToDataType>(columns, arguments))
|
||||
{
|
||||
UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0;
|
||||
if (arguments.size() > 1)
|
||||
scale = extractToDecimalScale(block[arguments[1]]);
|
||||
scale = extractToDecimalScale(columns[arguments[1]]);
|
||||
|
||||
if (scale == 0)
|
||||
ok = executeInternal<DataTypeDateTime>(block, arguments, result, input_rows_count);
|
||||
ok = executeInternal<DataTypeDateTime>(columns, arguments, result, input_rows_count);
|
||||
else
|
||||
{
|
||||
ok = executeInternal<DataTypeDateTime64>(block, arguments, result, input_rows_count, static_cast<UInt32>(scale));
|
||||
ok = executeInternal<DataTypeDateTime64>(columns, arguments, result, input_rows_count, static_cast<UInt32>(scale));
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
ok = executeInternal<ToDataType>(block, arguments, result, input_rows_count);
|
||||
ok = executeInternal<ToDataType>(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
}
|
||||
|
||||
if (!ok)
|
||||
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName()
|
||||
throw Exception("Illegal type " + columns[arguments[0]].type->getName() + " of argument of function " + getName()
|
||||
+ ". Only String or FixedString argument is accepted for try-conversion function."
|
||||
+ " For other arguments, use function without 'orZero' or 'orNull'.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -1864,14 +1864,14 @@ public:
|
||||
String getName() const override { return name; }
|
||||
|
||||
protected:
|
||||
void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
/// drop second argument, pass others
|
||||
ColumnNumbers new_arguments{arguments.front()};
|
||||
if (arguments.size() > 2)
|
||||
new_arguments.insert(std::end(new_arguments), std::next(std::begin(arguments), 2), std::end(arguments));
|
||||
|
||||
wrapper_function(block, new_arguments, result, input_rows_count);
|
||||
wrapper_function(columns, new_arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
@ -1903,7 +1903,7 @@ public:
|
||||
const DataTypes & getArgumentTypes() const override { return argument_types; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & /*sample_columns*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
||||
{
|
||||
return std::make_unique<ExecutableFunctionCast>(
|
||||
prepareUnpackDictionaries(getArgumentTypes()[0], getReturnType()), name);
|
||||
@ -1950,9 +1950,9 @@ private:
|
||||
FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
|
||||
.build({ColumnWithTypeAndName{nullptr, from_type, ""}});
|
||||
|
||||
return [function_adaptor] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
return [function_adaptor] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
function_adaptor->execute(block, arguments, result, input_rows_count);
|
||||
function_adaptor->execute(columns, arguments, result, input_rows_count);
|
||||
};
|
||||
}
|
||||
|
||||
@ -1964,9 +1964,9 @@ private:
|
||||
FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
|
||||
.build({ColumnWithTypeAndName{nullptr, from_type, ""}});
|
||||
|
||||
return [function_adaptor] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
return [function_adaptor] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
function_adaptor->execute(block, arguments, result, input_rows_count);
|
||||
function_adaptor->execute(columns, arguments, result, input_rows_count);
|
||||
};
|
||||
}
|
||||
|
||||
@ -1975,9 +1975,9 @@ private:
|
||||
if (!isStringOrFixedString(from_type))
|
||||
throw Exception{"CAST AS FixedString is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED};
|
||||
|
||||
return [N] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
return [N] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
FunctionToFixedString::executeForN(block, arguments, result, N);
|
||||
FunctionToFixedString::executeForN(columns, arguments, result, N);
|
||||
};
|
||||
}
|
||||
|
||||
@ -1992,9 +1992,9 @@ private:
|
||||
FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
|
||||
.build({ColumnWithTypeAndName{nullptr, from_type, ""}});
|
||||
|
||||
return [function_adaptor] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
return [function_adaptor] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
function_adaptor->execute(block, arguments, result, input_rows_count);
|
||||
function_adaptor->execute(columns, arguments, result, input_rows_count);
|
||||
};
|
||||
}
|
||||
|
||||
@ -2016,7 +2016,7 @@ private:
|
||||
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
|
||||
ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
|
||||
return [type_index, scale, to_type] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
return [type_index, scale, to_type] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
auto res = callOnIndexAndDataType<ToDataType>(type_index, [&](const auto & types) -> bool
|
||||
{
|
||||
@ -2024,7 +2024,7 @@ private:
|
||||
using LeftDataType = typename Types::LeftType;
|
||||
using RightDataType = typename Types::RightType;
|
||||
|
||||
ConvertImpl<LeftDataType, RightDataType, NameCast>::execute(block, arguments, result, input_rows_count, scale);
|
||||
ConvertImpl<LeftDataType, RightDataType, NameCast>::execute(columns, arguments, result, input_rows_count, scale);
|
||||
return true;
|
||||
});
|
||||
|
||||
@ -2042,9 +2042,9 @@ private:
|
||||
/// Conversion from String through parsing.
|
||||
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
||||
{
|
||||
return [] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
return [] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
ConvertImplGenericFromString::execute(block, arguments, result);
|
||||
ConvertImplGenericFromString::execute(columns, arguments, result);
|
||||
};
|
||||
}
|
||||
else
|
||||
@ -2057,9 +2057,9 @@ private:
|
||||
/// Conversion from String through parsing.
|
||||
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
||||
{
|
||||
return [] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
return [] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
ConvertImplGenericFromString::execute(block, arguments, result);
|
||||
ConvertImplGenericFromString::execute(columns, arguments, result);
|
||||
};
|
||||
}
|
||||
|
||||
@ -2086,25 +2086,25 @@ private:
|
||||
const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type);
|
||||
|
||||
return [nested_function, from_nested_type, to_nested_type](
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
const auto & array_arg = block[arguments.front()];
|
||||
const auto & array_arg = columns[arguments.front()];
|
||||
|
||||
if (const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(array_arg.column.get()))
|
||||
{
|
||||
/// create block for converting nested column containing original and result columns
|
||||
ColumnsWithTypeAndName nested_block_columns
|
||||
/// create columns for converting nested column containing original and result columns
|
||||
ColumnsWithTypeAndName nested_columns_columns
|
||||
{
|
||||
{ col_array->getDataPtr(), from_nested_type, "" },
|
||||
{ nullptr, to_nested_type, "" }
|
||||
};
|
||||
ColumnsWithTypeAndName nested_block(nested_block_columns);
|
||||
ColumnsWithTypeAndName nested_columns(nested_columns_columns);
|
||||
|
||||
/// convert nested column
|
||||
nested_function(nested_block, {0}, 1, nested_block_columns.front().column->size());
|
||||
nested_function(nested_columns, {0}, 1, nested_columns_columns.front().column->size());
|
||||
|
||||
/// set converted nested column to result
|
||||
block[result].column = ColumnArray::create(nested_block[1].column, col_array->getOffsetsPtr());
|
||||
columns[result].column = ColumnArray::create(nested_columns[1].column, col_array->getOffsetsPtr());
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + array_arg.column->getName() + " for function CAST AS Array", ErrorCodes::LOGICAL_ERROR};
|
||||
@ -2116,9 +2116,9 @@ private:
|
||||
/// Conversion from String through parsing.
|
||||
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
||||
{
|
||||
return [] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
return [] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
ConvertImplGenericFromString::execute(block, arguments, result);
|
||||
ConvertImplGenericFromString::execute(columns, arguments, result);
|
||||
};
|
||||
}
|
||||
|
||||
@ -2141,37 +2141,37 @@ private:
|
||||
element_wrappers.push_back(prepareUnpackDictionaries(idx_type.second, to_element_types[idx_type.first]));
|
||||
|
||||
return [element_wrappers, from_element_types, to_element_types]
|
||||
(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
const auto col = block[arguments.front()].column.get();
|
||||
const auto col = columns[arguments.front()].column.get();
|
||||
|
||||
/// copy tuple elements to a separate block
|
||||
ColumnsWithTypeAndName element_block_columns;
|
||||
/// copy tuple elements to a separate columns
|
||||
ColumnsWithTypeAndName element_columns_columns;
|
||||
|
||||
size_t tuple_size = from_element_types.size();
|
||||
const ColumnTuple & column_tuple = typeid_cast<const ColumnTuple &>(*col);
|
||||
|
||||
/// create columns for source elements
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
element_block_columns.emplace_back(ColumnWithTypeAndName{ column_tuple.getColumns()[i], from_element_types[i], "" });
|
||||
element_columns_columns.emplace_back(ColumnWithTypeAndName{ column_tuple.getColumns()[i], from_element_types[i], "" });
|
||||
|
||||
/// create columns for converted elements
|
||||
for (const auto & to_element_type : to_element_types)
|
||||
element_block_columns.emplace_back(ColumnWithTypeAndName{ nullptr, to_element_type, "" });
|
||||
element_columns_columns.emplace_back(ColumnWithTypeAndName{ nullptr, to_element_type, "" });
|
||||
|
||||
/// insert column for converted tuple
|
||||
element_block_columns.emplace_back(ColumnWithTypeAndName{ nullptr, std::make_shared<DataTypeTuple>(to_element_types), "" });
|
||||
element_columns_columns.emplace_back(ColumnWithTypeAndName{ nullptr, std::make_shared<DataTypeTuple>(to_element_types), "" });
|
||||
|
||||
/// invoke conversion for each element
|
||||
for (const auto idx_element_wrapper : ext::enumerate(element_wrappers))
|
||||
idx_element_wrapper.second(element_block_columns, { idx_element_wrapper.first },
|
||||
idx_element_wrapper.second(element_columns_columns, { idx_element_wrapper.first },
|
||||
tuple_size + idx_element_wrapper.first, input_rows_count);
|
||||
|
||||
Columns converted_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
converted_columns[i] = element_block_columns[tuple_size + i].column;
|
||||
converted_columns[i] = element_columns_columns[tuple_size + i].column;
|
||||
|
||||
block[result].column = ColumnTuple::create(converted_columns);
|
||||
columns[result].column = ColumnTuple::create(converted_columns);
|
||||
};
|
||||
}
|
||||
|
||||
@ -2196,9 +2196,9 @@ private:
|
||||
auto func_or_adaptor = FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
|
||||
.build(ColumnsWithTypeAndName{{nullptr, from_type, "" }});
|
||||
|
||||
return [func_or_adaptor] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
return [func_or_adaptor] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
func_or_adaptor->execute(block, arguments, result, input_rows_count);
|
||||
func_or_adaptor->execute(columns, arguments, result, input_rows_count);
|
||||
};
|
||||
}
|
||||
else
|
||||
@ -2235,22 +2235,22 @@ private:
|
||||
WrapperType createStringToEnumWrapper(ssize_t source_is_nullable) const
|
||||
{
|
||||
const char * function_name = name;
|
||||
return [function_name, source_is_nullable] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
return [function_name, source_is_nullable] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
const auto first_col = block[arguments.front()].column.get();
|
||||
const auto first_col = columns[arguments.front()].column.get();
|
||||
|
||||
auto & col_with_type_and_name = block[result];
|
||||
auto & col_with_type_and_name = columns[result];
|
||||
const auto & result_type = typeid_cast<const EnumType &>(*col_with_type_and_name.type);
|
||||
|
||||
const ColumnStringType * col = typeid_cast<const ColumnStringType *>(first_col);
|
||||
const ColumnNullable * nullable_col = nullptr;
|
||||
if (source_is_nullable)
|
||||
{
|
||||
if (block.size() <= arguments.front() + 1)
|
||||
if (columns.size() <= arguments.front() + 1)
|
||||
throw Exception("Not enough columns", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
size_t nullable_pos = block.size() - 1;
|
||||
nullable_col = typeid_cast<const ColumnNullable *>(block[nullable_pos].column.get());
|
||||
size_t nullable_pos = columns.size() - 1;
|
||||
nullable_col = typeid_cast<const ColumnNullable *>(columns[nullable_pos].column.get());
|
||||
if (col && nullable_col && nullable_col->size() != col->size())
|
||||
throw Exception("ColumnNullable is not compatible with original", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
@ -2287,19 +2287,19 @@ private:
|
||||
|
||||
WrapperType createIdentityWrapper(const DataTypePtr &) const
|
||||
{
|
||||
return [] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
return [] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
block[result].column = block[arguments.front()].column;
|
||||
columns[result].column = columns[arguments.front()].column;
|
||||
};
|
||||
}
|
||||
|
||||
WrapperType createNothingWrapper(const IDataType * to_type) const
|
||||
{
|
||||
ColumnPtr res = to_type->createColumnConstWithDefaultValue(1);
|
||||
return [res] (ColumnsWithTypeAndName & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
||||
return [res] (ColumnsWithTypeAndName & columns, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
/// Column of Nothing type is trivially convertible to any other column
|
||||
block[result].column = res->cloneResized(input_rows_count)->convertToFullColumnIfConst();
|
||||
columns[result].column = res->cloneResized(input_rows_count)->convertToFullColumnIfConst();
|
||||
};
|
||||
}
|
||||
|
||||
@ -2315,9 +2315,9 @@ private:
|
||||
if (!to_nested->isNullable())
|
||||
throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
|
||||
return [](ColumnsWithTypeAndName & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
||||
return [](ColumnsWithTypeAndName & columns, const ColumnNumbers &, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
auto & res = block[result];
|
||||
auto & res = columns[result];
|
||||
res.column = res.type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst();
|
||||
};
|
||||
}
|
||||
@ -2333,10 +2333,10 @@ private:
|
||||
return wrapper;
|
||||
|
||||
return [wrapper, from_low_cardinality, to_low_cardinality, skip_not_null_check]
|
||||
(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
auto & arg = block[arguments[0]];
|
||||
auto & res = block[result];
|
||||
auto & arg = columns[arguments[0]];
|
||||
auto & res = columns[result];
|
||||
|
||||
ColumnPtr res_indexes;
|
||||
/// For some types default can't be casted (for example, String to Int). In that case convert column to full.
|
||||
@ -2344,7 +2344,7 @@ private:
|
||||
|
||||
{
|
||||
/// Replace argument and result columns (and types) to dictionary key columns (and types).
|
||||
/// Call nested wrapper in order to cast dictionary keys. Then restore block.
|
||||
/// Call nested wrapper in order to cast dictionary keys. Then restore columns.
|
||||
auto prev_arg_col = arg.column;
|
||||
auto prev_arg_type = arg.type;
|
||||
auto prev_res_type = res.type;
|
||||
@ -2376,7 +2376,7 @@ private:
|
||||
}
|
||||
|
||||
/// Perform the requested conversion.
|
||||
wrapper(block, arguments, result, tmp_rows_count);
|
||||
wrapper(columns, arguments, result, tmp_rows_count);
|
||||
|
||||
arg.column = prev_arg_col;
|
||||
arg.type = prev_arg_type;
|
||||
@ -2415,57 +2415,57 @@ private:
|
||||
if (result_is_nullable)
|
||||
{
|
||||
return [wrapper, source_is_nullable]
|
||||
(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
/// Create a temporary block on which to perform the operation.
|
||||
auto & res = block[result];
|
||||
/// Create a temporary columns on which to perform the operation.
|
||||
auto & res = columns[result];
|
||||
const auto & ret_type = res.type;
|
||||
const auto & nullable_type = static_cast<const DataTypeNullable &>(*ret_type);
|
||||
const auto & nested_type = nullable_type.getNestedType();
|
||||
|
||||
ColumnsWithTypeAndName tmp_block_columns;
|
||||
ColumnsWithTypeAndName tmp_columns_columns;
|
||||
if (source_is_nullable)
|
||||
tmp_block_columns = createBlockWithNestedColumns(block, arguments);
|
||||
tmp_columns_columns = createBlockWithNestedColumns(columns, arguments);
|
||||
else
|
||||
tmp_block_columns = block;
|
||||
tmp_columns_columns = columns;
|
||||
|
||||
size_t tmp_res_index = block.size();
|
||||
tmp_block_columns.emplace_back(ColumnWithTypeAndName {nullptr, nested_type, ""});
|
||||
size_t tmp_res_index = columns.size();
|
||||
tmp_columns_columns.emplace_back(ColumnWithTypeAndName {nullptr, nested_type, ""});
|
||||
/// Add original ColumnNullable for createStringToEnumWrapper()
|
||||
if (source_is_nullable)
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Invalid number of arguments", ErrorCodes::LOGICAL_ERROR);
|
||||
tmp_block_columns.emplace_back(block[arguments.front()]);
|
||||
tmp_columns_columns.emplace_back(columns[arguments.front()]);
|
||||
}
|
||||
|
||||
/// Perform the requested conversion.
|
||||
wrapper(tmp_block_columns, arguments, tmp_res_index, input_rows_count);
|
||||
wrapper(tmp_columns_columns, arguments, tmp_res_index, input_rows_count);
|
||||
|
||||
const auto & tmp_res = tmp_block_columns[tmp_res_index];
|
||||
const auto & tmp_res = tmp_columns_columns[tmp_res_index];
|
||||
|
||||
/// May happen in fuzzy tests. For debug purpose.
|
||||
if (!tmp_res.column)
|
||||
throw Exception("Couldn't convert " + block[arguments[0]].type->getName() + " to "
|
||||
throw Exception("Couldn't convert " + columns[arguments[0]].type->getName() + " to "
|
||||
+ nested_type->getName() + " in " + " prepareRemoveNullable wrapper.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
res.column = wrapInNullable(tmp_res.column, {block[arguments[0]], tmp_res}, {0}, 1, input_rows_count);
|
||||
res.column = wrapInNullable(tmp_res.column, {columns[arguments[0]], tmp_res}, {0}, 1, input_rows_count);
|
||||
};
|
||||
}
|
||||
else if (source_is_nullable)
|
||||
{
|
||||
/// Conversion from Nullable to non-Nullable.
|
||||
|
||||
return [wrapper, skip_not_null_check] (ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
return [wrapper, skip_not_null_check] (ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
auto tmp_block_columns = createBlockWithNestedColumns(block, arguments, result);
|
||||
auto tmp_columns_columns = createBlockWithNestedColumns(columns, arguments, result);
|
||||
|
||||
/// Check that all values are not-NULL.
|
||||
/// Check can be skipped in case if LowCardinality dictionary is transformed.
|
||||
/// In that case, correctness will be checked beforehand.
|
||||
if (!skip_not_null_check)
|
||||
{
|
||||
const auto & col = block[arguments[0]].column;
|
||||
const auto & col = columns[arguments[0]].column;
|
||||
const auto & nullable_col = assert_cast<const ColumnNullable &>(*col);
|
||||
const auto & null_map = nullable_col.getNullMapData();
|
||||
|
||||
@ -2474,8 +2474,8 @@ private:
|
||||
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
|
||||
}
|
||||
|
||||
wrapper(tmp_block_columns, arguments, result, input_rows_count);
|
||||
block[result].column = tmp_block_columns[result].column;
|
||||
wrapper(tmp_columns_columns, arguments, result, input_rows_count);
|
||||
columns[result].column = tmp_columns_columns[result].column;
|
||||
};
|
||||
}
|
||||
else
|
||||
|
@ -183,17 +183,17 @@ public:
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
/// The dictionary key that defines the "point of view".
|
||||
std::string dict_key;
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
|
||||
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(columns[arguments[1]].column.get());
|
||||
|
||||
if (!key_col)
|
||||
throw Exception("Illegal column " + block[arguments[1]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[1]].column->getName()
|
||||
+ " of second ('point of view') argument of function " + name
|
||||
+ ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -203,7 +203,7 @@ public:
|
||||
|
||||
const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
|
||||
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<T>::create();
|
||||
|
||||
@ -215,10 +215,10 @@ public:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(vec_from[i], dict);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -279,17 +279,17 @@ public:
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
/// The dictionary key that defines the "point of view".
|
||||
std::string dict_key;
|
||||
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block[arguments[2]].column.get());
|
||||
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(columns[arguments[2]].column.get());
|
||||
|
||||
if (!key_col)
|
||||
throw Exception("Illegal column " + block[arguments[2]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[2]].column->getName()
|
||||
+ " of third ('point of view') argument of function " + name
|
||||
+ ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -299,10 +299,10 @@ public:
|
||||
|
||||
const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
|
||||
|
||||
const ColumnVector<T> * col_vec1 = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get());
|
||||
const ColumnVector<T> * col_vec2 = checkAndGetColumn<ColumnVector<T>>(block[arguments[1]].column.get());
|
||||
const ColumnConst * col_const1 = checkAndGetColumnConst<ColumnVector<T>>(block[arguments[0]].column.get());
|
||||
const ColumnConst * col_const2 = checkAndGetColumnConst<ColumnVector<T>>(block[arguments[1]].column.get());
|
||||
const ColumnVector<T> * col_vec1 = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get());
|
||||
const ColumnVector<T> * col_vec2 = checkAndGetColumn<ColumnVector<T>>(columns[arguments[1]].column.get());
|
||||
const ColumnConst * col_const1 = checkAndGetColumnConst<ColumnVector<T>>(columns[arguments[0]].column.get());
|
||||
const ColumnConst * col_const2 = checkAndGetColumnConst<ColumnVector<T>>(columns[arguments[1]].column.get());
|
||||
|
||||
if (col_vec1 && col_vec2)
|
||||
{
|
||||
@ -317,7 +317,7 @@ public:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(vec_from1[i], vec_from2[i], dict);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else if (col_vec1 && col_const2)
|
||||
{
|
||||
@ -332,7 +332,7 @@ public:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(vec_from1[i], const_from2, dict);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else if (col_const1 && col_vec2)
|
||||
{
|
||||
@ -347,16 +347,16 @@ public:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Transform::apply(const_from1, vec_from2[i], dict);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else if (col_const1 && col_const2)
|
||||
{
|
||||
block[result].column = DataTypeUInt8().createColumnConst(col_const1->size(),
|
||||
columns[result].column = DataTypeUInt8().createColumnConst(col_const1->size(),
|
||||
toField(Transform::apply(col_const1->template getValue<T>(), col_const2->template getValue<T>(), dict)));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal columns " + block[arguments[0]].column->getName()
|
||||
+ " and " + block[arguments[1]].column->getName()
|
||||
throw Exception("Illegal columns " + columns[arguments[0]].column->getName()
|
||||
+ " and " + columns[arguments[1]].column->getName()
|
||||
+ " of arguments of function " + name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -415,17 +415,17 @@ public:
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
/// The dictionary key that defines the "point of view".
|
||||
std::string dict_key;
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
|
||||
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(columns[arguments[1]].column.get());
|
||||
|
||||
if (!key_col)
|
||||
throw Exception("Illegal column " + block[arguments[1]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[1]].column->getName()
|
||||
+ " of second ('point of view') argument of function " + name
|
||||
+ ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -435,7 +435,7 @@ public:
|
||||
|
||||
const typename DictGetter::Dst & dict = DictGetter::get(*owned_dict, dict_key);
|
||||
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_values = ColumnVector<T>::create();
|
||||
auto col_offsets = ColumnArray::ColumnOffsets::create();
|
||||
@ -459,10 +459,10 @@ public:
|
||||
res_offsets[i] = res_values.size();
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(col_values), std::move(col_offsets));
|
||||
columns[result].column = ColumnArray::create(std::move(col_values), std::move(col_offsets));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -620,24 +620,24 @@ public:
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
RegionsNames::Language language = RegionsNames::Language::ru;
|
||||
|
||||
/// If the result language is specified
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
if (const ColumnConst * col_language = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get()))
|
||||
if (const ColumnConst * col_language = checkAndGetColumnConst<ColumnString>(columns[arguments[1]].column.get()))
|
||||
language = RegionsNames::getLanguageEnum(col_language->getValue<String>());
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[1]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[1]].column->getName()
|
||||
+ " of the second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
const RegionsNames & dict = *owned_dict;
|
||||
|
||||
if (const ColumnUInt32 * col_from = typeid_cast<const ColumnUInt32 *>(block[arguments[0]].column.get()))
|
||||
if (const ColumnUInt32 * col_from = typeid_cast<const ColumnUInt32 *>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
@ -649,10 +649,10 @@ public:
|
||||
col_to->insertDataWithTerminatingZero(name_ref.data, name_ref.size + 1);
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of the first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -69,9 +69,9 @@ DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const ColumnsWithTypeAndNam
|
||||
return type;
|
||||
}
|
||||
|
||||
void FunctionModelEvaluate::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
void FunctionModelEvaluate::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
{
|
||||
const auto * name_col = checkAndGetColumnConst<ColumnString>(block[arguments[0]].column.get());
|
||||
const auto * name_col = checkAndGetColumnConst<ColumnString>(columns[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 & block, const Co
|
||||
columns.reserve(arguments.size());
|
||||
for (auto arg : ext::range(1, arguments.size()))
|
||||
{
|
||||
auto & column = block[arguments[arg]].column;
|
||||
auto & column = columns[arguments[arg]].column;
|
||||
columns.push_back(column.get());
|
||||
if (auto full_column = column->convertToFullColumnIfConst())
|
||||
{
|
||||
@ -130,7 +130,7 @@ void FunctionModelEvaluate::executeImpl(ColumnsWithTypeAndName & block, const Co
|
||||
res = ColumnNullable::create(res, null_map);
|
||||
}
|
||||
|
||||
block[result].column = res;
|
||||
columns[result].column = res;
|
||||
}
|
||||
|
||||
void registerFunctionsExternalModels(FunctionFactory & factory)
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
const ExternalModelsLoader & models_loader;
|
||||
|
@ -57,17 +57,17 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const 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)))
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
if (!(executeType<UInt8>(columns, arguments, result)
|
||||
|| executeType<UInt16>(columns, arguments, result)
|
||||
|| executeType<UInt32>(columns, arguments, result)
|
||||
|| executeType<UInt64>(columns, arguments, result)
|
||||
|| executeType<Int8>(columns, arguments, result)
|
||||
|| executeType<Int16>(columns, arguments, result)
|
||||
|| executeType<Int32>(columns, arguments, result)
|
||||
|| executeType<Int64>(columns, arguments, result)))
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -93,9 +93,9 @@ private:
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeType(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
bool executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
@ -116,7 +116,7 @@ private:
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -153,28 +153,28 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const 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)))
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
if (!(executeType<UInt8>(columns, arguments, result)
|
||||
|| executeType<UInt16>(columns, arguments, result)
|
||||
|| executeType<UInt32>(columns, arguments, result)
|
||||
|| executeType<UInt64>(columns, arguments, result)
|
||||
|| executeType<Int8>(columns, arguments, result)
|
||||
|| executeType<Int16>(columns, arguments, result)
|
||||
|| executeType<Int32>(columns, arguments, result)
|
||||
|| executeType<Int64>(columns, arguments, result)
|
||||
|| executeType<Float32>(columns, arguments, result)
|
||||
|| executeType<Float64>(columns, arguments, result)))
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
bool executeType(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
bool executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
@ -195,7 +195,7 @@ private:
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -229,28 +229,28 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const 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)))
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
if (!(executeType<UInt8>(columns, arguments, result)
|
||||
|| executeType<UInt16>(columns, arguments, result)
|
||||
|| executeType<UInt32>(columns, arguments, result)
|
||||
|| executeType<UInt64>(columns, arguments, result)
|
||||
|| executeType<Int8>(columns, arguments, result)
|
||||
|| executeType<Int16>(columns, arguments, result)
|
||||
|| executeType<Int32>(columns, arguments, result)
|
||||
|| executeType<Int64>(columns, arguments, result)
|
||||
|| executeType<Float32>(columns, arguments, result)
|
||||
|| executeType<Float64>(columns, arguments, result)))
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
bool executeType(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
bool executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
|
||||
@ -271,7 +271,7 @@ private:
|
||||
}
|
||||
|
||||
buf_to.finalize();
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -543,9 +543,9 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block[arguments[0]].column.get()))
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_to = ColumnFixedString::create(Impl::length);
|
||||
|
||||
@ -566,10 +566,10 @@ public:
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else if (
|
||||
const ColumnFixedString * col_from_fix = checkAndGetColumn<ColumnFixedString>(block[arguments[0]].column.get()))
|
||||
const ColumnFixedString * col_from_fix = checkAndGetColumn<ColumnFixedString>(columns[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]));
|
||||
}
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[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 & block, const ColumnNumbers & arguments, size_t result) const
|
||||
void executeType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<FromType>, ColumnDecimal<FromType>, ColumnVector<FromType>>;
|
||||
|
||||
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(block[arguments[0]].column.get()))
|
||||
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(columns[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]);
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + Name::name,
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -647,25 +647,25 @@ public:
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const IDataType * from_type = block[arguments[0]].type.get();
|
||||
const IDataType * from_type = columns[arguments[0]].type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (which.isUInt8()) executeType<UInt8>(block, arguments, result);
|
||||
else if (which.isUInt16()) executeType<UInt16>(block, arguments, result);
|
||||
else if (which.isUInt32()) executeType<UInt32>(block, arguments, result);
|
||||
else if (which.isUInt64()) executeType<UInt64>(block, arguments, result);
|
||||
else if (which.isInt8()) executeType<Int8>(block, arguments, result);
|
||||
else if (which.isInt16()) executeType<Int16>(block, arguments, result);
|
||||
else if (which.isInt32()) executeType<Int32>(block, arguments, result);
|
||||
else if (which.isInt64()) executeType<Int64>(block, arguments, result);
|
||||
else if (which.isDate()) executeType<UInt16>(block, arguments, result);
|
||||
else if (which.isDateTime()) executeType<UInt32>(block, arguments, result);
|
||||
else if (which.isDecimal32()) executeType<Decimal32>(block, arguments, result);
|
||||
else if (which.isDecimal64()) executeType<Decimal64>(block, arguments, result);
|
||||
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);
|
||||
else
|
||||
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
throw Exception("Illegal type " + columns[arguments[0]].type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
@ -689,9 +689,9 @@ public:
|
||||
#endif
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
selector.selectAndExecute(block, arguments, result, input_rows_count);
|
||||
selector.selectAndExecute(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
@ -1041,7 +1041,7 @@ public:
|
||||
return std::make_shared<DataTypeNumber<ToType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
size_t rows = input_rows_count;
|
||||
auto col_to = ColumnVector<ToType>::create(rows);
|
||||
@ -1059,11 +1059,11 @@ public:
|
||||
bool is_first_argument = true;
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & col = block[arguments[i]];
|
||||
const ColumnWithTypeAndName & col = columns[arguments[i]];
|
||||
executeForArgument(col.type.get(), col.column.get(), vec_to, is_first_argument);
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -1086,9 +1086,9 @@ public:
|
||||
#endif
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
selector.selectAndExecute(block, arguments, result, input_rows_count);
|
||||
selector.selectAndExecute(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
@ -1209,22 +1209,22 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto arg_count = arguments.size();
|
||||
|
||||
if (arg_count == 1)
|
||||
executeSingleArg(block, arguments, result);
|
||||
executeSingleArg(columns, arguments, result);
|
||||
else if (arg_count == 2)
|
||||
executeTwoArgs(block, arguments, result);
|
||||
executeTwoArgs(columns, arguments, result);
|
||||
else
|
||||
throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
||||
private:
|
||||
void executeSingleArg(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result) const
|
||||
void executeSingleArg(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result) const
|
||||
{
|
||||
const auto col_untyped = block[arguments.front()].column.get();
|
||||
const auto col_untyped = columns[arguments.front()].column.get();
|
||||
|
||||
if (const auto col_from = checkAndGetColumn<ColumnString>(col_untyped))
|
||||
{
|
||||
@ -1245,22 +1245,22 @@ private:
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + block[arguments[0]].column->getName() +
|
||||
throw Exception{"Illegal column " + columns[arguments[0]].column->getName() +
|
||||
" of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
void executeTwoArgs(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const size_t result) const
|
||||
void executeTwoArgs(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const size_t result) const
|
||||
{
|
||||
const auto level_col = block[arguments.back()].column.get();
|
||||
const auto level_col = columns[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 = block[arguments.front()].column.get();
|
||||
const auto col_untyped = columns[arguments.front()].column.get();
|
||||
if (const auto col_from = checkAndGetColumn<ColumnString>(col_untyped))
|
||||
{
|
||||
const auto size = col_from->size();
|
||||
@ -1281,10 +1281,10 @@ private:
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + block[arguments[0]].column->getName() +
|
||||
throw Exception{"Illegal column " + columns[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 & block, 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, const ColumnNumbers & arguments, 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 = block[arguments[i]];
|
||||
const auto & column = columns[arguments[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,15 +55,15 @@ public:
|
||||
class Executor
|
||||
{
|
||||
public:
|
||||
static void run(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count)
|
||||
static void run(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count)
|
||||
{
|
||||
MutableColumnPtr to{block[result_pos].type->createColumn()};
|
||||
MutableColumnPtr to{columns[result_pos].type->createColumn()};
|
||||
to->reserve(input_rows_count);
|
||||
|
||||
if (arguments.size() < 1)
|
||||
throw Exception{"Function " + String(Name::name) + " requires at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
const auto & first_column = block[arguments[0]];
|
||||
const auto & first_column = columns[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};
|
||||
@ -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(block, arguments);
|
||||
std::vector<Move> moves = prepareMoves(Name::name, block, arguments, 1, num_index_arguments);
|
||||
size_t num_index_arguments = Impl<JSONParser>::getNumberOfIndexArguments(columns, arguments);
|
||||
std::vector<Move> moves = prepareMoves(Name::name, columns, arguments, 1, num_index_arguments);
|
||||
|
||||
/// Preallocate memory in parser if necessary.
|
||||
JSONParser parser;
|
||||
@ -95,7 +95,7 @@ public:
|
||||
|
||||
/// 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, block, arguments, result_pos);
|
||||
impl.prepare(Name::name, columns, arguments, result_pos);
|
||||
|
||||
using Element = typename JSONParser::Element;
|
||||
|
||||
@ -121,7 +121,7 @@ public:
|
||||
/// Perform moves.
|
||||
Element element;
|
||||
std::string_view last_key;
|
||||
bool moves_ok = performMoves<JSONParser>(block, arguments, i, document, moves, element, last_key);
|
||||
bool moves_ok = performMoves<JSONParser>(columns, 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();
|
||||
}
|
||||
block[result_pos].column = std::move(to);
|
||||
columns[result_pos].column = std::move(to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -166,11 +166,11 @@ private:
|
||||
String key;
|
||||
};
|
||||
|
||||
static std::vector<Move> prepareMoves(const char * function_name, ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t first_index_argument, size_t num_index_arguments);
|
||||
static std::vector<Move> prepareMoves(const char * function_name, ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, 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 & block, const ColumnNumbers & arguments, size_t row,
|
||||
static bool performMoves(const ColumnsWithTypeAndName & columns, const ColumnNumbers & 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 = (*block[arguments[j + 1]].column)[row].get<Int64>();
|
||||
Int64 index = (*columns[arguments[j + 1]].column)[row].get<Int64>();
|
||||
if (!moveToElementByIndex<JSONParser>(res_element, index, key))
|
||||
return false;
|
||||
break;
|
||||
}
|
||||
case MoveType::Key:
|
||||
{
|
||||
key = std::string_view{(*block[arguments[j + 1]].column).getDataAt(row)};
|
||||
key = std::string_view{(*columns[arguments[j + 1]].column).getDataAt(row)};
|
||||
if (!moveToElementByKey<JSONParser>(res_element, key))
|
||||
return false;
|
||||
break;
|
||||
@ -286,21 +286,21 @@ public:
|
||||
return Impl<DummyJSONParser>::getReturnType(Name::name, arguments);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) const override
|
||||
{
|
||||
/// Choose JSONParser.
|
||||
#if USE_SIMDJSON
|
||||
if (context.getSettingsRef().allow_simdjson)
|
||||
{
|
||||
FunctionJSONHelpers::Executor<Name, Impl, SimdJSONParser>::run(block, arguments, result_pos, input_rows_count);
|
||||
FunctionJSONHelpers::Executor<Name, Impl, SimdJSONParser>::run(columns, arguments, result_pos, input_rows_count);
|
||||
return;
|
||||
}
|
||||
#endif
|
||||
|
||||
#if USE_RAPIDJSON
|
||||
FunctionJSONHelpers::Executor<Name, Impl, RapidJSONParser>::run(block, arguments, result_pos, input_rows_count);
|
||||
FunctionJSONHelpers::Executor<Name, Impl, RapidJSONParser>::run(columns, arguments, result_pos, input_rows_count);
|
||||
#else
|
||||
FunctionJSONHelpers::Executor<Name, Impl, DummyJSONParser>::run(block, arguments, result_pos, input_rows_count);
|
||||
FunctionJSONHelpers::Executor<Name, Impl, DummyJSONParser>::run(columns, arguments, result_pos, input_rows_count);
|
||||
#endif
|
||||
}
|
||||
|
||||
@ -911,9 +911,9 @@ public:
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 2; }
|
||||
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result_pos)
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result_pos)
|
||||
{
|
||||
extract_tree = JSONExtractTree<JSONParser>::build(function_name, block[result_pos].type);
|
||||
extract_tree = JSONExtractTree<JSONParser>::build(function_name, columns[result_pos].type);
|
||||
}
|
||||
|
||||
bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
|
||||
@ -952,9 +952,9 @@ public:
|
||||
|
||||
static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &, const ColumnNumbers & arguments) { return arguments.size() - 2; }
|
||||
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result_pos)
|
||||
void prepare(const char * function_name, const ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result_pos)
|
||||
{
|
||||
const auto & result_type = block[result_pos].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);
|
||||
|
@ -290,8 +290,8 @@ private:
|
||||
|
||||
|
||||
/// Apply target function by feeding it "batches" of N columns
|
||||
/// Combining 10 columns per pass is the fastest for large block sizes.
|
||||
/// For small block sizes - more columns is faster.
|
||||
/// Combining 10 columns per pass is the fastest for large columns sizes.
|
||||
/// For small columns sizes - more columns is faster.
|
||||
template <
|
||||
typename Op, template <typename, size_t> typename OperationApplierImpl, size_t N = 10>
|
||||
struct OperationApplier
|
||||
@ -512,13 +512,13 @@ DataTypePtr FunctionAnyArityLogical<Impl, Name>::getReturnTypeImpl(const DataTyp
|
||||
|
||||
template <typename Impl, typename Name>
|
||||
void FunctionAnyArityLogical<Impl, Name>::executeImpl(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) const
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) const
|
||||
{
|
||||
ColumnRawPtrs args_in;
|
||||
for (const auto arg_index : arguments)
|
||||
args_in.push_back(block[arg_index].column.get());
|
||||
args_in.push_back(columns[arg_index].column.get());
|
||||
|
||||
auto & result_info = block[result_index];
|
||||
auto & result_info = columns[result_index];
|
||||
if (result_info.type->isNullable())
|
||||
executeForTernaryLogicImpl<Impl>(std::move(args_in), result_info, input_rows_count);
|
||||
else
|
||||
@ -554,9 +554,9 @@ DataTypePtr FunctionUnaryLogical<Impl, Name>::getReturnTypeImpl(const DataTypes
|
||||
}
|
||||
|
||||
template <template <typename> class Impl, typename T>
|
||||
bool functionUnaryExecuteType(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result)
|
||||
bool functionUnaryExecuteType(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (auto col = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
|
||||
if (auto col = checkAndGetColumn<ColumnVector<T>>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
auto col_res = ColumnUInt8::create();
|
||||
|
||||
@ -564,7 +564,7 @@ bool functionUnaryExecuteType(ColumnsWithTypeAndName & block, const ColumnNumber
|
||||
vec_res.resize(col->getData().size());
|
||||
UnaryOperationImpl<T, Impl<T>>::vector(col->getData(), vec_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -572,19 +572,19 @@ bool functionUnaryExecuteType(ColumnsWithTypeAndName & block, const ColumnNumber
|
||||
}
|
||||
|
||||
template <template <typename> class Impl, typename Name>
|
||||
void FunctionUnaryLogical<Impl, Name>::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
void FunctionUnaryLogical<Impl, Name>::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
{
|
||||
if (!(functionUnaryExecuteType<Impl, UInt8>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, UInt16>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, UInt32>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, UInt64>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int8>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int16>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int32>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Int64>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Float32>(block, arguments, result)
|
||||
|| functionUnaryExecuteType<Impl, Float64>(block, arguments, result)))
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
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()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -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 & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_index, 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 & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, 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 & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
DB::Block expr_block;
|
||||
DB::Block expr_columns;
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const auto & argument = block[arguments[i]];
|
||||
const auto & argument = columns[arguments[i]];
|
||||
/// Replace column name with value from argument_names.
|
||||
expr_block.insert({argument.column, argument.type, signature->argument_names[i]});
|
||||
expr_columns.insert({argument.column, argument.type, signature->argument_names[i]});
|
||||
}
|
||||
|
||||
expression_actions->execute(expr_block);
|
||||
expression_actions->execute(expr_columns);
|
||||
|
||||
block[result].column = expr_block.getByName(signature->return_name).column;
|
||||
columns[result].column = expr_columns.getByName(signature->return_name).column;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
@ -119,7 +119,7 @@ public:
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
||||
|
||||
void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
ColumnsWithTypeAndName columns;
|
||||
columns.reserve(arguments.size());
|
||||
@ -140,12 +140,12 @@ public:
|
||||
}
|
||||
|
||||
for (const auto & argument : arguments)
|
||||
columns.push_back(block[argument]);
|
||||
columns.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));
|
||||
block[result].column = ColumnFunction::create(input_rows_count, std::move(function_adaptor), columns);
|
||||
columns[result].column = ColumnFunction::create(input_rows_count, std::move(function_adaptor), columns);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -205,7 +205,7 @@ public:
|
||||
const String & expression_return_name_)
|
||||
: expression_actions(std::move(expression_actions_))
|
||||
{
|
||||
/// Check that expression does not contain unusual actions that will break blocks structure.
|
||||
/// Check that expression does not contain unusual actions that will break columnss structure.
|
||||
for (const auto & action : expression_actions->getActions())
|
||||
if (action.type == ExpressionAction::Type::ARRAY_JOIN)
|
||||
throw Exception("Expression with arrayJoin or other unusual action cannot be captured", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
@ -66,15 +66,15 @@ public:
|
||||
return Impl::getReturnType();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = block[arguments[0]].column;
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
|
||||
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
|
||||
|
||||
const ColumnPtr & num_ptr = block[arguments[1]].column;
|
||||
const ColumnPtr & num_ptr = columns[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 " + block[arguments[1]].column->getName()
|
||||
"Illegal column " + columns[arguments[1]].column->getName()
|
||||
+ ". The number is not const or does not fit in UInt32",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
|
||||
const ColumnPtr & arr_ptr = block[arguments[2]].column;
|
||||
const ColumnPtr & arr_ptr = columns[arguments[2]].column;
|
||||
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
|
||||
|
||||
if (!col_const_arr)
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[2]].column->getName() + ". The array is not const",
|
||||
"Illegal column " + columns[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 " + block[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if constexpr (Impl::is_column_array)
|
||||
block[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
columns[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
else
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -73,20 +73,20 @@ public:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = block[arguments[0]].column;
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
|
||||
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
|
||||
|
||||
const ColumnPtr & arr_ptr = block[arguments[1]].column;
|
||||
const ColumnPtr & arr_ptr = columns[arguments[1]].column;
|
||||
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
|
||||
|
||||
if (!col_const_arr)
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[1]].column->getName() + ". The array is not const",
|
||||
"Illegal column " + columns[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 " + block[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal column " + columns[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;
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
columns[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -75,20 +75,20 @@ public:
|
||||
return Impl::getReturnType();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = block[arguments[0]].column;
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
|
||||
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
|
||||
|
||||
const ColumnPtr & arr_ptr = block[arguments[1]].column;
|
||||
const ColumnPtr & arr_ptr = columns[arguments[1]].column;
|
||||
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
|
||||
|
||||
if (!col_const_arr)
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[1]].column->getName() + ". The array is not const",
|
||||
"Illegal column " + columns[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 " + block[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if constexpr (Impl::is_column_array)
|
||||
block[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
columns[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||
else
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -32,8 +32,8 @@ namespace ErrorCodes
|
||||
* randConstant - service function, produces a constant column with a random value.
|
||||
*
|
||||
* The time is used as the seed.
|
||||
* Note: it is reinitialized for each block.
|
||||
* This means that the timer must be of sufficient resolution to give different values to each block.
|
||||
* Note: it is reinitialized for each columns.
|
||||
* This means that the timer must be of sufficient resolution to give different values to each columns.
|
||||
*/
|
||||
|
||||
DECLARE_MULTITARGET_CODE(
|
||||
@ -74,7 +74,7 @@ public:
|
||||
return std::make_shared<DataTypeNumber<ToType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, 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));
|
||||
|
||||
block[result].column = std::move(col_to);
|
||||
columns[result].column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -102,9 +102,9 @@ public:
|
||||
#endif
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
selector.selectAndExecute(block, arguments, result, input_rows_count);
|
||||
selector.selectAndExecute(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
|
@ -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 & block, const ColumnVector<T> * col, Int64 scale_arg, size_t result)
|
||||
static void apply(ColumnsWithTypeAndName & columns, const ColumnVector<T> * col, Int64 scale_arg, size_t result)
|
||||
{
|
||||
auto col_res = ColumnVector<T>::create();
|
||||
|
||||
@ -484,10 +484,10 @@ class Dispatcher
|
||||
}
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
|
||||
static void apply(ColumnsWithTypeAndName & block, const ColumnDecimal<T> * col, Int64 scale_arg, size_t result)
|
||||
static void apply(ColumnsWithTypeAndName & columns, const ColumnDecimal<T> * col, Int64 scale_arg, size_t result)
|
||||
{
|
||||
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);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
|
||||
public:
|
||||
static void apply(ColumnsWithTypeAndName & block, const IColumn * column, Int64 scale_arg, size_t result)
|
||||
static void apply(ColumnsWithTypeAndName & columns, const IColumn * column, Int64 scale_arg, size_t result)
|
||||
{
|
||||
if constexpr (IsNumber<T>)
|
||||
apply(block, checkAndGetColumn<ColumnVector<T>>(column), scale_arg, result);
|
||||
apply(columns, checkAndGetColumn<ColumnVector<T>>(column), scale_arg, result);
|
||||
else if constexpr (IsDecimalNumber<T>)
|
||||
apply(block, checkAndGetColumn<ColumnDecimal<T>>(column), scale_arg, result);
|
||||
apply(columns, checkAndGetColumn<ColumnDecimal<T>>(column), scale_arg, result);
|
||||
}
|
||||
};
|
||||
|
||||
@ -545,11 +545,11 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
static Int64 getScaleArg(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments)
|
||||
static Int64 getScaleArg(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const IColumn & scale_column = *block[arguments[1]].column;
|
||||
const IColumn & scale_column = *columns[arguments[1]].column;
|
||||
if (!isColumnConst(scale_column))
|
||||
throw Exception("Scale argument for rounding functions must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -566,10 +566,10 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnWithTypeAndName & column = block[arguments[0]];
|
||||
Int64 scale_arg = getScaleArg(block, arguments);
|
||||
const ColumnWithTypeAndName & column = columns[arguments[0]];
|
||||
Int64 scale_arg = getScaleArg(columns, arguments);
|
||||
|
||||
auto call = [&](const auto & types) -> bool
|
||||
{
|
||||
@ -579,7 +579,7 @@ public:
|
||||
if constexpr (IsDataTypeNumber<DataType> || IsDataTypeDecimal<DataType>)
|
||||
{
|
||||
using FieldType = typename DataType::FieldType;
|
||||
Dispatcher<FieldType, rounding_mode, tie_breaking_mode>::apply(block, column.column.get(), scale_arg, result);
|
||||
Dispatcher<FieldType, rounding_mode, tie_breaking_mode>::apply(columns, column.column.get(), scale_arg, result);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
@ -646,23 +646,23 @@ public:
|
||||
return getLeastSupertype({type_x, type_arr_nested});
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
{
|
||||
auto in_column = block[arguments[0]].column;
|
||||
const auto & in_type = block[arguments[0]].type;
|
||||
auto in_column = columns[arguments[0]].column;
|
||||
const auto & in_type = columns[arguments[0]].type;
|
||||
|
||||
auto array_column = block[arguments[1]].column;
|
||||
const auto & array_type = block[arguments[1]].type;
|
||||
auto array_column = columns[arguments[1]].column;
|
||||
const auto & array_type = columns[arguments[1]].type;
|
||||
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
auto column_result = return_type->createColumn();
|
||||
auto out = column_result.get();
|
||||
|
||||
if (!in_type->equals(*return_type))
|
||||
in_column = castColumn(block[arguments[0]], return_type);
|
||||
in_column = castColumn(columns[arguments[0]], return_type);
|
||||
|
||||
if (!array_type->equals(*return_type))
|
||||
array_column = castColumn(block[arguments[1]], std::make_shared<DataTypeArray>(return_type));
|
||||
array_column = castColumn(columns[arguments[1]], std::make_shared<DataTypeArray>(return_type));
|
||||
|
||||
const auto in = in_column.get();
|
||||
auto boundaries = typeid_cast<const ColumnConst &>(*array_column).getValue<Array>();
|
||||
@ -687,7 +687,7 @@ public:
|
||||
throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
block[result].column = std::move(column_result);
|
||||
columns[result].column = std::move(column_result);
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -75,7 +75,7 @@ public:
|
||||
}
|
||||
|
||||
/// Initialize by the function arguments.
|
||||
void init(ColumnsWithTypeAndName & /*block*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*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 & block, const ColumnNumbers & arguments)
|
||||
void init(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
{
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block[arguments[0]].column.get());
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[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 & block, const ColumnNumbers & arguments)
|
||||
void init(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
{
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block[arguments[0]].column.get());
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[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 & block, const ColumnNumbers & arguments)
|
||||
void init(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments)
|
||||
{
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block[arguments[1]].column.get());
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(columns[arguments[1]].column.get());
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + block[arguments[1]].column->getName()
|
||||
throw Exception("Illegal column " + columns[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 & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
Generator generator;
|
||||
generator.init(block, arguments);
|
||||
generator.init(columns, arguments);
|
||||
size_t array_argument_position = arguments[generator.getStringsArgumentPosition()];
|
||||
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(block[array_argument_position].column.get());
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(columns[array_argument_position].column.get());
|
||||
const ColumnConst * col_const_str =
|
||||
checkAndGetColumnConstStringOrFixedString(block[array_argument_position].column.get());
|
||||
checkAndGetColumnConstStringOrFixedString(columns[array_argument_position].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);
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(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));
|
||||
|
||||
block[result].column = block[result].type->createColumnConst(col_const_str->size(), dst);
|
||||
columns[result].column = columns[result].type->createColumnConst(col_const_str->size(), dst);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal columns " + block[array_argument_position].column->getName()
|
||||
+ ", " + block[array_argument_position].column->getName()
|
||||
throw Exception("Illegal columns " + columns[array_argument_position].column->getName()
|
||||
+ ", " + columns[array_argument_position].column->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
@ -536,19 +536,19 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
String delimiter;
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(block[arguments[1]].column.get());
|
||||
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(columns[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>(block[arguments[0]].column.get()))
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(columns[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 &>();
|
||||
}
|
||||
|
||||
block[result].column = block[result].type->createColumnConst(col_const_arr->size(), dst_str);
|
||||
columns[result].column = columns[result].type->createColumnConst(col_const_arr->size(), dst_str);
|
||||
}
|
||||
else
|
||||
{
|
||||
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*block[arguments[0]].column);
|
||||
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*columns[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());
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -97,16 +97,16 @@ public:
|
||||
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = block[arguments[0]].column;
|
||||
const ColumnPtr & column_needle = block[arguments[1]].column;
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
const ColumnPtr & column_needle = columns[arguments[1]].column;
|
||||
|
||||
ColumnPtr column_start_pos = nullptr;
|
||||
if (arguments.size() >= 3)
|
||||
column_start_pos = block[arguments[2]].column;
|
||||
column_start_pos = columns[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,10 +127,10 @@ public:
|
||||
vec_res);
|
||||
|
||||
if (is_col_start_pos_const)
|
||||
block[result].column
|
||||
= block[result].type->createColumnConst(col_haystack_const->size(), toField(vec_res[0]));
|
||||
columns[result].column
|
||||
= columns[result].type->createColumnConst(col_haystack_const->size(), toField(vec_res[0]));
|
||||
else
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
|
||||
return;
|
||||
}
|
||||
@ -175,11 +175,11 @@ public:
|
||||
vec_res);
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal columns " + block[arguments[0]].column->getName() + " and "
|
||||
+ block[arguments[1]].column->getName() + " of arguments of function " + getName(),
|
||||
"Illegal columns " + columns[arguments[0]].column->getName() + " and "
|
||||
+ columns[arguments[1]].column->getName() + " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -58,10 +58,10 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column = block[arguments[0]].column;
|
||||
const ColumnPtr column_needle = block[arguments[1]].column;
|
||||
const ColumnPtr column = columns[arguments[0]].column;
|
||||
const ColumnPtr column_needle = columns[arguments[1]].column;
|
||||
|
||||
const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle);
|
||||
if (!col_needle)
|
||||
@ -75,11 +75,11 @@ public:
|
||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||
Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
@ -51,12 +51,12 @@ public:
|
||||
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
using ResultType = typename Impl::ResultType;
|
||||
|
||||
const ColumnPtr & column_haystack = block[arguments[0]].column;
|
||||
const ColumnPtr & column_needle = block[arguments[1]].column;
|
||||
const ColumnPtr & column_haystack = columns[arguments[0]].column;
|
||||
const ColumnPtr & column_needle = columns[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,8 +73,8 @@ public:
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
}
|
||||
Impl::constantConstant(col_haystack_const->getValue<String>(), needle, res);
|
||||
block[result].column
|
||||
= block[result].type->createColumnConst(col_haystack_const->size(), toField(res));
|
||||
columns[result].column
|
||||
= columns[result].type->createColumnConst(col_haystack_const->size(), toField(res));
|
||||
return;
|
||||
}
|
||||
|
||||
@ -122,12 +122,12 @@ public:
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal columns " + block[arguments[0]].column->getName() + " and "
|
||||
+ block[arguments[1]].column->getName() + " of arguments of function " + getName(),
|
||||
"Illegal columns " + columns[arguments[0]].column->getName() + " and "
|
||||
+ columns[arguments[1]].column->getName() + " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -526,10 +526,10 @@ DataTypePtr FunctionOverloadResolverAdaptor::getReturnTypeWithoutLowCardinality(
|
||||
}
|
||||
if (null_presence.has_nullable)
|
||||
{
|
||||
Block nested_block = createBlockWithNestedColumns(
|
||||
Block nested_columns = createBlockWithNestedColumns(
|
||||
arguments,
|
||||
ext::collection_cast<ColumnNumbers>(ext::range(0, arguments.size())));
|
||||
auto return_type = impl->getReturnType(ColumnsWithTypeAndName(nested_block.begin(), nested_block.end()));
|
||||
auto return_type = impl->getReturnType(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end()));
|
||||
return makeNullable(return_type);
|
||||
}
|
||||
}
|
||||
@ -586,14 +586,14 @@ llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes
|
||||
values[i] = [value = b.CreateExtractValue(value, {0})]() { return value; };
|
||||
}
|
||||
auto * result = b.CreateInsertValue(zero, compileImpl(builder, *denulled, std::move(values)), {0});
|
||||
auto * result_block = b.GetInsertBlock();
|
||||
auto * result_columns = b.GetInsertBlock();
|
||||
b.CreateBr(join);
|
||||
b.SetInsertPoint(fail);
|
||||
auto * null = b.CreateInsertValue(zero, b.getTrue(), {1});
|
||||
b.CreateBr(join);
|
||||
b.SetInsertPoint(join);
|
||||
auto * phi = b.CreatePHI(result->getType(), 2);
|
||||
phi->addIncoming(result, result_block);
|
||||
phi->addIncoming(result, result_columns);
|
||||
phi->addIncoming(null, fail);
|
||||
return phi;
|
||||
}
|
||||
|
@ -34,7 +34,7 @@ class Field;
|
||||
|
||||
/// The simplest executable object.
|
||||
/// Motivation:
|
||||
/// * Prepare something heavy once before main execution loop instead of doing it for each block.
|
||||
/// * Prepare something heavy once before main execution loop instead of doing it for each columns.
|
||||
/// * Provide const interface for IFunctionBase (later).
|
||||
/// * Create one executable function per thread to use caches without synchronization (later).
|
||||
class IExecutableFunction
|
||||
@ -70,8 +70,8 @@ public:
|
||||
virtual const DataTypePtr & getReturnType() const = 0;
|
||||
|
||||
/// Do preparations and return executable.
|
||||
/// sample_block should contain data types of arguments and values of constants, if relevant.
|
||||
virtual ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & sample_block, const ColumnNumbers & arguments, size_t result) const = 0;
|
||||
/// sample_columns should contain data types of arguments and values of constants, if relevant.
|
||||
virtual ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & sample_columns, const ColumnNumbers & arguments, size_t result) const = 0;
|
||||
|
||||
/// TODO: make const
|
||||
virtual void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run = false)
|
||||
@ -111,7 +111,7 @@ public:
|
||||
* There is no need to implement function if it has zero arguments.
|
||||
* Must return ColumnConst with single row or nullptr.
|
||||
*/
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*block*/, const ColumnNumbers & /*arguments*/) const { return nullptr; }
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) const { return nullptr; }
|
||||
|
||||
/** Function is called "injective" if it returns different result for different values of arguments.
|
||||
* Example: hex, negate, tuple...
|
||||
@ -133,13 +133,13 @@ public:
|
||||
* as it returns 'nan' for many different representation of NaNs.
|
||||
* But we assume, that it is injective. This could be documented as implementation-specific behaviour.
|
||||
*
|
||||
* sample_block should contain data types of arguments and values of constants, if relevant.
|
||||
* sample_columns should contain data types of arguments and values of constants, if relevant.
|
||||
* NOTE: to check is function injective with any arguments, you can pass
|
||||
* empty block as sample_block (since most of the time function will
|
||||
* empty columns as sample_columns (since most of the time function will
|
||||
* ignore it anyway, and creating arguments just for checking is
|
||||
* function injective or not is overkill).
|
||||
*/
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_block*/) const { return false; }
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
|
||||
|
||||
/** Function is called "deterministic", if it returns same result for same values of arguments.
|
||||
* Most of functions are deterministic. Notable counterexample is rand().
|
||||
@ -226,7 +226,7 @@ public:
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
|
||||
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in blocks.
|
||||
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in columnss.
|
||||
* Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL.
|
||||
*/
|
||||
ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count);
|
||||
|
@ -44,9 +44,9 @@ public:
|
||||
const DataTypes & getArgumentTypes() const final { return impl->getArgumentTypes(); }
|
||||
const DataTypePtr & getReturnType() const final { return impl->getReturnType(); }
|
||||
|
||||
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & sample_block, const ColumnNumbers & arguments, size_t result) const final
|
||||
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & sample_columns, const ColumnNumbers & arguments, size_t result) const final
|
||||
{
|
||||
ColumnsWithTypeAndName columns(const_cast<ColumnsWithTypeAndName &>(sample_block));
|
||||
ColumnsWithTypeAndName columns(const_cast<ColumnsWithTypeAndName &>(sample_columns));
|
||||
return std::make_shared<ExecutableFunctionAdaptor>(impl->prepare(columns, arguments, result));
|
||||
}
|
||||
|
||||
@ -69,7 +69,7 @@ public:
|
||||
return impl->getResultIfAlwaysReturnsConstantAndHasArguments(columns, arguments);
|
||||
}
|
||||
|
||||
bool isInjective(const ColumnsWithTypeAndName & sample_block) const final { return impl->isInjective(sample_block); }
|
||||
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const final { return impl->isInjective(sample_columns); }
|
||||
bool isDeterministic() const final { return impl->isDeterministic(); }
|
||||
bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); }
|
||||
bool hasInformationAboutMonotonicity() const final { return impl->hasInformationAboutMonotonicity(); }
|
||||
@ -148,13 +148,13 @@ public:
|
||||
String getName() const override { return function->getName(); }
|
||||
|
||||
protected:
|
||||
void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
{
|
||||
return function->executeImpl(block, arguments, result, input_rows_count);
|
||||
return function->executeImpl(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
void executeDryRun(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
void executeDryRun(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
{
|
||||
return function->executeImplDryRun(block, arguments, result, input_rows_count);
|
||||
return function->executeImplDryRun(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
|
||||
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
|
||||
@ -185,7 +185,7 @@ public:
|
||||
|
||||
#endif
|
||||
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & /*sample_columns*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
||||
{
|
||||
return std::make_unique<DefaultExecutable>(function);
|
||||
}
|
||||
@ -198,7 +198,7 @@ public:
|
||||
|
||||
bool isStateful() const override { return function->isStateful(); }
|
||||
|
||||
bool isInjective(const ColumnsWithTypeAndName & sample_block) const override { return function->isInjective(sample_block); }
|
||||
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override { return function->isInjective(sample_columns); }
|
||||
|
||||
bool isDeterministic() const override { return function->isDeterministic(); }
|
||||
|
||||
|
@ -35,15 +35,15 @@ public:
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual void execute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
|
||||
virtual void executeDryRun(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
virtual void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
|
||||
virtual void executeDryRun(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
execute(block, arguments, result, input_rows_count);
|
||||
execute(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
|
||||
* if some of arguments are NULL constants then return NULL constant,
|
||||
* if some of arguments are Nullable, then execute function as usual for block,
|
||||
* if some of arguments are Nullable, then execute function as usual for columns,
|
||||
* where Nullable columns are substituted with nested columns (they have arbitrary values in rows corresponding to NULL value)
|
||||
* and wrap result in Nullable column where NULLs are in all rows where any of arguments are NULL.
|
||||
*/
|
||||
@ -89,7 +89,7 @@ public:
|
||||
virtual const DataTypes & getArgumentTypes() const = 0;
|
||||
virtual const DataTypePtr & getReturnType() const = 0;
|
||||
|
||||
virtual ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & sample_block, const ColumnNumbers & arguments, size_t result) const = 0;
|
||||
virtual ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & sample_columns, const ColumnNumbers & arguments, size_t result) const = 0;
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
@ -107,7 +107,7 @@ public:
|
||||
virtual bool isSuitableForConstantFolding() const { return true; }
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) const { return nullptr; }
|
||||
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_block*/) const { return false; }
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
|
||||
virtual bool isDeterministic() const { return true; }
|
||||
virtual bool isDeterministicInScopeOfQuery() const { return true; }
|
||||
virtual bool hasInformationAboutMonotonicity() const { return false; }
|
||||
@ -197,15 +197,15 @@ public:
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const = 0;
|
||||
virtual void executeImplDryRun(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
virtual void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const = 0;
|
||||
virtual void executeImplDryRun(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
executeImpl(block, arguments, result, input_rows_count);
|
||||
executeImpl(columns, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
|
||||
* if some of arguments are NULL constants then return NULL constant,
|
||||
* if some of arguments are Nullable, then execute function as usual for block,
|
||||
* if some of arguments are Nullable, then execute function as usual for columns,
|
||||
* where Nullable columns are substituted with nested columns (they have arbitrary values in rows corresponding to NULL value)
|
||||
* and wrap result in Nullable column where NULLs are in all rows where any of arguments are NULL.
|
||||
*/
|
||||
@ -253,7 +253,7 @@ public:
|
||||
/// Properties from IFunctionBase (see IFunction.h)
|
||||
virtual bool isSuitableForConstantFolding() const { return true; }
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) const { return nullptr; }
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_block*/) const { return false; }
|
||||
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
|
||||
virtual bool isDeterministic() const { return true; }
|
||||
virtual bool isDeterministicInScopeOfQuery() const { return true; }
|
||||
virtual bool isStateful() const { return false; }
|
||||
|
@ -46,20 +46,20 @@ private:
|
||||
return getLeastSupertype(types);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
size_t num_arguments = arguments.size();
|
||||
if (1 == num_arguments)
|
||||
{
|
||||
block[result].column = block[arguments[0]].column;
|
||||
columns[result].column = columns[arguments[0]].column;
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_type = block[result].type;
|
||||
auto result_type = columns[result].type;
|
||||
|
||||
Columns converted_columns(num_arguments);
|
||||
for (size_t arg = 0; arg < num_arguments; ++arg)
|
||||
converted_columns[arg] = castColumn(block[arguments[arg]], result_type)->convertToFullColumnIfConst();
|
||||
converted_columns[arg] = castColumn(columns[arguments[arg]], result_type)->convertToFullColumnIfConst();
|
||||
|
||||
auto result_column = result_type->createColumn();
|
||||
result_column->reserve(input_rows_count);
|
||||
@ -86,7 +86,7 @@ private:
|
||||
result_column->insertFrom(*converted_columns[best_arg], row_num);
|
||||
}
|
||||
|
||||
block[result].column = std::move(result_column);
|
||||
columns[result].column = std::move(result_column);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -70,7 +70,7 @@ static inline bool likePatternIsStrstr(const String & pattern, String & res)
|
||||
}
|
||||
|
||||
/** 'like' - if true, treat pattern as SQL LIKE or ILIKE; if false - treat pattern as re2 regexp.
|
||||
* NOTE: We want to run regexp search for whole block by one call (as implemented in function 'position')
|
||||
* NOTE: We want to run regexp search for whole columns by one call (as implemented in function 'position')
|
||||
* but for that, regexp engine must support \0 bytes and their interpretation as string boundaries.
|
||||
*/
|
||||
template <bool like, bool revert = false, bool case_insensitive = false>
|
||||
|
@ -204,22 +204,22 @@ public:
|
||||
* If FunctionInterface is IFunction, then "executeImpl" method of the implementation will be called
|
||||
* and "execute" otherwise.
|
||||
*/
|
||||
void selectAndExecute(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void selectAndExecute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
if (implementations.empty())
|
||||
throw Exception("There are no available implementations for function " "TODO(dakovalkov): add name",
|
||||
ErrorCodes::NO_SUITABLE_FUNCTION_IMPLEMENTATION);
|
||||
|
||||
/// Statistics shouldn't rely on small blocks.
|
||||
/// Statistics shouldn't rely on small columnss.
|
||||
bool considerable = (input_rows_count > 1000);
|
||||
|
||||
size_t id = statistics.select(considerable);
|
||||
Stopwatch watch;
|
||||
|
||||
if constexpr (std::is_same_v<FunctionInterface, IFunction>)
|
||||
implementations[id]->executeImpl(block, arguments, result, input_rows_count);
|
||||
implementations[id]->executeImpl(columns, arguments, result, input_rows_count);
|
||||
else
|
||||
implementations[id]->execute(block, arguments, result, input_rows_count);
|
||||
implementations[id]->execute(columns, arguments, result, input_rows_count);
|
||||
|
||||
watch.stop();
|
||||
|
||||
|
@ -169,7 +169,7 @@ struct ReplaceRegexpImpl
|
||||
|
||||
Instructions instructions = createInstructions(replacement, num_captures);
|
||||
|
||||
/// Cannot perform search for whole block. Will process each string separately.
|
||||
/// Cannot perform search for whole columns. Will process each string separately.
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
int from = i > 0 ? offsets[i - 1] : 0;
|
||||
|
@ -29,7 +29,7 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void init(ColumnsWithTypeAndName & /*block*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*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 & /*block*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*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 & /*block*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*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 & /*block*/, const ColumnNumbers & /*arguments*/) {}
|
||||
void init(ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) {}
|
||||
|
||||
/// Returns the position of the argument that is the column of rows
|
||||
static size_t getStringsArgumentPosition()
|
||||
|
@ -48,18 +48,18 @@ struct FunctionPort : public IFunction
|
||||
}
|
||||
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
{
|
||||
UInt16 default_port = 0;
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const auto * port_column = checkAndGetColumn<ColumnConst>(block[arguments[1]].column.get());
|
||||
const auto * port_column = checkAndGetColumn<ColumnConst>(columns[arguments[1]].column.get());
|
||||
if (!port_column)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant UInt16", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
default_port = port_column->getValue<UInt16>();
|
||||
}
|
||||
|
||||
const ColumnPtr url_column = block[arguments[0]].column;
|
||||
const ColumnPtr url_column = columns[arguments[0]].column;
|
||||
if (const ColumnString * url_strs = checkAndGetColumn<ColumnString>(url_column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<UInt16>::create();
|
||||
@ -67,11 +67,11 @@ struct FunctionPort : public IFunction
|
||||
vec_res.resize(url_column->size());
|
||||
|
||||
vector(default_port, url_strs->getChars(), url_strs->getOffsets(), vec_res);
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
|
@ -48875,7 +48875,7 @@ const char * TopLevelDomainLookupHash::isValid(const char * str, size_t len)
|
||||
"",
|
||||
"",
|
||||
"",
|
||||
"myshopblocks.com",
|
||||
"myshopcolumnss.com",
|
||||
"blogspot.bj",
|
||||
"",
|
||||
"",
|
||||
|
@ -4834,7 +4834,7 @@ biz.ua
|
||||
co.ua
|
||||
pp.ua
|
||||
shiftedit.io
|
||||
myshopblocks.com
|
||||
myshopcolumnss.com
|
||||
mo-siemens.io
|
||||
1kapp.com
|
||||
appchizi.com
|
||||
|
@ -216,11 +216,11 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
if (input_rows_count == 0)
|
||||
{
|
||||
block[result].column = ColumnString::create();
|
||||
columns[result].column = ColumnString::create();
|
||||
return;
|
||||
}
|
||||
|
||||
@ -229,7 +229,7 @@ public:
|
||||
String dist;
|
||||
bool higher_is_better;
|
||||
|
||||
if (const ColumnConst * col_dist = checkAndGetColumnConst<ColumnString>(block[arguments[0]].column.get()))
|
||||
if (const ColumnConst * col_dist = checkAndGetColumnConst<ColumnString>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
dist = col_dist->getDataAt(0).data;
|
||||
dist = Poco::toLower(dist);
|
||||
@ -239,12 +239,12 @@ public:
|
||||
else
|
||||
throw Exception("First argument for function " + getName() + " must be Constant string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (const ColumnConst * col_higher_is_better = checkAndGetColumnConst<ColumnUInt8>(block[arguments[1]].column.get()))
|
||||
if (const ColumnConst * col_higher_is_better = checkAndGetColumnConst<ColumnUInt8>(columns[arguments[1]].column.get()))
|
||||
higher_is_better = col_higher_is_better->getBool(0);
|
||||
else
|
||||
throw Exception("Second argument for function " + getName() + " must be Constatnt boolean", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(block[arguments[2]].column.get()))
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(columns[arguments[2]].column.get()))
|
||||
{
|
||||
if (!col_const_arr)
|
||||
throw Exception("Third argument for function " + getName() + " must be Array of constant strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -259,7 +259,7 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(block[arguments[3]].column.get()))
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(columns[arguments[3]].column.get()))
|
||||
{
|
||||
if (!col_const_arr)
|
||||
throw Exception("Forth argument for function " + getName() + " must be Array of constant numbers", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -267,7 +267,7 @@ public:
|
||||
throw Exception("Forth and fifth Argument for function " + getName() + " must be Array of constant Numbers", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(block[arguments[4]].column.get()))
|
||||
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(columns[arguments[4]].column.get()))
|
||||
{
|
||||
if (!col_const_arr)
|
||||
throw Exception("Fifth argument for function " + getName() + " must be Array of constant numbers", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -294,7 +294,7 @@ public:
|
||||
auto dst = ColumnString::create();
|
||||
std::string result_str = convertToJson(variant_names, variants);
|
||||
dst->insertData(result_str.c_str(), result_str.length());
|
||||
block[result].column = std::move(dst);
|
||||
columns[result].column = std::move(dst);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -72,9 +72,9 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
const ColumnUInt64 * column_concrete = checkAndGetColumn<ColumnUInt64>(column.get());
|
||||
|
||||
if (!column_concrete)
|
||||
@ -89,7 +89,7 @@ public:
|
||||
result_column->insertData(res_str.data, res_str.size);
|
||||
}
|
||||
|
||||
block[result].column = std::move(result_column);
|
||||
columns[result].column = std::move(result_column);
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -64,11 +64,11 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const SymbolIndex & symbol_index = SymbolIndex::instance();
|
||||
|
||||
const ColumnPtr & column = block[arguments[0]].column;
|
||||
const ColumnPtr & column = columns[arguments[0]].column;
|
||||
const ColumnUInt64 * column_concrete = checkAndGetColumn<ColumnUInt64>(column.get());
|
||||
|
||||
if (!column_concrete)
|
||||
@ -85,7 +85,7 @@ public:
|
||||
result_column->insertDefault();
|
||||
}
|
||||
|
||||
block[result].column = std::move(result_column);
|
||||
columns[result].column = std::move(result_column);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -55,10 +55,10 @@ private:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto & column = block[arguments[0]].column;
|
||||
const auto & column_char = block[arguments[1]].column;
|
||||
const auto & column = columns[arguments[0]].column;
|
||||
const auto & column_char = columns[arguments[1]].column;
|
||||
|
||||
if (!checkColumnConst<ColumnString>(column_char.get()))
|
||||
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
|
||||
@ -103,10 +103,10 @@ private:
|
||||
}
|
||||
|
||||
dst_data.resize_assume_reserved(dst_offset);
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
throw Exception{"Illegal column " + columns[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
};
|
||||
|
@ -135,11 +135,11 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
ColumnPtr column_array_ptr = block[arguments[0]].column;
|
||||
ColumnPtr column_array_ptr = columns[arguments[0]].column;
|
||||
const auto * column_array = checkAndGetColumn<ColumnArray>(column_array_ptr.get());
|
||||
|
||||
if (!column_array)
|
||||
@ -151,11 +151,11 @@ public:
|
||||
column_array = assert_cast<const ColumnArray *>(column_array_ptr.get());
|
||||
}
|
||||
|
||||
block[result].column = Impl::execute(*column_array, column_array->getDataPtr());
|
||||
columns[result].column = Impl::execute(*column_array, column_array->getDataPtr());
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & column_with_type_and_name = block[arguments[0]];
|
||||
const auto & column_with_type_and_name = columns[arguments[0]];
|
||||
|
||||
if (!column_with_type_and_name.column)
|
||||
throw Exception("First argument for function " + getName() + " must be a function.",
|
||||
@ -177,7 +177,7 @@ public:
|
||||
|
||||
for (size_t i = 1; i < arguments.size(); ++i)
|
||||
{
|
||||
const auto & array_with_type_and_name = block[arguments[i]];
|
||||
const auto & array_with_type_and_name = columns[arguments[i]];
|
||||
|
||||
ColumnPtr column_array_ptr = array_with_type_and_name.column;
|
||||
const auto * column_array = checkAndGetColumn<ColumnArray>(column_array_ptr.get());
|
||||
@ -220,7 +220,7 @@ public:
|
||||
array_with_type_and_name.name));
|
||||
}
|
||||
|
||||
/// Put all the necessary columns multiplied by the sizes of arrays into the block.
|
||||
/// Put all the necessary columns multiplied by the sizes of arrays into the columns.
|
||||
auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_array->getOffsets()));
|
||||
auto * replicated_column_function = typeid_cast<ColumnFunction *>(replicated_column_function_ptr.get());
|
||||
replicated_column_function->appendArguments(arrays);
|
||||
@ -229,7 +229,7 @@ public:
|
||||
if (lambda_result->lowCardinality())
|
||||
lambda_result = lambda_result->convertToFullColumnIfLowCardinality();
|
||||
|
||||
block[result].column = Impl::execute(*column_first_array, lambda_result);
|
||||
columns[result].column = Impl::execute(*column_first_array, lambda_result);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -30,21 +30,21 @@ public:
|
||||
return std::make_shared<DataTypeArray>(getLeastSupertype(arguments));
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
size_t num_elements = arguments.size();
|
||||
|
||||
if (num_elements == 0)
|
||||
{
|
||||
/// We should return constant empty array.
|
||||
block[result].column = block[result].type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
columns[result].column = columns[result].type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
const DataTypePtr & return_type = block[result].type;
|
||||
const DataTypePtr & return_type = columns[result].type;
|
||||
const DataTypePtr & elem_type = static_cast<const DataTypeArray &>(*return_type).getNestedType();
|
||||
|
||||
size_t block_size = input_rows_count;
|
||||
size_t columns_size = input_rows_count;
|
||||
|
||||
/** If part of columns have not same type as common type of all elements of array,
|
||||
* then convert them to common type.
|
||||
@ -57,7 +57,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < num_elements; ++i)
|
||||
{
|
||||
const auto & arg = block[arguments[i]];
|
||||
const auto & arg = columns[arguments[i]];
|
||||
|
||||
ColumnPtr preprocessed_column = arg.column;
|
||||
|
||||
@ -76,11 +76,11 @@ public:
|
||||
IColumn & out_data = out->getData();
|
||||
IColumn::Offsets & out_offsets = out->getOffsets();
|
||||
|
||||
out_data.reserve(block_size * num_elements);
|
||||
out_offsets.resize(block_size);
|
||||
out_data.reserve(columns_size * num_elements);
|
||||
out_offsets.resize(columns_size);
|
||||
|
||||
IColumn::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < block_size; ++i)
|
||||
for (size_t i = 0; i < columns_size; ++i)
|
||||
{
|
||||
for (size_t j = 0; j < num_elements; ++j)
|
||||
out_data.insertFrom(*columns[j], i);
|
||||
@ -89,7 +89,7 @@ public:
|
||||
out_offsets[i] = current_offset;
|
||||
}
|
||||
|
||||
block[result].column = std::move(out);
|
||||
columns[result].column = std::move(out);
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -49,13 +49,13 @@ public:
|
||||
return getLeastSupertype(arguments);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const DataTypePtr & return_type = block[result].type;
|
||||
const DataTypePtr & return_type = columns[result].type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
columns[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -66,7 +66,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & arg = block[arguments[i]];
|
||||
const ColumnWithTypeAndName & arg = columns[arguments[i]];
|
||||
ColumnPtr preprocessed_column = arg.column;
|
||||
|
||||
if (!arg.type->equals(*return_type))
|
||||
@ -95,7 +95,7 @@ public:
|
||||
|
||||
auto sink = GatherUtils::concat(sources);
|
||||
|
||||
block[result].column = std::move(sink);
|
||||
columns[result].column = std::move(sink);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
return std::make_shared<DataTypeArray>(nested_type);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
@ -85,12 +85,12 @@ private:
|
||||
};
|
||||
|
||||
|
||||
void FunctionArrayDistinct::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
void FunctionArrayDistinct::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
{
|
||||
ColumnPtr array_ptr = block[arguments[0]].column;
|
||||
ColumnPtr array_ptr = columns[arguments[0]].column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
|
||||
auto res_ptr = return_type->createColumn();
|
||||
ColumnArray & res = assert_cast<ColumnArray &>(*res_ptr);
|
||||
@ -127,7 +127,7 @@ void FunctionArrayDistinct::executeImpl(ColumnsWithTypeAndName & block, const Co
|
||||
|| executeString(*inner_col, offsets, res_data, res_offsets, nullable_col)))
|
||||
executeHashed(*inner_col, offsets, res_data, res_offsets, nullable_col);
|
||||
|
||||
block[result].column = std::move(res_ptr);
|
||||
columns[result].column = std::move(res_ptr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -46,46 +46,46 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
void perform(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result,
|
||||
void perform(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) const;
|
||||
|
||||
template <typename DataType>
|
||||
static bool executeNumberConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
static bool executeNumberConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType, typename DataType>
|
||||
static bool executeNumber(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
static bool executeNumber(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
static bool executeStringConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
static bool executeStringConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
static bool executeString(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
static bool executeString(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
static bool executeGenericConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
static bool executeGenericConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
static bool executeGeneric(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
static bool executeGeneric(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
static bool executeConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result,
|
||||
static bool executeConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result,
|
||||
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
|
||||
size_t input_rows_count);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeArgument(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result,
|
||||
bool executeArgument(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) const;
|
||||
|
||||
/** For a tuple array, the function is evaluated component-wise for each element of the tuple.
|
||||
*/
|
||||
bool executeTuple(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const;
|
||||
bool executeTuple(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const;
|
||||
};
|
||||
|
||||
|
||||
@ -426,10 +426,10 @@ FunctionPtr FunctionArrayElement::create(const Context &)
|
||||
|
||||
|
||||
template <typename DataType>
|
||||
bool FunctionArrayElement::executeNumberConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
bool FunctionArrayElement::executeNumberConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -450,15 +450,15 @@ bool FunctionArrayElement::executeNumberConst(ColumnsWithTypeAndName & block, co
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType, typename DataType>
|
||||
bool FunctionArrayElement::executeNumber(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
bool FunctionArrayElement::executeNumber(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -473,14 +473,14 @@ bool FunctionArrayElement::executeNumber(ColumnsWithTypeAndName & block, const C
|
||||
ArrayElementNumImpl<DataType>::template vector<IndexType>(
|
||||
col_nested->getData(), col_array->getOffsets(), indices, col_res->getData(), builder);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::executeStringConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
bool FunctionArrayElement::executeStringConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -513,15 +513,15 @@ bool FunctionArrayElement::executeStringConst(ColumnsWithTypeAndName & block, co
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeString(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
bool FunctionArrayElement::executeString(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -542,14 +542,14 @@ bool FunctionArrayElement::executeString(ColumnsWithTypeAndName & block, const C
|
||||
col_res->getOffsets(),
|
||||
builder);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::executeGenericConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
bool FunctionArrayElement::executeGenericConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -566,15 +566,15 @@ bool FunctionArrayElement::executeGenericConst(ColumnsWithTypeAndName & block, c
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeGeneric(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
bool FunctionArrayElement::executeGeneric(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -585,21 +585,21 @@ bool FunctionArrayElement::executeGeneric(ColumnsWithTypeAndName & block, const
|
||||
ArrayElementGenericImpl::vector<IndexType>(
|
||||
col_nested, col_array->getOffsets(), indices, *col_res, builder);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result,
|
||||
bool FunctionArrayElement::executeConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result,
|
||||
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
|
||||
size_t input_rows_count)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumnConstData<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = checkAndGetColumnConstData<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
auto res = block[result].type->createColumn();
|
||||
auto res = columns[result].type->createColumn();
|
||||
|
||||
size_t rows = input_rows_count;
|
||||
const IColumn & array_elements = col_array->getData();
|
||||
@ -630,15 +630,15 @@ bool FunctionArrayElement::executeConst(ColumnsWithTypeAndName & block, const Co
|
||||
}
|
||||
}
|
||||
|
||||
block[result].column = std::move(res);
|
||||
columns[result].column = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeArgument(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result,
|
||||
bool FunctionArrayElement::executeArgument(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) const
|
||||
{
|
||||
auto index = checkAndGetColumn<ColumnVector<IndexType>>(block[arguments[1]].column.get());
|
||||
auto index = checkAndGetColumn<ColumnVector<IndexType>>(columns[arguments[1]].column.get());
|
||||
|
||||
if (!index)
|
||||
return false;
|
||||
@ -648,28 +648,28 @@ bool FunctionArrayElement::executeArgument(ColumnsWithTypeAndName & block, const
|
||||
if (builder)
|
||||
builder.initSink(index_data.size());
|
||||
|
||||
if (!(executeNumber<IndexType, UInt8>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt16>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt32>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt64>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int8>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int16>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int32>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int64>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Float32>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Float64>(block, arguments, result, index_data, builder)
|
||||
|| executeConst<IndexType>(block, arguments, result, index_data, builder, input_rows_count)
|
||||
|| executeString<IndexType>(block, arguments, result, index_data, builder)
|
||||
|| executeGeneric<IndexType>(block, arguments, result, index_data, builder)))
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
if (!(executeNumber<IndexType, UInt8>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt16>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt32>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt64>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int8>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int16>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int32>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int64>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Float32>(columns, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Float64>(columns, arguments, result, index_data, builder)
|
||||
|| executeConst<IndexType>(columns, arguments, result, index_data, builder, input_rows_count)
|
||||
|| executeString<IndexType>(columns, arguments, result, index_data, builder)
|
||||
|| executeGeneric<IndexType>(columns, arguments, result, index_data, builder)))
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::executeTuple(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
bool FunctionArrayElement::executeTuple(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -683,10 +683,10 @@ bool FunctionArrayElement::executeTuple(ColumnsWithTypeAndName & block, const Co
|
||||
size_t tuple_size = tuple_columns.size();
|
||||
|
||||
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(
|
||||
*typeid_cast<const DataTypeArray &>(*block[arguments[0]].type).getNestedType()).getElements();
|
||||
*typeid_cast<const DataTypeArray &>(*columns[arguments[0]].type).getNestedType()).getElements();
|
||||
|
||||
/** We will calculate the function for the tuple of the internals of the array.
|
||||
* To do this, create a temporary block.
|
||||
* To do this, create a temporary columns.
|
||||
* It will consist of the following columns
|
||||
* - the index of the array to be taken;
|
||||
* - an array of the first elements of the tuples;
|
||||
@ -696,7 +696,7 @@ bool FunctionArrayElement::executeTuple(ColumnsWithTypeAndName & block, const Co
|
||||
* ...
|
||||
*/
|
||||
ColumnsWithTypeAndName temporary_results;
|
||||
temporary_results.emplace_back(block[arguments[1]]);
|
||||
temporary_results.emplace_back(columns[arguments[1]]);
|
||||
|
||||
/// results of taking elements by index for arrays from each element of the tuples;
|
||||
Columns result_tuple_columns;
|
||||
@ -718,7 +718,7 @@ bool FunctionArrayElement::executeTuple(ColumnsWithTypeAndName & block, const Co
|
||||
result_tuple_columns.emplace_back(std::move(temporary_results[i * 2 + 2].column));
|
||||
}
|
||||
|
||||
block[result].column = ColumnTuple::create(result_tuple_columns);
|
||||
columns[result].column = ColumnTuple::create(result_tuple_columns);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -748,7 +748,7 @@ DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments)
|
||||
return array_type->getNestedType();
|
||||
}
|
||||
|
||||
void FunctionArrayElement::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void FunctionArrayElement::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
/// Check nullability.
|
||||
bool is_array_of_nullable = false;
|
||||
@ -756,23 +756,23 @@ void FunctionArrayElement::executeImpl(ColumnsWithTypeAndName & block, const Col
|
||||
const ColumnArray * col_array = nullptr;
|
||||
const ColumnArray * col_const_array = nullptr;
|
||||
|
||||
col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
col_array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
if (col_array)
|
||||
is_array_of_nullable = isColumnNullable(col_array->getData());
|
||||
else
|
||||
{
|
||||
col_const_array = checkAndGetColumnConstData<ColumnArray>(block[arguments[0]].column.get());
|
||||
col_const_array = checkAndGetColumnConstData<ColumnArray>(columns[arguments[0]].column.get());
|
||||
if (col_const_array)
|
||||
is_array_of_nullable = isColumnNullable(col_const_array->getData());
|
||||
else
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (!is_array_of_nullable)
|
||||
{
|
||||
ArrayImpl::NullMapBuilder builder;
|
||||
perform(block, arguments, result, builder, input_rows_count);
|
||||
perform(columns, arguments, result, builder, input_rows_count);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -781,9 +781,9 @@ void FunctionArrayElement::executeImpl(ColumnsWithTypeAndName & block, const Col
|
||||
ColumnsWithTypeAndName source_columns;
|
||||
|
||||
const DataTypePtr & input_type = typeid_cast<const DataTypeNullable &>(
|
||||
*typeid_cast<const DataTypeArray &>(*block[arguments[0]].type).getNestedType()).getNestedType();
|
||||
*typeid_cast<const DataTypeArray &>(*columns[arguments[0]].type).getNestedType()).getNestedType();
|
||||
|
||||
DataTypePtr tmp_ret_type = removeNullable(block[result].type);
|
||||
DataTypePtr tmp_ret_type = removeNullable(columns[result].type);
|
||||
|
||||
if (col_array)
|
||||
{
|
||||
@ -798,7 +798,7 @@ void FunctionArrayElement::executeImpl(ColumnsWithTypeAndName & block, const Col
|
||||
std::make_shared<DataTypeArray>(input_type),
|
||||
""
|
||||
},
|
||||
block[arguments[1]],
|
||||
columns[arguments[1]],
|
||||
{
|
||||
nullptr,
|
||||
tmp_ret_type,
|
||||
@ -821,7 +821,7 @@ void FunctionArrayElement::executeImpl(ColumnsWithTypeAndName & block, const Col
|
||||
std::make_shared<DataTypeArray>(input_type),
|
||||
""
|
||||
},
|
||||
block[arguments[1]],
|
||||
columns[arguments[1]],
|
||||
{
|
||||
nullptr,
|
||||
tmp_ret_type,
|
||||
@ -836,33 +836,33 @@ void FunctionArrayElement::executeImpl(ColumnsWithTypeAndName & block, const Col
|
||||
|
||||
/// Store the result.
|
||||
const ColumnWithTypeAndName & source_col = source_columns[2];
|
||||
ColumnWithTypeAndName & dest_col = block[result];
|
||||
ColumnWithTypeAndName & dest_col = columns[result];
|
||||
dest_col.column = ColumnNullable::create(source_col.column, builder ? std::move(builder).getNullMapColumnPtr() : ColumnUInt8::create());
|
||||
}
|
||||
}
|
||||
|
||||
void FunctionArrayElement::perform(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result,
|
||||
void FunctionArrayElement::perform(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) const
|
||||
{
|
||||
if (executeTuple(block, arguments, result, input_rows_count))
|
||||
if (executeTuple(columns, arguments, result, input_rows_count))
|
||||
{
|
||||
}
|
||||
else if (!isColumnConst(*block[arguments[1]].column))
|
||||
else if (!isColumnConst(*columns[arguments[1]].column))
|
||||
{
|
||||
if (!(executeArgument<UInt8>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt16>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt32>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt64>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int8>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int16>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int32>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int64>(block, arguments, result, builder, input_rows_count)))
|
||||
if (!(executeArgument<UInt8>(columns, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt16>(columns, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt32>(columns, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt64>(columns, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int8>(columns, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int16>(columns, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int32>(columns, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int64>(columns, arguments, result, builder, input_rows_count)))
|
||||
throw Exception("Second argument for function " + getName() + " must must have UInt or Int type.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
Field index = (*block[arguments[1]].column)[0];
|
||||
Field index = (*columns[arguments[1]].column)[0];
|
||||
|
||||
if (builder)
|
||||
builder.initSink(input_rows_count);
|
||||
@ -870,19 +870,19 @@ void FunctionArrayElement::perform(ColumnsWithTypeAndName & block, const ColumnN
|
||||
if (index == 0u)
|
||||
throw Exception("Array indices are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
|
||||
if (!(executeNumberConst<UInt8>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt16>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt32>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt64>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int8>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int16>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int32>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int64>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Float32>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Float64>(block, arguments, result, index, builder)
|
||||
|| executeStringConst (block, arguments, result, index, builder)
|
||||
|| executeGenericConst (block, arguments, result, index, builder)))
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
if (!(executeNumberConst<UInt8>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt16>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt32>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt64>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int8>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int16>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int32>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int64>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<Float32>(columns, arguments, result, index, builder)
|
||||
|| executeNumberConst<Float64>(columns, arguments, result, index, builder)
|
||||
|| executeStringConst (columns, arguments, result, index, builder)
|
||||
|| executeGenericConst (columns, arguments, result, index, builder)))
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -45,9 +45,9 @@ public:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>());
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
{
|
||||
if (const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get()))
|
||||
if (const ColumnArray * array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
|
||||
@ -63,11 +63,11 @@ public:
|
||||
prev_off = off;
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(res_nested), array->getOffsetsPtr());
|
||||
columns[result].column = ColumnArray::create(std::move(res_nested), array->getOffsetsPtr());
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -56,7 +56,7 @@ public:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>());
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 64 elements. NOTE: This is just a guess.
|
||||
@ -121,7 +121,7 @@ private:
|
||||
|
||||
|
||||
template <typename Derived>
|
||||
void FunctionArrayEnumerateExtended<Derived>::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
void FunctionArrayEnumerateExtended<Derived>::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
{
|
||||
const ColumnArray::Offsets * offsets = nullptr;
|
||||
size_t num_arguments = arguments.size();
|
||||
@ -131,14 +131,14 @@ void FunctionArrayEnumerateExtended<Derived>::executeImpl(ColumnsWithTypeAndName
|
||||
ColumnPtr offsets_column;
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
const ColumnPtr & array_ptr = block[arguments[i]].column;
|
||||
const ColumnPtr & array_ptr = columns[arguments[i]].column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(
|
||||
block[arguments[i]].column.get());
|
||||
columns[arguments[i]].column.get());
|
||||
if (!const_array)
|
||||
throw Exception("Illegal column " + block[arguments[i]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[i]].column->getName()
|
||||
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_holders.emplace_back(const_array->convertToFullColumn());
|
||||
@ -201,7 +201,7 @@ void FunctionArrayEnumerateExtended<Derived>::executeImpl(ColumnsWithTypeAndName
|
||||
executeHashed(*offsets, data_columns, res_values);
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(res_nested), offsets_column);
|
||||
columns[result].column = ColumnArray::create(std::move(res_nested), offsets_column);
|
||||
}
|
||||
|
||||
template <typename Derived>
|
||||
|
@ -116,7 +116,7 @@ public:
|
||||
return type;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 64 elements. NOTE: This is just a guess.
|
||||
@ -150,7 +150,7 @@ static inline UInt128 ALWAYS_INLINE hash128depths(const std::vector<size_t> & in
|
||||
|
||||
template <typename Derived>
|
||||
void FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
{
|
||||
size_t num_arguments = arguments.size();
|
||||
ColumnRawPtrs data_columns;
|
||||
@ -161,7 +161,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
|
||||
ColumnsWithTypeAndName args;
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
args.emplace_back(block[arguments[i]]);
|
||||
args.emplace_back(columns[arguments[i]]);
|
||||
|
||||
const ArraysDepths arrays_depths = getArraysDepths(args);
|
||||
|
||||
@ -186,7 +186,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
|
||||
size_t array_num = 0;
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
const auto * array = get_array_column(block[arguments[i]].column.get());
|
||||
const auto * array = get_array_column(columns[arguments[i]].column.get());
|
||||
if (!array)
|
||||
continue;
|
||||
|
||||
@ -258,7 +258,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
|
||||
for (ssize_t depth = arrays_depths.max_array_depth - 1; depth >= 0; --depth)
|
||||
result_nested_array = ColumnArray::create(std::move(result_nested_array), offsetsptr_by_depth[depth]);
|
||||
|
||||
block[result].column = result_nested_array;
|
||||
columns[result].column = result_nested_array;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
return std::make_shared<DataTypeArray>(nested_type);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
/** We create an array column with array elements as the most deep elements of nested arrays,
|
||||
* and construct offsets by selecting elements of most deep offsets by values of ancestor offsets.
|
||||
@ -79,10 +79,10 @@ result offsets: 3, 4
|
||||
result: Row 1: [1, 2, 3], Row2: [4]
|
||||
*/
|
||||
|
||||
const ColumnArray * src_col = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * src_col = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!src_col)
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName() + " in argument of function 'arrayFlatten'",
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName() + " in argument of function 'arrayFlatten'",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
const IColumn::Offsets & src_offsets = src_col->getOffsets();
|
||||
@ -107,7 +107,7 @@ result: Row 1: [1, 2, 3], Row2: [4]
|
||||
prev_data = &next_col->getData();
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(
|
||||
columns[result].column = ColumnArray::create(
|
||||
prev_data->getPtr(),
|
||||
result_offsets_column ? std::move(result_offsets_column) : src_col->getOffsetsPtr());
|
||||
}
|
||||
|
@ -384,7 +384,7 @@ public:
|
||||
|
||||
/**
|
||||
* If one or both arguments passed to this function are nullable,
|
||||
* we create a new block that contains non-nullable arguments:
|
||||
* we create a new columns that contains non-nullable arguments:
|
||||
*
|
||||
* - if the 1st argument is a non-constant array of nullable values,
|
||||
* it is turned into a non-constant array of ordinary values + a null
|
||||
@ -396,9 +396,9 @@ public:
|
||||
* (they are vectors of Fields, which may represent the NULL value),
|
||||
* they do not require any preprocessing.
|
||||
*/
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
ColumnPtr& ptr = block[arguments[0]].column;
|
||||
ColumnPtr& ptr = columns[arguments[0]].column;
|
||||
|
||||
/**
|
||||
* The columns here have two general cases, either being Array(T) or Const(Array(T)).
|
||||
@ -411,71 +411,71 @@ public:
|
||||
if (col_array)
|
||||
nullable = checkAndGetColumn<ColumnNullable>(col_array->getData());
|
||||
|
||||
auto & arg_column = block[arguments[1]].column;
|
||||
auto & arg_column = columns[arguments[1]].column;
|
||||
const ColumnNullable * arg_nullable = checkAndGetColumn<ColumnNullable>(*arg_column);
|
||||
|
||||
if (!nullable && !arg_nullable)
|
||||
executeOnNonNullable(block, arguments, result);
|
||||
executeOnNonNullable(columns, arguments, result);
|
||||
else
|
||||
{
|
||||
/**
|
||||
* To correctly process the Nullable values (either #col_array, #arg_column or both) we create a new block
|
||||
* and operate on it. The block structure follows:
|
||||
* To correctly process the Nullable values (either #col_array, #arg_column or both) we create a new columns
|
||||
* and operate on it. The columns structure follows:
|
||||
* {0, 1, 2, 3, 4}
|
||||
* {data (array) argument, "value" argument, data null map, "value" null map, function result}.
|
||||
*/
|
||||
ColumnsWithTypeAndName source_block = { {}, {}, {}, {}, {nullptr, block[result].type, ""} };
|
||||
ColumnsWithTypeAndName source_columns = { {}, {}, {}, {}, {nullptr, columns[result].type, ""} };
|
||||
|
||||
if (nullable)
|
||||
{
|
||||
const auto & nested_col = nullable->getNestedColumnPtr();
|
||||
|
||||
auto & data = source_block[0];
|
||||
auto & data = source_columns[0];
|
||||
|
||||
data.column = ColumnArray::create(nested_col, col_array->getOffsetsPtr());
|
||||
data.type = std::make_shared<DataTypeArray>(
|
||||
static_cast<const DataTypeNullable &>(
|
||||
*static_cast<const DataTypeArray &>(
|
||||
*block[arguments[0]].type
|
||||
*columns[arguments[0]].type
|
||||
).getNestedType()
|
||||
).getNestedType());
|
||||
|
||||
auto & null_map = source_block[2];
|
||||
auto & null_map = source_columns[2];
|
||||
|
||||
null_map.column = nullable->getNullMapColumnPtr();
|
||||
null_map.type = std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & data = source_block[0];
|
||||
data = block[arguments[0]];
|
||||
auto & data = source_columns[0];
|
||||
data = columns[arguments[0]];
|
||||
}
|
||||
|
||||
if (arg_nullable)
|
||||
{
|
||||
auto & arg = source_block[1];
|
||||
auto & arg = source_columns[1];
|
||||
arg.column = arg_nullable->getNestedColumnPtr();
|
||||
arg.type =
|
||||
static_cast<const DataTypeNullable &>(
|
||||
*block[arguments[1]].type
|
||||
*columns[arguments[1]].type
|
||||
).getNestedType();
|
||||
|
||||
auto & null_map = source_block[3];
|
||||
auto & null_map = source_columns[3];
|
||||
null_map.column = arg_nullable->getNullMapColumnPtr();
|
||||
null_map.type = std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & arg = source_block[1];
|
||||
arg = block[arguments[1]];
|
||||
auto & arg = source_columns[1];
|
||||
arg = columns[arguments[1]];
|
||||
}
|
||||
|
||||
/// Now perform the function.
|
||||
executeOnNonNullable(source_block, {0, 1, 2, 3}, 4);
|
||||
executeOnNonNullable(source_columns, {0, 1, 2, 3}, 4);
|
||||
|
||||
/// Move the result to its final position.
|
||||
const ColumnWithTypeAndName & source_col = source_block[4];
|
||||
ColumnWithTypeAndName & dest_col = block[result];
|
||||
const ColumnWithTypeAndName & source_col = source_columns[4];
|
||||
ColumnWithTypeAndName & dest_col = columns[result];
|
||||
dest_col.column = std::move(source_col.column);
|
||||
}
|
||||
}
|
||||
@ -492,12 +492,12 @@ private:
|
||||
const IColumn& left;
|
||||
const IColumn& right;
|
||||
const ColumnArray::Offsets& offsets;
|
||||
ColumnsWithTypeAndName & block;
|
||||
ColumnsWithTypeAndName & columns;
|
||||
size_t result_pos;
|
||||
NullMaps maps;
|
||||
ResultColumnPtr result { ResultColumnType::create() };
|
||||
|
||||
inline void move_result() { block[result_pos].column = std::move(result); }
|
||||
inline void move_result() { columns[result_pos].column = std::move(result); }
|
||||
};
|
||||
|
||||
static inline bool allowNested(const DataTypePtr & left, const DataTypePtr & right)
|
||||
@ -579,13 +579,13 @@ private:
|
||||
|
||||
#define INTEGRAL_TPL_PACK UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64
|
||||
|
||||
void executeOnNonNullable(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
void executeOnNonNullable(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
if (const auto* const left_arr = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get()))
|
||||
if (const auto* const left_arr = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
if (checkAndGetColumn<ColumnLowCardinality>(&left_arr->getData()))
|
||||
{
|
||||
if (executeLowCardinality(block, arguments, result))
|
||||
if (executeLowCardinality(columns, arguments, result))
|
||||
return;
|
||||
|
||||
throw Exception(
|
||||
@ -594,10 +594,10 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
if (!(executeIntegral<INTEGRAL_TPL_PACK>(block, arguments, result)
|
||||
|| executeConst(block, arguments, result)
|
||||
|| executeString(block, arguments, result)
|
||||
|| executeGeneric(block, arguments, result)))
|
||||
if (!(executeIntegral<INTEGRAL_TPL_PACK>(columns, arguments, result)
|
||||
|| executeConst(columns, arguments, result)
|
||||
|| executeString(columns, arguments, result)
|
||||
|| executeGeneric(columns, arguments, result)))
|
||||
throw Exception(
|
||||
"Illegal internal type of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -614,7 +614,7 @@ private:
|
||||
* @return {nullptr, null_map_item} if there are four arguments but the third is missing.
|
||||
* @return {null_map_data, null_map_item} if there are four arguments.
|
||||
*/
|
||||
static NullMaps getNullMaps(const ColumnsWithTypeAndName & block, const ColumnNumbers & arguments) noexcept
|
||||
static NullMaps getNullMaps(const ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments) noexcept
|
||||
{
|
||||
if (arguments.size() < 3)
|
||||
return {nullptr, nullptr};
|
||||
@ -622,10 +622,10 @@ private:
|
||||
const NullMap * null_map_data = nullptr;
|
||||
const NullMap * null_map_item = nullptr;
|
||||
|
||||
if (const auto & data_map = block[arguments[2]].column; data_map)
|
||||
if (const auto & data_map = columns[arguments[2]].column; data_map)
|
||||
null_map_data = &assert_cast<const ColumnUInt8 &>(*data_map).getData();
|
||||
|
||||
if (const auto & item_map = block[arguments[3]].column; item_map)
|
||||
if (const auto & item_map = columns[arguments[3]].column; item_map)
|
||||
null_map_item = &assert_cast<const ColumnUInt8 &>(*item_map).getData();
|
||||
|
||||
return {null_map_data, null_map_item};
|
||||
@ -637,22 +637,22 @@ private:
|
||||
* (s1, s1, s2, ...), (s2, s1, s2, ...), (s3, s1, s2, ...)
|
||||
*/
|
||||
template <class ...Integral>
|
||||
static inline bool executeIntegral(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result_pos)
|
||||
static inline bool executeIntegral(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_pos)
|
||||
{
|
||||
const ColumnArray * const left = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * const left = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!left)
|
||||
return false;
|
||||
|
||||
const IColumn& right = *block[arguments[1]].column.get();
|
||||
const IColumn& right = *columns[arguments[1]].column.get();
|
||||
|
||||
ExecutionData data = {
|
||||
left->getData(),
|
||||
right,
|
||||
left->getOffsets(),
|
||||
block,
|
||||
columns,
|
||||
result_pos,
|
||||
getNullMaps(block, arguments)
|
||||
getNullMaps(columns, arguments)
|
||||
};
|
||||
|
||||
return executeIntegral<Integral...>(data);
|
||||
@ -727,10 +727,10 @@ private:
|
||||
*
|
||||
* Tips and tricks tried can be found at https://github.com/ClickHouse/ClickHouse/pull/12550 .
|
||||
*/
|
||||
static bool executeLowCardinality(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result)
|
||||
static bool executeLowCardinality(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnArray * const col_array = checkAndGetColumn<ColumnArray>(
|
||||
block[arguments[0]].column.get());
|
||||
columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
@ -740,9 +740,9 @@ private:
|
||||
if (!col_lc)
|
||||
return false;
|
||||
|
||||
const auto [null_map_data, null_map_item] = getNullMaps(block, arguments);
|
||||
const auto [null_map_data, null_map_item] = getNullMaps(columns, arguments);
|
||||
|
||||
const IColumn& col_arg = *block[arguments[1]].column.get();
|
||||
const IColumn& col_arg = *columns[arguments[1]].column.get();
|
||||
|
||||
if (const ColumnConst * const col_arg_const = checkAndGetColumn<ColumnConst>(col_arg))
|
||||
{
|
||||
@ -759,11 +759,11 @@ private:
|
||||
&& different_inner_types;
|
||||
|
||||
const DataTypeArray * const array_type = checkAndGetDataType<DataTypeArray>(
|
||||
block[arguments[0]].type.get());
|
||||
columns[arguments[0]].type.get());
|
||||
const DataTypePtr target_type_ptr = recursiveRemoveLowCardinality(array_type->getNestedType());
|
||||
|
||||
const ColumnPtr col_arg_cloned = use_cloned_arg
|
||||
? castColumn(block[arguments[1]], target_type_ptr)
|
||||
? castColumn(columns[arguments[1]], target_type_ptr)
|
||||
: col_arg_const->getPtr();
|
||||
|
||||
const StringRef elem = col_arg_cloned->getDataAt(0);
|
||||
@ -782,7 +782,7 @@ private:
|
||||
|
||||
data.resize_fill(offsets_size);
|
||||
|
||||
block[result].column = std::move(col_result);
|
||||
columns[result].column = std::move(col_result);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@ -795,7 +795,7 @@ private:
|
||||
null_map_data,
|
||||
null_map_item);
|
||||
|
||||
block[result].column = std::move(col_result);
|
||||
columns[result].column = std::move(col_result);
|
||||
return true;
|
||||
}
|
||||
else if (col_lc->nestedIsNullable()) // LC(Nullable(T)) and U
|
||||
@ -821,7 +821,7 @@ private:
|
||||
ExecutionData data = {
|
||||
left_ptr, right_ptr,
|
||||
col_array->getOffsets(),
|
||||
block, result,
|
||||
columns, result,
|
||||
{null_map_left_casted, null_map_right_casted}};
|
||||
|
||||
return dispatchConvertedLCColumns(data);
|
||||
@ -842,7 +842,7 @@ private:
|
||||
|
||||
ExecutionData data = {
|
||||
*left_casted.get(), *right_casted.get(), col_array->getOffsets(),
|
||||
block, result, {null_map_data, null_map_item}
|
||||
columns, result, {null_map_data, null_map_item}
|
||||
};
|
||||
|
||||
return dispatchConvertedLCColumns(data);
|
||||
@ -869,9 +869,9 @@ private:
|
||||
|
||||
#undef INTEGRAL_TPL_PACK
|
||||
|
||||
static bool executeString(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result_pos)
|
||||
static bool executeString(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result_pos)
|
||||
{
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!array)
|
||||
return false;
|
||||
@ -881,11 +881,11 @@ private:
|
||||
if (!left)
|
||||
return false;
|
||||
|
||||
const IColumn & right = *block[arguments[1]].column.get();
|
||||
const IColumn & right = *columns[arguments[1]].column.get();
|
||||
|
||||
ExecutionData data = {
|
||||
*left, right, array->getOffsets(),
|
||||
block, result_pos, getNullMaps(block, arguments),
|
||||
columns, result_pos, getNullMaps(columns, arguments),
|
||||
std::move(ResultColumnType::create())
|
||||
};
|
||||
|
||||
@ -954,17 +954,17 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
static bool executeConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result)
|
||||
static bool executeConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnConst * col_array = checkAndGetColumnConst<ColumnArray>(
|
||||
block[arguments[0]].column.get());
|
||||
columns[arguments[0]].column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
Array arr = col_array->getValue<Array>();
|
||||
|
||||
const IColumn * item_arg = block[arguments[1]].column.get();
|
||||
const IColumn * item_arg = columns[arguments[1]].column.get();
|
||||
|
||||
if (isColumnConst(*item_arg))
|
||||
{
|
||||
@ -982,7 +982,7 @@ private:
|
||||
break;
|
||||
}
|
||||
|
||||
block[result].column = block[result].type->createColumnConst(
|
||||
columns[result].column = columns[result].type->createColumnConst(
|
||||
item_arg->size(), static_cast<ResultType>(current));
|
||||
}
|
||||
else
|
||||
@ -991,7 +991,7 @@ private:
|
||||
const NullMap * null_map = nullptr;
|
||||
|
||||
if (arguments.size() > 2)
|
||||
if (const auto & col = block[arguments[3]].column; col)
|
||||
if (const auto & col = columns[arguments[3]].column; col)
|
||||
null_map = &assert_cast<const ColumnUInt8 &>(*col).getData();
|
||||
|
||||
const size_t size = item_arg->size();
|
||||
@ -1025,25 +1025,25 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
static bool executeGeneric(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result)
|
||||
static bool executeGeneric(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnArray * col = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * col = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
|
||||
if (!col)
|
||||
return false;
|
||||
|
||||
const IColumn & col_nested = col->getData();
|
||||
const IColumn & item_arg = *block[arguments[1]].column;
|
||||
const IColumn & item_arg = *columns[arguments[1]].column;
|
||||
|
||||
auto col_res = ResultColumnType::create();
|
||||
|
||||
auto [null_map_data, null_map_item] = getNullMaps(block, arguments);
|
||||
auto [null_map_data, null_map_item] = getNullMaps(columns, arguments);
|
||||
|
||||
if (item_arg.onlyNull())
|
||||
Impl::Null<ConcreteAction>::process(
|
||||
@ -1067,7 +1067,7 @@ private:
|
||||
null_map_data,
|
||||
null_map_item);
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
@ -48,7 +48,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
@ -88,7 +88,7 @@ private:
|
||||
ColumnsWithTypeAndName casted;
|
||||
};
|
||||
|
||||
static CastArgumentsResult castColumns(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments,
|
||||
static CastArgumentsResult castColumns(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments,
|
||||
const DataTypePtr & return_type, const DataTypePtr & return_type_with_nulls);
|
||||
UnpackedArrays prepareArrays(const ColumnsWithTypeAndName & columns, ColumnsWithTypeAndName & initial_columns) const;
|
||||
|
||||
@ -206,7 +206,7 @@ ColumnPtr FunctionArrayIntersect::castRemoveNullable(const ColumnPtr & column, c
|
||||
}
|
||||
|
||||
FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns(
|
||||
ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, const DataTypePtr & return_type,
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, const DataTypePtr & return_type,
|
||||
const DataTypePtr & return_type_with_nulls)
|
||||
{
|
||||
size_t num_args = arguments.size();
|
||||
@ -233,7 +233,7 @@ FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns(
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & arg = block[arguments[i]];
|
||||
const ColumnWithTypeAndName & arg = columns[arguments[i]];
|
||||
initial_columns[i] = arg;
|
||||
columns[i] = arg;
|
||||
auto & column = columns[i];
|
||||
@ -383,9 +383,9 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
|
||||
return arrays;
|
||||
}
|
||||
|
||||
void FunctionArrayIntersect::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void FunctionArrayIntersect::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
const auto * return_type_array = checkAndGetDataType<DataTypeArray>(return_type.get());
|
||||
|
||||
if (!return_type_array)
|
||||
@ -395,7 +395,7 @@ void FunctionArrayIntersect::executeImpl(ColumnsWithTypeAndName & block, const C
|
||||
|
||||
if (typeid_cast<const DataTypeNothing *>(nested_return_type.get()))
|
||||
{
|
||||
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
columns[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -403,11 +403,11 @@ void FunctionArrayIntersect::executeImpl(ColumnsWithTypeAndName & block, const C
|
||||
DataTypes data_types;
|
||||
data_types.reserve(num_args);
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
data_types.push_back(block[arguments[i]].type);
|
||||
data_types.push_back(columns[arguments[i]].type);
|
||||
|
||||
auto return_type_with_nulls = getMostSubtype(data_types, true, true);
|
||||
|
||||
auto columns = castColumns(block, arguments, return_type, return_type_with_nulls);
|
||||
auto columns = castColumns(columns, arguments, return_type, return_type_with_nulls);
|
||||
|
||||
UnpackedArrays arrays = prepareArrays(columns.casted, columns.initial);
|
||||
|
||||
@ -446,7 +446,7 @@ void FunctionArrayIntersect::executeImpl(ColumnsWithTypeAndName & block, const C
|
||||
}
|
||||
}
|
||||
|
||||
block[result].column = std::move(result_column);
|
||||
columns[result].column = std::move(result_column);
|
||||
}
|
||||
|
||||
template <typename T, size_t>
|
||||
|
@ -39,17 +39,17 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
columns[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
const auto & array_column = block[arguments[0]].column;
|
||||
const auto & array_column = columns[arguments[0]].column;
|
||||
|
||||
std::unique_ptr<GatherUtils::IArraySource> source;
|
||||
|
||||
@ -67,7 +67,7 @@ public:
|
||||
else
|
||||
sink = GatherUtils::sliceFromLeftConstantOffsetBounded(*source, 0, -1);
|
||||
|
||||
block[result].column = std::move(sink);
|
||||
columns[result].column = std::move(sink);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -47,27 +47,27 @@ public:
|
||||
return std::make_shared<DataTypeArray>(getLeastSupertype(types));
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
columns[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_column = return_type->createColumn();
|
||||
|
||||
auto array_column = block[arguments[0]].column;
|
||||
auto appended_column = block[arguments[1]].column;
|
||||
auto array_column = columns[arguments[0]].column;
|
||||
auto appended_column = columns[arguments[1]].column;
|
||||
|
||||
if (!block[arguments[0]].type->equals(*return_type))
|
||||
array_column = castColumn(block[arguments[0]], return_type);
|
||||
if (!columns[arguments[0]].type->equals(*return_type))
|
||||
array_column = castColumn(columns[arguments[0]], return_type);
|
||||
|
||||
const DataTypePtr & return_nested_type = typeid_cast<const DataTypeArray &>(*return_type).getNestedType();
|
||||
if (!block[arguments[1]].type->equals(*return_nested_type))
|
||||
appended_column = castColumn(block[arguments[1]], return_nested_type);
|
||||
if (!columns[arguments[1]].type->equals(*return_nested_type))
|
||||
appended_column = castColumn(columns[arguments[1]], return_nested_type);
|
||||
|
||||
std::unique_ptr<GatherUtils::IArraySource> array_source;
|
||||
std::unique_ptr<GatherUtils::IValueSource> value_source;
|
||||
@ -100,7 +100,7 @@ public:
|
||||
|
||||
GatherUtils::push(*array_source, *value_source, *sink, push_front);
|
||||
|
||||
block[result].column = std::move(result_column);
|
||||
columns[result].column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
/// lazy initialization in getReturnTypeImpl
|
||||
@ -105,7 +105,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName
|
||||
}
|
||||
|
||||
|
||||
void FunctionArrayReduce::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void FunctionArrayReduce::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
IAggregateFunction & agg_func = *aggregate_function;
|
||||
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
|
||||
@ -120,7 +120,7 @@ void FunctionArrayReduce::executeImpl(ColumnsWithTypeAndName & block, const Colu
|
||||
|
||||
for (size_t i = 0; i < num_arguments_columns; ++i)
|
||||
{
|
||||
const IColumn * col = block[arguments[i + 1]].column.get();
|
||||
const IColumn * col = columns[arguments[i + 1]].column.get();
|
||||
|
||||
const ColumnArray::Offsets * offsets_i = nullptr;
|
||||
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(col))
|
||||
@ -146,7 +146,7 @@ void FunctionArrayReduce::executeImpl(ColumnsWithTypeAndName & block, const Colu
|
||||
}
|
||||
const IColumn ** aggregate_arguments = aggregate_arguments_vec.data();
|
||||
|
||||
MutableColumnPtr result_holder = block[result].type->createColumn();
|
||||
MutableColumnPtr result_holder = columns[result].type->createColumn();
|
||||
IColumn & res_col = *result_holder;
|
||||
|
||||
/// AggregateFunction's states should be inserted into column using specific way
|
||||
@ -154,7 +154,7 @@ void FunctionArrayReduce::executeImpl(ColumnsWithTypeAndName & block, const Colu
|
||||
|
||||
if (!res_col_aggregate_function && agg_func.isState())
|
||||
throw Exception("State function " + agg_func.getName() + " inserts results into non-state column "
|
||||
+ block[result].type->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
+ columns[result].type->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
PODArray<AggregateDataPtr> places(input_rows_count);
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
@ -191,7 +191,7 @@ void FunctionArrayReduce::executeImpl(ColumnsWithTypeAndName & block, const Colu
|
||||
agg_func.insertResultInto(places[i], res_col, arena.get());
|
||||
else
|
||||
res_col_aggregate_function->insertFrom(places[i]);
|
||||
block[result].column = std::move(result_holder);
|
||||
columns[result].column = std::move(result_holder);
|
||||
}
|
||||
|
||||
|
||||
|
@ -52,7 +52,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
/// lazy initialization in getReturnTypeImpl
|
||||
@ -123,7 +123,7 @@ DataTypePtr FunctionArrayReduceInRanges::getReturnTypeImpl(const ColumnsWithType
|
||||
}
|
||||
|
||||
|
||||
void FunctionArrayReduceInRanges::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void FunctionArrayReduceInRanges::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
IAggregateFunction & agg_func = *aggregate_function;
|
||||
std::unique_ptr<Arena> arena = std::make_unique<Arena>();
|
||||
@ -133,7 +133,7 @@ void FunctionArrayReduceInRanges::executeImpl(ColumnsWithTypeAndName & block, co
|
||||
|
||||
/// Handling ranges
|
||||
|
||||
const IColumn * ranges_col_array = block[arguments[1]].column.get();
|
||||
const IColumn * ranges_col_array = columns[arguments[1]].column.get();
|
||||
const IColumn * ranges_col_tuple = nullptr;
|
||||
const ColumnArray::Offsets * ranges_offsets = nullptr;
|
||||
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(ranges_col_array))
|
||||
@ -164,7 +164,7 @@ void FunctionArrayReduceInRanges::executeImpl(ColumnsWithTypeAndName & block, co
|
||||
|
||||
for (size_t i = 0; i < num_arguments_columns; ++i)
|
||||
{
|
||||
const IColumn * col = block[arguments[i + 2]].column.get();
|
||||
const IColumn * col = columns[arguments[i + 2]].column.get();
|
||||
|
||||
const ColumnArray::Offsets * offsets_i = nullptr;
|
||||
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(col))
|
||||
@ -192,7 +192,7 @@ void FunctionArrayReduceInRanges::executeImpl(ColumnsWithTypeAndName & block, co
|
||||
|
||||
/// Handling results
|
||||
|
||||
MutableColumnPtr result_holder = block[result].type->createColumn();
|
||||
MutableColumnPtr result_holder = columns[result].type->createColumn();
|
||||
ColumnArray * result_arr = static_cast<ColumnArray *>(result_holder.get());
|
||||
IColumn & result_data = result_arr->getData();
|
||||
|
||||
@ -203,7 +203,7 @@ void FunctionArrayReduceInRanges::executeImpl(ColumnsWithTypeAndName & block, co
|
||||
|
||||
if (!res_col_aggregate_function && agg_func.isState())
|
||||
throw Exception("State function " + agg_func.getName() + " inserts results into non-state column "
|
||||
+ block[result].type->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
+ columns[result].type->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
/// Perform the aggregation
|
||||
|
||||
@ -383,7 +383,7 @@ void FunctionArrayReduceInRanges::executeImpl(ColumnsWithTypeAndName & block, co
|
||||
}
|
||||
}
|
||||
|
||||
block[result].column = std::move(result_holder);
|
||||
columns[result].column = std::move(result_holder);
|
||||
}
|
||||
|
||||
|
||||
|
@ -63,23 +63,23 @@ public:
|
||||
return std::make_shared<DataTypeArray>(getLeastSupertype({array_type->getNestedType(), arguments[2]}));
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
columns[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_column = return_type->createColumn();
|
||||
|
||||
auto array_column = block[arguments[0]].column;
|
||||
auto size_column = block[arguments[1]].column;
|
||||
auto array_column = columns[arguments[0]].column;
|
||||
auto size_column = columns[arguments[1]].column;
|
||||
|
||||
if (!block[arguments[0]].type->equals(*return_type))
|
||||
array_column = castColumn(block[arguments[0]], return_type);
|
||||
if (!columns[arguments[0]].type->equals(*return_type))
|
||||
array_column = castColumn(columns[arguments[0]], return_type);
|
||||
|
||||
const DataTypePtr & return_nested_type = typeid_cast<const DataTypeArray &>(*return_type).getNestedType();
|
||||
size_t size = array_column->size();
|
||||
@ -87,9 +87,9 @@ public:
|
||||
ColumnPtr appended_column;
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
appended_column = block[arguments[2]].column;
|
||||
if (!block[arguments[2]].type->equals(*return_nested_type))
|
||||
appended_column = castColumn(block[arguments[2]], return_nested_type);
|
||||
appended_column = columns[arguments[2]].column;
|
||||
if (!columns[arguments[2]].type->equals(*return_nested_type))
|
||||
appended_column = castColumn(columns[arguments[2]], return_nested_type);
|
||||
}
|
||||
else
|
||||
appended_column = return_nested_type->createColumnConstWithDefaultValue(size);
|
||||
@ -127,7 +127,7 @@ public:
|
||||
else
|
||||
GatherUtils::resizeDynamicSize(*array_source, *value_source, *sink, *size_column);
|
||||
|
||||
block[result].column = std::move(result_column);
|
||||
columns[result].column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override;
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
@ -53,11 +53,11 @@ private:
|
||||
};
|
||||
|
||||
|
||||
void FunctionArrayReverse::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const
|
||||
void FunctionArrayReverse::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const
|
||||
{
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
if (!array)
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
auto res_ptr = array->cloneEmpty();
|
||||
@ -96,7 +96,7 @@ void FunctionArrayReverse::executeImpl(ColumnsWithTypeAndName & block, const Col
|
||||
+ " of null map of the first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block[result].column = std::move(res_ptr);
|
||||
columns[result].column = std::move(res_ptr);
|
||||
}
|
||||
|
||||
|
||||
|
@ -31,26 +31,26 @@ private:
|
||||
using ResultColumnType = ColumnVector<typename Method::ResultType>;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
bool executeNumber(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
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 executeNumberNumber<T, UInt8>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, UInt16>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, UInt32>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, UInt64>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, Int8>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, Int16>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, Int32>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, Int64>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, Float32>(columns, arguments, result)
|
||||
|| executeNumberNumber<T, Float64>(columns, arguments, result);
|
||||
}
|
||||
|
||||
|
||||
template <typename T, typename U>
|
||||
bool executeNumberNumber(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result) const
|
||||
bool executeNumberNumber(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result) const
|
||||
{
|
||||
ColumnPtr col1 = block[arguments[0]].column->convertToFullColumnIfConst();
|
||||
ColumnPtr col2 = block[arguments[1]].column->convertToFullColumnIfConst();
|
||||
ColumnPtr col1 = columns[arguments[0]].column->convertToFullColumnIfConst();
|
||||
ColumnPtr col2 = columns[arguments[1]].column->convertToFullColumnIfConst();
|
||||
if (!col1 || !col2)
|
||||
return false;
|
||||
|
||||
@ -75,7 +75,7 @@ private:
|
||||
col_array1->getOffsets(),
|
||||
col_res->getData());
|
||||
|
||||
block[result].column = std::move(col_res);
|
||||
columns[result].column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -123,19 +123,19 @@ public:
|
||||
return Method::getReturnType(nested_types[0], nested_types[1]);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) const override
|
||||
{
|
||||
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)))
|
||||
throw Exception{"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function "
|
||||
if (!(executeNumber<UInt8>(columns, arguments, result)
|
||||
|| executeNumber<UInt16>(columns, arguments, result)
|
||||
|| executeNumber<UInt32>(columns, arguments, result)
|
||||
|| executeNumber<UInt64>(columns, arguments, result)
|
||||
|| executeNumber<Int8>(columns, arguments, result)
|
||||
|| executeNumber<Int16>(columns, arguments, result)
|
||||
|| executeNumber<Int32>(columns, arguments, result)
|
||||
|| executeNumber<Int64>(columns, arguments, result)
|
||||
|| executeNumber<Float32>(columns, arguments, result)
|
||||
|| executeNumber<Float64>(columns, arguments, result)))
|
||||
throw Exception{"Illegal column " + columns[arguments[0]].column->getName() + " of first argument of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
@ -69,19 +69,19 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
const auto & return_type = block[result].type;
|
||||
const auto & return_type = columns[result].type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
columns[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto & array_column = block[arguments[0]].column;
|
||||
const auto & offset_column = block[arguments[1]].column;
|
||||
const auto & length_column = arguments.size() > 2 ? block[arguments[2]].column : nullptr;
|
||||
auto & array_column = columns[arguments[0]].column;
|
||||
const auto & offset_column = columns[arguments[1]].column;
|
||||
const auto & length_column = arguments.size() > 2 ? columns[arguments[2]].column : nullptr;
|
||||
|
||||
std::unique_ptr<GatherUtils::IArraySource> source;
|
||||
|
||||
@ -105,7 +105,7 @@ public:
|
||||
{
|
||||
if (!length_column || length_column->onlyNull())
|
||||
{
|
||||
block[result].column = array_column;
|
||||
columns[result].column = array_column;
|
||||
return;
|
||||
}
|
||||
else if (isColumnConst(*length_column))
|
||||
@ -146,7 +146,7 @@ public:
|
||||
sink = GatherUtils::sliceDynamicOffsetBounded(*source, *offset_column, *length_column);
|
||||
}
|
||||
|
||||
block[result].column = std::move(sink);
|
||||
columns[result].column = std::move(sink);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -57,7 +57,7 @@ public:
|
||||
return std::make_shared<DataTypeUInt32>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
@ -121,7 +121,7 @@ private:
|
||||
};
|
||||
|
||||
|
||||
void FunctionArrayUniq::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
void FunctionArrayUniq::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
|
||||
{
|
||||
const ColumnArray::Offsets * offsets = nullptr;
|
||||
const size_t num_arguments = arguments.size();
|
||||
@ -131,14 +131,14 @@ void FunctionArrayUniq::executeImpl(ColumnsWithTypeAndName & block, const Column
|
||||
Columns array_holders;
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
const ColumnPtr & array_ptr = block[arguments[i]].column;
|
||||
const ColumnPtr & array_ptr = columns[arguments[i]].column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(
|
||||
block[arguments[i]].column.get());
|
||||
columns[arguments[i]].column.get());
|
||||
if (!const_array)
|
||||
throw Exception("Illegal column " + block[arguments[i]].column->getName()
|
||||
throw Exception("Illegal column " + columns[arguments[i]].column->getName()
|
||||
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_holders.emplace_back(const_array->convertToFullColumn());
|
||||
@ -196,7 +196,7 @@ void FunctionArrayUniq::executeImpl(ColumnsWithTypeAndName & block, const Column
|
||||
executeHashed(*offsets, data_columns, res_values);
|
||||
}
|
||||
|
||||
block[result].column = std::move(res);
|
||||
columns[result].column = std::move(res);
|
||||
}
|
||||
|
||||
template <typename Method, bool has_null_map>
|
||||
|
@ -16,7 +16,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
/// Reasonable threshold.
|
||||
static constexpr size_t max_arrays_size_in_block = 1000000000;
|
||||
static constexpr size_t max_arrays_size_in_columns = 1000000000;
|
||||
|
||||
|
||||
/* arrayWithConstant(num, const) - make array of constants with length num.
|
||||
@ -47,10 +47,10 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t num_rows) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t num_rows) const override
|
||||
{
|
||||
const auto * col_num = block[arguments[0]].column.get();
|
||||
const auto * col_value = block[arguments[1]].column.get();
|
||||
const auto * col_num = columns[arguments[0]].column.get();
|
||||
const auto * col_value = columns[arguments[1]].column.get();
|
||||
|
||||
auto offsets_col = ColumnArray::ColumnOffsets::create();
|
||||
ColumnArray::Offsets & offsets = offsets_col->getData();
|
||||
@ -66,13 +66,13 @@ public:
|
||||
|
||||
offset += array_size;
|
||||
|
||||
if (unlikely(offset > max_arrays_size_in_block))
|
||||
if (unlikely(offset > max_arrays_size_in_columns))
|
||||
throw Exception("Too large array size while executing function " + getName(), ErrorCodes::TOO_LARGE_ARRAY_SIZE);
|
||||
|
||||
offsets.push_back(offset);
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(col_value->replicate(offsets)->convertToFullColumnIfConst(), std::move(offsets_col));
|
||||
columns[result].column = ColumnArray::create(col_value->replicate(offsets)->convertToFullColumnIfConst(), std::move(offsets_col));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(arguments_types));
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
size_t num_arguments = arguments.size();
|
||||
|
||||
@ -65,7 +65,7 @@ public:
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
/// Constant columns cannot be inside tuple. It's only possible to have constant tuple as a whole.
|
||||
ColumnPtr holder = block[arguments[i]].column->convertToFullColumnIfConst();
|
||||
ColumnPtr holder = columns[arguments[i]].column->convertToFullColumnIfConst();
|
||||
|
||||
const ColumnArray * column_array = checkAndGetColumn<ColumnArray>(holder.get());
|
||||
|
||||
@ -86,7 +86,7 @@ public:
|
||||
tuple_columns[i] = column_array->getDataPtr();
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(
|
||||
columns[result].column = ColumnArray::create(
|
||||
ColumnTuple::create(tuple_columns), static_cast<const ColumnArray &>(*first_array_column).getOffsetsPtr());
|
||||
}
|
||||
};
|
||||
|
@ -44,11 +44,11 @@ private:
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataType>());
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
using UnderlyingColumnType = typename TypeToColumnType<typename DataType::FieldType>::ColumnType;
|
||||
|
||||
block[result].column = ColumnArray::create(
|
||||
columns[result].column = ColumnArray::create(
|
||||
UnderlyingColumnType::create(),
|
||||
ColumnArray::ColumnOffsets::create(input_rows_count, 0));
|
||||
}
|
||||
|
@ -45,7 +45,7 @@ public:
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||
};
|
||||
|
||||
|
||||
@ -53,20 +53,20 @@ namespace
|
||||
{
|
||||
namespace FunctionEmptyArrayToSingleImpl
|
||||
{
|
||||
bool executeConst(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
bool executeConst(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(block[arguments[0]].column.get()))
|
||||
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(columns[arguments[0]].column.get()))
|
||||
{
|
||||
if (const_array->getValue<Array>().empty())
|
||||
{
|
||||
auto nested_type = typeid_cast<const DataTypeArray &>(*block[arguments[0]].type).getNestedType();
|
||||
auto nested_type = typeid_cast<const DataTypeArray &>(*columns[arguments[0]].type).getNestedType();
|
||||
|
||||
block[result].column = block[result].type->createColumnConst(
|
||||
columns[result].column = columns[result].type->createColumnConst(
|
||||
input_rows_count,
|
||||
Array{nested_type->getDefault()});
|
||||
}
|
||||
else
|
||||
block[result].column = block[arguments[0]].column;
|
||||
columns[result].column = columns[arguments[0]].column;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -369,14 +369,14 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
void FunctionEmptyArrayToSingle::executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
void FunctionEmptyArrayToSingle::executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
if (FunctionEmptyArrayToSingleImpl::executeConst(block, arguments, result, input_rows_count))
|
||||
if (FunctionEmptyArrayToSingleImpl::executeConst(columns, arguments, result, input_rows_count))
|
||||
return;
|
||||
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(columns[arguments[0]].column.get());
|
||||
if (!array)
|
||||
throw Exception("Illegal column " + block[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
throw Exception("Illegal column " + columns[arguments[0]].column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
MutableColumnPtr res_ptr = array->cloneEmpty();
|
||||
@ -414,7 +414,7 @@ void FunctionEmptyArrayToSingle::executeImpl(ColumnsWithTypeAndName & block, con
|
||||
else
|
||||
FunctionEmptyArrayToSingleImpl::executeDispatch<false>(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map);
|
||||
|
||||
block[result].column = std::move(res_ptr);
|
||||
columns[result].column = std::move(res_ptr);
|
||||
}
|
||||
|
||||
|
||||
|
@ -49,20 +49,20 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
size_t rows = input_rows_count;
|
||||
size_t num_args = arguments.size();
|
||||
|
||||
DataTypePtr common_type = nullptr;
|
||||
auto commonType = [&common_type, &block, &arguments]()
|
||||
auto commonType = [&common_type, &columns, &arguments]()
|
||||
{
|
||||
if (common_type == nullptr)
|
||||
{
|
||||
DataTypes data_types;
|
||||
data_types.reserve(arguments.size());
|
||||
for (const auto & argument : arguments)
|
||||
data_types.push_back(block[argument].type);
|
||||
data_types.push_back(columns[argument].type);
|
||||
|
||||
common_type = getLeastSupertype(data_types);
|
||||
}
|
||||
@ -74,7 +74,7 @@ public:
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const auto & argument = block[arguments[i]];
|
||||
const auto & argument = columns[arguments[i]];
|
||||
ColumnPtr preprocessed_column = argument.column;
|
||||
|
||||
const auto argument_type = typeid_cast<const DataTypeArray *>(argument.type.get());
|
||||
@ -109,7 +109,7 @@ public:
|
||||
auto result_column_ptr = typeid_cast<ColumnUInt8 *>(result_column.get());
|
||||
GatherUtils::sliceHas(*sources[0], *sources[1], search_type, *result_column_ptr);
|
||||
|
||||
block[result].column = std::move(result_column);
|
||||
columns[result].column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
@ -121,7 +121,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename KeyType, bool is_str_key, typename ValType>
|
||||
void execute2(ColumnsWithTypeAndName & block, const size_t result, size_t row_count, TupleMaps & args, const DataTypeTuple & res_type) const
|
||||
void execute2(ColumnsWithTypeAndName & columns, const size_t result, size_t row_count, TupleMaps & args, const DataTypeTuple & res_type) const
|
||||
{
|
||||
MutableColumnPtr res_tuple = res_type.createColumn();
|
||||
|
||||
@ -199,18 +199,18 @@ private:
|
||||
// same offsets as in keys
|
||||
to_vals_arr.getOffsets().insert(to_keys_offset.begin(), to_keys_offset.end());
|
||||
|
||||
block[result].column = std::move(res_tuple);
|
||||
columns[result].column = std::move(res_tuple);
|
||||
}
|
||||
|
||||
template <typename KeyType, bool is_str_key>
|
||||
void execute1(ColumnsWithTypeAndName & block, const size_t result, size_t row_count, const DataTypeTuple & res_type, TupleMaps & args) const
|
||||
void execute1(ColumnsWithTypeAndName & columns, const size_t result, size_t row_count, const DataTypeTuple & res_type, TupleMaps & args) const
|
||||
{
|
||||
const auto & promoted_type = (assert_cast<const DataTypeArray *>(res_type.getElements()[1].get()))->getNestedType();
|
||||
#define MATCH_EXECUTE(is_str) \
|
||||
switch (promoted_type->getTypeId()) { \
|
||||
case TypeIndex::Int64: execute2<KeyType, is_str, Int64>(block, result, row_count, args, res_type); break; \
|
||||
case TypeIndex::UInt64: execute2<KeyType, is_str, UInt64>(block, result, row_count, args, res_type); break; \
|
||||
case TypeIndex::Float64: execute2<KeyType, is_str, Float64>(block, result, row_count, args, res_type); break; \
|
||||
case TypeIndex::Int64: execute2<KeyType, is_str, Int64>(columns, result, row_count, args, res_type); break; \
|
||||
case TypeIndex::UInt64: execute2<KeyType, is_str, UInt64>(columns, result, row_count, args, res_type); break; \
|
||||
case TypeIndex::Float64: execute2<KeyType, is_str, Float64>(columns, result, row_count, args, res_type); break; \
|
||||
default: \
|
||||
throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; \
|
||||
}
|
||||
@ -226,9 +226,9 @@ private:
|
||||
#undef MATCH_EXECUTE
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
{
|
||||
const DataTypeTuple * tup_type = checkAndGetDataType<DataTypeTuple>((block[arguments[0]]).type.get());
|
||||
const DataTypeTuple * tup_type = checkAndGetDataType<DataTypeTuple>((columns[arguments[0]]).type.get());
|
||||
const DataTypeArray * key_array_type = checkAndGetDataType<DataTypeArray>(tup_type->getElements()[0].get());
|
||||
const DataTypeArray * val_array_type = checkAndGetDataType<DataTypeArray>(tup_type->getElements()[1].get());
|
||||
|
||||
@ -243,7 +243,7 @@ private:
|
||||
//prepare columns, extract data columns for direct access and put them to the vector
|
||||
for (auto arg : arguments)
|
||||
{
|
||||
auto & col = block[arg];
|
||||
auto & col = columns[arg];
|
||||
const ColumnTuple * tup;
|
||||
bool is_const = isColumnConst(*col.column);
|
||||
if (is_const)
|
||||
@ -274,45 +274,45 @@ private:
|
||||
args.push_back({key_column, val_column, key_offsets, val_offsets, is_const});
|
||||
}
|
||||
|
||||
size_t row_count = block[arguments[0]].column->size();
|
||||
size_t row_count = columns[arguments[0]].column->size();
|
||||
auto key_type_id = key_array_type->getNestedType()->getTypeId();
|
||||
|
||||
switch (key_type_id)
|
||||
{
|
||||
case TypeIndex::Enum8:
|
||||
case TypeIndex::Int8:
|
||||
execute1<Int8, false>(block, result, row_count, res_type, args);
|
||||
execute1<Int8, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::Enum16:
|
||||
case TypeIndex::Int16:
|
||||
execute1<Int16, false>(block, result, row_count, res_type, args);
|
||||
execute1<Int16, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::Int32:
|
||||
execute1<Int32, false>(block, result, row_count, res_type, args);
|
||||
execute1<Int32, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::Int64:
|
||||
execute1<Int64, false>(block, result, row_count, res_type, args);
|
||||
execute1<Int64, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::UInt8:
|
||||
execute1<UInt8, false>(block, result, row_count, res_type, args);
|
||||
execute1<UInt8, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::Date:
|
||||
case TypeIndex::UInt16:
|
||||
execute1<UInt16, false>(block, result, row_count, res_type, args);
|
||||
execute1<UInt16, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::DateTime:
|
||||
case TypeIndex::UInt32:
|
||||
execute1<UInt32, false>(block, result, row_count, res_type, args);
|
||||
execute1<UInt32, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::UInt64:
|
||||
execute1<UInt64, false>(block, result, row_count, res_type, args);
|
||||
execute1<UInt64, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::UUID:
|
||||
execute1<UInt128, false>(block, result, row_count, res_type, args);
|
||||
execute1<UInt128, false>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
case TypeIndex::FixedString:
|
||||
case TypeIndex::String:
|
||||
execute1<String, true>(block, result, row_count, res_type, args);
|
||||
execute1<String, true>(columns, result, row_count, res_type, args);
|
||||
break;
|
||||
default:
|
||||
throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
@ -72,7 +72,7 @@ private:
|
||||
|
||||
template <typename KeyType, typename ValType>
|
||||
void execute2(
|
||||
ColumnsWithTypeAndName & block, size_t result, ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type)
|
||||
ColumnsWithTypeAndName & columns, size_t result, ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type)
|
||||
const
|
||||
{
|
||||
MutableColumnPtr res_tuple = res_type.createColumn();
|
||||
@ -213,50 +213,50 @@ private:
|
||||
}
|
||||
|
||||
to_vals_arr.getOffsets().insert(to_keys_offsets.begin(), to_keys_offsets.end());
|
||||
block[result].column = std::move(res_tuple);
|
||||
columns[result].column = std::move(res_tuple);
|
||||
}
|
||||
|
||||
template <typename KeyType>
|
||||
void execute1(
|
||||
ColumnsWithTypeAndName & block, size_t result, ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type)
|
||||
ColumnsWithTypeAndName & columns, size_t result, ColumnPtr key_column, ColumnPtr val_column, ColumnPtr max_key_column, const DataTypeTuple & res_type)
|
||||
const
|
||||
{
|
||||
const auto & val_type = (assert_cast<const DataTypeArray *>(res_type.getElements()[1].get()))->getNestedType();
|
||||
switch (val_type->getTypeId())
|
||||
{
|
||||
case TypeIndex::Int8:
|
||||
execute2<KeyType, Int8>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, Int8>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::Int16:
|
||||
execute2<KeyType, Int16>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, Int16>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::Int32:
|
||||
execute2<KeyType, Int32>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, Int32>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::Int64:
|
||||
execute2<KeyType, Int64>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, Int64>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt8:
|
||||
execute2<KeyType, UInt8>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, UInt8>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt16:
|
||||
execute2<KeyType, UInt16>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, UInt16>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt32:
|
||||
execute2<KeyType, UInt32>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, UInt32>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt64:
|
||||
execute2<KeyType, UInt64>(block, result, key_column, val_column, max_key_column, res_type);
|
||||
execute2<KeyType, UInt64>(columns, result, key_column, val_column, max_key_column, res_type);
|
||||
break;
|
||||
default:
|
||||
throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
{
|
||||
auto col1 = block[arguments[0]];
|
||||
auto col2 = block[arguments[1]];
|
||||
auto col1 = columns[arguments[0]];
|
||||
auto col2 = columns[arguments[1]];
|
||||
|
||||
const auto * k = assert_cast<const DataTypeArray *>(col1.type.get());
|
||||
const auto * v = assert_cast<const DataTypeArray *>(col2.type.get());
|
||||
@ -270,34 +270,34 @@ private:
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
/* max key provided */
|
||||
max_key_column = block[arguments[2]].column;
|
||||
max_key_column = columns[arguments[2]].column;
|
||||
}
|
||||
|
||||
switch (k->getNestedType()->getTypeId())
|
||||
{
|
||||
case TypeIndex::Int8:
|
||||
execute1<Int8>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<Int8>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::Int16:
|
||||
execute1<Int16>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<Int16>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::Int32:
|
||||
execute1<Int32>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<Int32>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::Int64:
|
||||
execute1<Int64>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<Int64>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt8:
|
||||
execute1<UInt8>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<UInt8>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt16:
|
||||
execute1<UInt16>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<UInt16>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt32:
|
||||
execute1<UInt32>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<UInt32>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
case TypeIndex::UInt64:
|
||||
execute1<UInt64>(block, result, col1.column, col2.column, max_key_column, res_type);
|
||||
execute1<UInt64>(columns, result, col1.column, col2.column, max_key_column, res_type);
|
||||
break;
|
||||
default:
|
||||
throw Exception{"Illegal columns in arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
@ -57,7 +57,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeInternal(ColumnsWithTypeAndName & block, const IColumn * arg, const size_t result) const
|
||||
bool executeInternal(ColumnsWithTypeAndName & columns, const IColumn * arg, const size_t result) const
|
||||
{
|
||||
if (const auto in = checkAndGetColumn<ColumnVector<T>>(arg))
|
||||
{
|
||||
@ -94,7 +94,7 @@ private:
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
columns[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
else
|
||||
@ -103,7 +103,7 @@ private:
|
||||
|
||||
template <typename T>
|
||||
bool executeConstStartStep(
|
||||
ColumnsWithTypeAndName & block, const IColumn * end_arg, const T start, const T step, const size_t input_rows_count, const size_t result) const
|
||||
ColumnsWithTypeAndName & columns, const IColumn * end_arg, const T start, const T step, const size_t input_rows_count, const size_t result) const
|
||||
{
|
||||
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
||||
if (!end_column)
|
||||
@ -157,13 +157,13 @@ private:
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
columns[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeConstStep(
|
||||
ColumnsWithTypeAndName & block, const IColumn * start_arg, const IColumn * end_arg, const T step, const size_t input_rows_count, const size_t result) const
|
||||
ColumnsWithTypeAndName & columns, const IColumn * start_arg, const IColumn * end_arg, const T step, const size_t input_rows_count, const size_t result) const
|
||||
{
|
||||
auto start_column = checkAndGetColumn<ColumnVector<T>>(start_arg);
|
||||
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
||||
@ -219,13 +219,13 @@ private:
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
columns[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeConstStart(
|
||||
ColumnsWithTypeAndName & block, const IColumn * end_arg, const IColumn * step_arg, const T start, const size_t input_rows_count, const size_t result) const
|
||||
ColumnsWithTypeAndName & columns, const IColumn * end_arg, const IColumn * step_arg, const T start, const size_t input_rows_count, const size_t result) const
|
||||
{
|
||||
auto end_column = checkAndGetColumn<ColumnVector<T>>(end_arg);
|
||||
auto step_column = checkAndGetColumn<ColumnVector<T>>(step_arg);
|
||||
@ -281,13 +281,13 @@ private:
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
columns[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeGeneric(
|
||||
ColumnsWithTypeAndName & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col,
|
||||
ColumnsWithTypeAndName & columns, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col,
|
||||
const size_t input_rows_count, const size_t result) const
|
||||
{
|
||||
auto start_column = checkAndGetColumn<ColumnVector<T>>(start_col);
|
||||
@ -347,19 +347,19 @@ private:
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
columns[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
const auto * col = block[arguments[0]].column.get();
|
||||
if (!executeInternal<UInt8>(block, col, result) &&
|
||||
!executeInternal<UInt16>(block, col, result) &&
|
||||
!executeInternal<UInt32>(block, col, result) &&
|
||||
!executeInternal<UInt64>(block, col, result))
|
||||
const auto * col = columns[arguments[0]].column.get();
|
||||
if (!executeInternal<UInt8>(columns, col, result) &&
|
||||
!executeInternal<UInt16>(columns, col, result) &&
|
||||
!executeInternal<UInt32>(columns, col, result) &&
|
||||
!executeInternal<UInt64>(columns, col, result))
|
||||
{
|
||||
throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
@ -369,14 +369,14 @@ private:
|
||||
Columns columns_holder(3);
|
||||
ColumnRawPtrs columns(3);
|
||||
|
||||
const auto return_type = checkAndGetDataType<DataTypeArray>(block[result].type.get())->getNestedType();
|
||||
const auto return_type = checkAndGetDataType<DataTypeArray>(columns[result].type.get())->getNestedType();
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
if (i == 1)
|
||||
columns_holder[i] = castColumn(block[arguments[i]], return_type)->convertToFullColumnIfConst();
|
||||
columns_holder[i] = castColumn(columns[arguments[i]], return_type)->convertToFullColumnIfConst();
|
||||
else
|
||||
columns_holder[i] = castColumn(block[arguments[i]], return_type);
|
||||
columns_holder[i] = castColumn(columns[arguments[i]], return_type);
|
||||
|
||||
columns[i] = columns_holder[i].get();
|
||||
}
|
||||
@ -396,35 +396,35 @@ private:
|
||||
UInt64 start = assert_cast<const ColumnConst &>(*columns[0]).getUInt(0);
|
||||
UInt64 step = assert_cast<const ColumnConst &>(*columns[2]).getUInt(0);
|
||||
|
||||
ok = executeConstStartStep<UInt8>(block, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt16>(block, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt32>(block, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt64>(block, columns[1], start, step, input_rows_count, result);
|
||||
ok = executeConstStartStep<UInt8>(columns, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt16>(columns, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt32>(columns, columns[1], start, step, input_rows_count, result) ||
|
||||
executeConstStartStep<UInt64>(columns, columns[1], start, step, input_rows_count, result);
|
||||
}
|
||||
else if (is_start_const && !is_step_const)
|
||||
{
|
||||
UInt64 start = assert_cast<const ColumnConst &>(*columns[0]).getUInt(0);
|
||||
|
||||
ok = executeConstStart<UInt8>(block, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt16>(block, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt32>(block, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt64>(block, columns[1], columns[2], start, input_rows_count, result);
|
||||
ok = executeConstStart<UInt8>(columns, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt16>(columns, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt32>(columns, columns[1], columns[2], start, input_rows_count, result) ||
|
||||
executeConstStart<UInt64>(columns, columns[1], columns[2], start, input_rows_count, result);
|
||||
}
|
||||
else if (!is_start_const && is_step_const)
|
||||
{
|
||||
UInt64 step = assert_cast<const ColumnConst &>(*columns[2]).getUInt(0);
|
||||
|
||||
ok = executeConstStep<UInt8>(block, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt16>(block, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt32>(block, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt64>(block, columns[0], columns[1], step, input_rows_count, result);
|
||||
ok = executeConstStep<UInt8>(columns, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt16>(columns, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt32>(columns, columns[0], columns[1], step, input_rows_count, result) ||
|
||||
executeConstStep<UInt64>(columns, columns[0], columns[1], step, input_rows_count, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
ok = executeGeneric<UInt8>(block, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt16>(block, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt32>(block, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt64>(block, columns[0], columns[1], columns[2], input_rows_count, result);
|
||||
ok = executeGeneric<UInt8>(columns, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt16>(columns, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt32>(columns, columns[0], columns[1], columns[2], input_rows_count, result) ||
|
||||
executeGeneric<UInt64>(columns, columns[0], columns[1], columns[2], input_rows_count, result);
|
||||
}
|
||||
|
||||
if (!ok)
|
||||
|
@ -39,10 +39,10 @@ public:
|
||||
return removeNullable(arguments[0]);
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t) const override
|
||||
{
|
||||
const ColumnPtr & col = block[arguments[0]].column;
|
||||
ColumnPtr & res_col = block[result].column;
|
||||
const ColumnPtr & col = columns[arguments[0]].column;
|
||||
ColumnPtr & res_col = columns[result].column;
|
||||
|
||||
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(*col))
|
||||
res_col = nullable_col->getNestedColumnPtr();
|
||||
|
@ -67,13 +67,13 @@ public:
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// The level at which the line has zero length.
|
||||
Int64 max = extractConstant<Int64>(block, arguments, 2, "Third"); /// The level at which the line has the maximum length.
|
||||
Int64 min = extractConstant<Int64>(columns, arguments, 1, "Second"); /// The level at which the line has zero length.
|
||||
Int64 max = extractConstant<Int64>(columns, arguments, 2, "Third"); /// The level at which the line has the maximum length.
|
||||
|
||||
/// The maximum width of the bar in characters, by default.
|
||||
Float64 max_width = arguments.size() == 4 ? extractConstant<Float64>(block, arguments, 3, "Fourth") : 80;
|
||||
Float64 max_width = arguments.size() == 4 ? extractConstant<Float64>(columns, arguments, 3, "Fourth") : 80;
|
||||
|
||||
if (max_width < 1)
|
||||
throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
@ -81,7 +81,7 @@ public:
|
||||
if (max_width > 1000)
|
||||
throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
const auto & src = *block[arguments[0]].column;
|
||||
const auto & src = *columns[arguments[0]].column;
|
||||
|
||||
auto res_column = ColumnString::create();
|
||||
|
||||
@ -96,19 +96,19 @@ public:
|
||||
|| executeNumber<Float32>(src, *res_column, min, max, max_width)
|
||||
|| executeNumber<Float64>(src, *res_column, min, max, max_width))
|
||||
{
|
||||
block[result].column = std::move(res_column);
|
||||
columns[result].column = std::move(res_column);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
"Illegal column " + columns[arguments[0]].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
T extractConstant(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const
|
||||
T extractConstant(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const
|
||||
{
|
||||
const auto & column = *block[arguments[argument_pos]].column;
|
||||
const auto & column = *columns[arguments[argument_pos]].column;
|
||||
|
||||
if (!isColumnConst(column))
|
||||
throw Exception(
|
||||
|
@ -10,14 +10,14 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
/** Incremental block number among calls of this function. */
|
||||
/** Incremental columns number among calls of this function. */
|
||||
class FunctionBlockNumber : public IFunction
|
||||
{
|
||||
private:
|
||||
mutable std::atomic<size_t> block_number{0};
|
||||
mutable std::atomic<size_t> columns_number{0};
|
||||
|
||||
public:
|
||||
static constexpr auto name = "blockNumber";
|
||||
static constexpr auto name = "columnsNumber";
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionBlockNumber>();
|
||||
@ -51,10 +51,10 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
size_t current_block_number = block_number++;
|
||||
block[result].column = ColumnUInt64::create(input_rows_count, current_block_number);
|
||||
size_t current_columns_number = columns_number++;
|
||||
columns[result].column = ColumnUInt64::create(input_rows_count, current_columns_number);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -9,11 +9,11 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Returns size on disk for *block* (without taking into account compression).
|
||||
/// Returns size on disk for *columns* (without taking into account compression).
|
||||
class FunctionBlockSerializedSize : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "blockSerializedSize";
|
||||
static constexpr auto name = "columnsSerializedSize";
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
@ -30,18 +30,18 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
UInt64 size = 0;
|
||||
|
||||
for (auto arg_pos : arguments)
|
||||
size += blockSerializedSizeOne(block[arg_pos]);
|
||||
size += columnsSerializedSizeOne(columns[arg_pos]);
|
||||
|
||||
block[result].column = DataTypeUInt64().createColumnConst(
|
||||
columns[result].column = DataTypeUInt64().createColumnConst(
|
||||
input_rows_count, size)->convertToFullColumnIfConst();
|
||||
}
|
||||
|
||||
static UInt64 blockSerializedSizeOne(const ColumnWithTypeAndName & elem)
|
||||
static UInt64 columnsSerializedSizeOne(const ColumnWithTypeAndName & elem)
|
||||
{
|
||||
ColumnPtr full_column = elem.column->convertToFullColumnIfConst();
|
||||
|
||||
|
@ -9,12 +9,12 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
/** blockSize() - get the block size in number of rows.
|
||||
/** columnsSize() - get the columns size in number of rows.
|
||||
*/
|
||||
class FunctionBlockSize : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "blockSize";
|
||||
static constexpr auto name = "columnsSize";
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionBlockSize>();
|
||||
@ -43,9 +43,9 @@ public:
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
block[result].column = ColumnUInt64::create(input_rows_count, input_rows_count);
|
||||
columns[result].column = ColumnUInt64::create(input_rows_count, input_rows_count);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -38,9 +38,9 @@ public:
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(ColumnsWithTypeAndName & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
|
||||
{
|
||||
block[result].column = DataTypeString().createColumnConst(input_rows_count, SymbolIndex::instance().getBuildIDHex());
|
||||
columns[result].column = DataTypeString().createColumnConst(input_rows_count, SymbolIndex::instance().getBuildIDHex());
|
||||
}
|
||||
};
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user