Replace getByPosition to []

This commit is contained in:
Nikolai Kochetov 2020-10-10 21:24:57 +03:00
parent 1e2b8a748b
commit d28325a353
193 changed files with 1106 additions and 1128 deletions

View File

@ -123,22 +123,22 @@ struct CustomWeekTransformImpl
UInt8 week_mode = DEFAULT_WEEK_MODE;
if (arguments.size() > 1)
{
if (const auto week_mode_column = checkAndGetColumnConst<ColumnUInt8>(block.getByPosition(arguments[1]).column.get()))
if (const auto week_mode_column = checkAndGetColumnConst<ColumnUInt8>(block[arguments[1]].column.get()))
week_mode = week_mode_column->getValue<UInt8>();
}
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block.data, arguments, 2, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
const ColumnPtr source_col = block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
{
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function "
"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function "
+ Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -689,19 +689,19 @@ struct DateTimeTransformImpl
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block.data, arguments, 1, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
const ColumnPtr source_col = block[arguments[0]].column;
if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
{
auto mutable_result_col = block.getByPosition(result).type->createColumn();
auto mutable_result_col = block[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.getByPosition(result).column = std::move(mutable_result_col);
block[result].column = std::move(mutable_result_col);
}
else
{
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -92,12 +92,12 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const ColumnPtr column_string = block.getByPosition(arguments[0]).column;
const ColumnPtr column_string = block[arguments[0]].column;
const ColumnString * input = checkAndGetColumn<ColumnString>(column_string.get());
if (!input)
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
auto dst_column = ColumnString::create();
@ -165,7 +165,7 @@ public:
dst_data.resize(dst_pos - dst);
block.getByPosition(result).column = std::move(dst_column);
block[result].column = std::move(dst_column);
}
};
}

View File

@ -616,14 +616,14 @@ class FunctionBinaryArithmetic : public IFunction
void executeAggregateMultiply(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
{
ColumnNumbers new_arguments = arguments;
if (WhichDataType(block.getByPosition(new_arguments[1]).type).isAggregateFunction())
if (WhichDataType(block[new_arguments[1]].type).isAggregateFunction())
std::swap(new_arguments[0], new_arguments[1]);
if (!isColumnConst(*block.getByPosition(new_arguments[1]).column))
throw Exception{"Illegal column " + block.getByPosition(new_arguments[1]).column->getName()
if (!isColumnConst(*block[new_arguments[1]].column))
throw Exception{"Illegal column " + block[new_arguments[1]].column->getName()
+ " of argument of aggregation state multiply. Should be integer constant", ErrorCodes::ILLEGAL_COLUMN};
const IColumn & agg_state_column = *block.getByPosition(new_arguments[0]).column;
const IColumn & agg_state_column = *block[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.getByPosition(new_arguments[1]).column.get())->getValue<UInt64>();
UInt64 m = typeid_cast<const ColumnConst *>(block[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.getByPosition(result).column = ColumnConst::create(std::move(column_to), input_rows_count);
block[result].column = ColumnConst::create(std::move(column_to), input_rows_count);
else
block.getByPosition(result).column = std::move(column_to);
block[result].column = std::move(column_to);
}
/// Merge two aggregation states together.
void executeAggregateAddition(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
{
const IColumn & lhs_column = *block.getByPosition(arguments[0]).column;
const IColumn & rhs_column = *block.getByPosition(arguments[1]).column;
const IColumn & lhs_column = *block[arguments[0]].column;
const IColumn & rhs_column = *block[arguments[1]].column;
bool lhs_is_const = isColumnConst(lhs_column);
bool rhs_is_const = isColumnConst(rhs_column);
@ -707,9 +707,9 @@ class FunctionBinaryArithmetic : public IFunction
}
if (lhs_is_const && rhs_is_const)
block.getByPosition(result).column = ColumnConst::create(std::move(column_to), input_rows_count);
block[result].column = ColumnConst::create(std::move(column_to), input_rows_count);
else
block.getByPosition(result).column = std::move(column_to);
block[result].column = std::move(column_to);
}
void executeDateTimeIntervalPlusMinus(Block & block, const ColumnNumbers & arguments,
@ -718,19 +718,19 @@ class FunctionBinaryArithmetic : public IFunction
ColumnNumbers new_arguments = arguments;
/// Interval argument must be second.
if (WhichDataType(block.getByPosition(arguments[1]).type).isDateOrDateTime())
if (WhichDataType(block[arguments[1]].type).isDateOrDateTime())
std::swap(new_arguments[0], new_arguments[1]);
/// Change interval argument type to its representation
Block new_block = block;
new_block.getByPosition(new_arguments[1]).type = std::make_shared<DataTypeNumber<DataTypeInterval::FieldType>>();
new_block[new_arguments[1]].type = std::make_shared<DataTypeNumber<DataTypeInterval::FieldType>>();
ColumnsWithTypeAndName new_arguments_with_type_and_name =
{new_block.getByPosition(new_arguments[0]), new_block.getByPosition(new_arguments[1])};
{new_block[new_arguments[0]], new_block[new_arguments[1]]};
auto function = function_builder->build(new_arguments_with_type_and_name);
function->execute(new_block.data, new_arguments, result, input_rows_count);
block.getByPosition(result).column = new_block.getByPosition(result).column;
block[result].column = new_block[result].column;
}
public:
@ -855,8 +855,8 @@ public:
{
using OpImpl = FixedStringOperationImpl<Op<UInt8, UInt8>>;
auto col_left_raw = block.getByPosition(arguments[0]).column.get();
auto col_right_raw = block.getByPosition(arguments[1]).column.get();
auto col_left_raw = block[arguments[0]].column.get();
auto col_right_raw = block[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.getByPosition(result).column = ColumnConst::create(std::move(col_res), col_left_raw->size());
block[result].column = ColumnConst::create(std::move(col_res), col_left_raw->size());
return true;
}
}
@ -922,7 +922,7 @@ public:
out_chars.size(),
col_left->getN());
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
return false;
@ -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.getByPosition(arguments[0]).column.get();
auto col_right_raw = block.getByPosition(arguments[1]).column.get();
auto col_left_raw = block[arguments[0]].column.get();
auto col_right_raw = block[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.getByPosition(result).column = ResultDataType(type.getPrecision(), type.getScale()).createColumnConst(
block[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.getByPosition(result).column = ResultDataType().createColumnConst(col_left_const->size(), toField(res));
block[result].column = ResultDataType().createColumnConst(col_left_const->size(), toField(res));
return true;
}
@ -1050,7 +1050,7 @@ public:
return false;
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
return false;
@ -1059,14 +1059,14 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
/// Special case when multiply aggregate function state
if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type))
if (isAggregateMultiply(block[arguments[0]].type, block[arguments[1]].type))
{
executeAggregateMultiply(block, arguments, result, input_rows_count);
return;
}
/// Special case - addition of two aggregate functions states
if (isAggregateAddition(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type))
if (isAggregateAddition(block[arguments[0]].type, block[arguments[1]].type))
{
executeAggregateAddition(block, arguments, result, input_rows_count);
return;
@ -1074,14 +1074,14 @@ public:
/// 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.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type, context))
= getFunctionForIntervalArithmetic(block[arguments[0]].type, block[arguments[1]].type, context))
{
executeDateTimeIntervalPlusMinus(block, arguments, result, input_rows_count, function_builder);
return;
}
const auto & left_argument = block.getByPosition(arguments[0]);
const auto & right_argument = block.getByPosition(arguments[1]);
const auto & left_argument = block[arguments[0]];
const auto & right_argument = block[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)
@ -1197,23 +1197,23 @@ public:
{
ColumnsWithTypeAndName block_with_constant
= {{left.column->cloneResized(input_rows_count), left.type, left.name},
block.getByPosition(arguments[0]),
block.getByPosition(result)};
block[arguments[0]],
block[result]};
FunctionArguments args(block_with_constant);
Base::executeImpl(args, {0, 1}, 2, input_rows_count);
block.getByPosition(result) = block_with_constant[2];
block[result] = block_with_constant[2];
}
else if (right.column && isColumnConst(*right.column) && arguments.size() == 1)
{
ColumnsWithTypeAndName block_with_constant
= {block.getByPosition(arguments[0]),
= {block[arguments[0]],
{right.column->cloneResized(input_rows_count), right.type, right.name},
block.getByPosition(result)};
block[result]};
FunctionArguments args(block_with_constant);
Base::executeImpl(args, {0, 1}, 2, input_rows_count);
block.getByPosition(result) = block_with_constant[2];
block[result] = block_with_constant[2];
}
else
Base::executeImpl(block, arguments, result, input_rows_count);

View File

@ -56,7 +56,7 @@ public:
void executeImpl(Block & block , const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const auto value_col = block.getByPosition(arguments.front()).column.get();
const auto value_col = block[arguments.front()].column.get();
if (!execute<UInt8>(block, arguments, result, value_col)
&& !execute<UInt16>(block, arguments, result, value_col)
@ -98,7 +98,7 @@ private:
out[i] = Impl::apply(val[i], mask[i]);
}
block.getByPosition(result).column = std::move(out_col);
block[result].column = std::move(out_col);
return true;
}
else if (const auto value_col_const = checkAndGetColumnConst<ColumnVector<T>>(value_col_untyped))
@ -110,7 +110,7 @@ private:
if (is_const)
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, toField(Impl::apply(val, const_mask)));
block[result].column = block[result].type->createColumnConst(size, toField(Impl::apply(val, const_mask)));
}
else
{
@ -122,7 +122,7 @@ private:
for (const auto i : ext::range(0, size))
out[i] = Impl::apply(val, mask[i]);
block.getByPosition(result).column = std::move(out_col);
block[result].column = std::move(out_col);
}
return true;
@ -139,7 +139,7 @@ private:
for (const auto i : ext::range(1, arguments.size()))
{
if (auto pos_col_const = checkAndGetColumnConst<ColumnVector<ValueType>>(block.getByPosition(arguments[i]).column.get()))
if (auto pos_col_const = checkAndGetColumnConst<ColumnVector<ValueType>>(block[arguments[i]].column.get()))
{
const auto pos = pos_col_const->getUInt(0);
if (pos < 8 * sizeof(ValueType))
@ -162,7 +162,7 @@ private:
for (const auto i : ext::range(1, arguments.size()))
{
const auto pos_col = block.getByPosition(arguments[i]).column.get();
const auto pos_col = block[arguments[i]].column.get();
if (!addToMaskImpl<UInt8>(mask, pos_col)
&& !addToMaskImpl<UInt16>(mask, pos_col)

View File

@ -97,7 +97,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
WhichDataType which(from_type);
if (which.isDate())
@ -114,7 +114,7 @@ public:
}
else
throw Exception(
"Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
"Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -314,14 +314,14 @@ struct DateTimeAddIntervalImpl
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block.data, arguments, 2, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
const ColumnPtr source_col = block[arguments[0]].column;
auto result_col = block.getByPosition(result).type->createColumn();
auto result_col = block[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.getByPosition(arguments[1]).column;
const IColumn & delta_column = *block[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);
@ -333,16 +333,16 @@ struct DateTimeAddIntervalImpl
op.constantVector(
sources_const->template getValue<FromValueType>(),
col_to->getData(),
*block.getByPosition(arguments[1]).column, time_zone);
*block[arguments[1]].column, time_zone);
}
else
{
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}
block.getByPosition(result).column = std::move(result_col);
block[result].column = std::move(result_col);
}
};
@ -464,7 +464,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
WhichDataType which(from_type);
if (which.isDate())
@ -483,7 +483,7 @@ public:
Transform{datetime64_type->getScale()}, block, arguments, result);
}
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of first argument of function " + getName(),
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};

View File

@ -96,7 +96,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
WhichDataType which(from_type);
if (which.isDate())
@ -110,7 +110,7 @@ public:
DateTimeTransformImpl<DataTypeDateTime64, ToDataType, decltype(transformer)>::execute(block, arguments, result, input_rows_count, transformer);
}
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -36,7 +36,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
block[result].column = block[result].type->createColumnConst(
input_rows_count, getFQDNOrHostName())->convertToFullColumnIfConst();
}
};

View File

@ -22,10 +22,10 @@ void ExecutableFunctionJoinGet<or_null>::execute(Block & block, const ColumnNumb
ColumnsWithTypeAndName keys;
for (size_t i = 2; i < arguments.size(); ++i)
{
auto key = block.getByPosition(arguments[i]);
auto key = block[arguments[i]];
keys.emplace_back(std::move(key));
}
block.getByPosition(result) = join->joinGet(keys, result_block);
block[result] = join->joinGet(keys, result_block);
}
template <bool or_null>

View File

@ -95,7 +95,7 @@ private:
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
}
block.getByPosition(result).column = std::move(dst);
block[result].column = std::move(dst);
return true;
}
@ -157,7 +157,7 @@ private:
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
}
block.getByPosition(result).column = std::move(dst);
block[result].column = std::move(dst);
return true;
}
if (const auto right_arg_typed = checkAndGetColumnConst<ColumnVector<RightType>>(right_arg))
@ -200,7 +200,7 @@ private:
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
}
block.getByPosition(result).column = std::move(dst);
block[result].column = std::move(dst);
return true;
}
@ -209,8 +209,8 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnWithTypeAndName & col_left = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & col_right = block.getByPosition(arguments[1]);
const ColumnWithTypeAndName & col_left = block[arguments[0]];
const ColumnWithTypeAndName & col_right = block[arguments[1]];
auto call = [&](const auto & types) -> bool
{

View File

@ -27,7 +27,7 @@ private:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Impl::value);
block[result].column = block[result].type->createColumnConst(input_rows_count, Impl::value);
}
};

View File

@ -124,7 +124,7 @@ private:
executeInIterations(src_data.data(), dst_data.data(), size);
block.getByPosition(result).column = std::move(dst);
block[result].column = std::move(dst);
return true;
}
@ -144,7 +144,7 @@ private:
executeInIterations(dst_data.data(), dst_data.data(), size);
block.getByPosition(result).column = std::move(dst);
block[result].column = std::move(dst);
return true;
}
@ -152,7 +152,7 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & col = block[arguments[0]];
auto call = [&](const auto & types) -> bool
{

View File

@ -47,7 +47,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const auto in = block.getByPosition(arguments.front()).column.get();
const auto in = block[arguments.front()].column.get();
if ( !execute<UInt8>(block, in, result)
&& !execute<UInt16>(block, in, result)
@ -77,7 +77,7 @@ public:
for (const auto i : ext::range(0, size))
out_data[i] = Impl::execute(in_data[i]);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
return true;
}

View File

@ -64,8 +64,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get();
const IColumn * needle_column = block.getByPosition(arguments[1]).column.get();
const IColumn * haystack_column = block[arguments[0]].column.get();
const IColumn * needle_column = block[arguments[1]].column.get();
auto col_res = ColumnVector<UInt8>::create();
typename ColumnVector<UInt8>::Container & vec_res = col_res->getData();
@ -83,7 +83,7 @@ public:
else
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
private:

View File

@ -51,7 +51,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column = block[arguments[0]].column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
auto col_res = ColumnVector<ResultType>::create();
@ -60,7 +60,7 @@ public:
vec_res.resize(col->size());
Impl::vector(col->getChars(), col->getOffsets(), vec_res);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column.get()))
{
@ -69,7 +69,7 @@ public:
ResultType res = 0;
Impl::vectorFixedToConstant(col_fixed->getChars(), col_fixed->getN(), res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_fixed->size(), toField(res));
block[result].column = block[result].type->createColumnConst(col_fixed->size(), toField(res));
}
else
{
@ -79,7 +79,7 @@ public:
vec_res.resize(col_fixed->size());
Impl::vectorFixedToVector(col_fixed->getChars(), col_fixed->getN(), vec_res);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
}
else if (const ColumnArray * col_arr = checkAndGetColumn<ColumnArray>(column.get()))
@ -90,10 +90,10 @@ public:
vec_res.resize(col_arr->size());
Impl::array(col_arr->getOffsets(), vec_res);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
throw Exception("Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};

View File

@ -54,15 +54,15 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr column_src = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const ColumnPtr column_replacement = block.getByPosition(arguments[2]).column;
const ColumnPtr column_src = block[arguments[0]].column;
const ColumnPtr column_needle = block[arguments[1]].column;
const ColumnPtr column_replacement = block[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.getByPosition(arguments[1]).column.get();
const IColumn * c2 = block.getByPosition(arguments[2]).column.get();
const IColumn * c1 = block[arguments[1]].column.get();
const IColumn * c2 = block[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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};

View File

@ -53,22 +53,22 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column = block[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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};

View File

@ -156,7 +156,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
bool valid = castType(block.getByPosition(arguments[0]).type.get(), [&](const auto & type)
bool valid = castType(block[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.getByPosition(arguments[0]).column.get()))
if (auto col = checkAndGetColumn<ColumnFixedString>(block[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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(arguments[0]).column.get()))
if (auto col = checkAndGetColumn<ColumnDecimal<T0>>(block[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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(arguments[0]).column.get()))
if (auto col = checkAndGetColumn<ColumnVector<T0>>(block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
}

View File

@ -70,8 +70,8 @@ public:
using SourceColumnType = typename SourceDataType::ColumnType;
using ResultColumnType = typename ResultDataType::ColumnType;
const auto & src = block.getByPosition(arguments[0]);
auto & res = block.getByPosition(result);
const auto & src = block[arguments[0]];
auto & res = block[result];
const auto & col = *src.column;
const SourceColumnType * source_col_typed = checkAndGetColumn<SourceColumnType>(col);

View File

@ -124,7 +124,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
auto array_type = typeid_cast<const DataTypeArray *>(from_type);
auto nested_type = array_type->getNestedType();
@ -149,7 +149,7 @@ private:
void executeBitmapData(Block & block, DataTypes & argument_types, const ColumnNumbers & arguments, size_t result) const
{
// input data
const ColumnArray * array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * array = typeid_cast<const ColumnArray *>(block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};
@ -210,14 +210,14 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
// input data
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[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.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
if (which.isUInt8())
@ -232,7 +232,7 @@ public:
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
block.getByPosition(result).column = std::move(res_ptr);
block[result].column = std::move(res_ptr);
}
private:
@ -244,7 +244,7 @@ private:
const
{
const ColumnAggregateFunction * column
= typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[0]).column.get());
= typeid_cast<const ColumnAggregateFunction *>(block[arguments[0]].column.get());
PaddedPODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
ColumnArray::Offset res_offset = 0;
@ -301,7 +301,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
if (which.isUInt8())
@ -333,7 +333,7 @@ private:
for (size_t i = 0; i < 3; ++i)
{
columns[i] = block.getByPosition(arguments[i]).column.get();
columns[i] = block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};
@ -437,7 +437,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
if (which.isUInt8())
@ -468,7 +468,7 @@ private:
for (size_t i = 0; i < 3; ++i)
{
columns[i] = block.getByPosition(arguments[i]).column.get();
columns[i] = block[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.getByPosition(arguments[1]).column.get());
array = typeid_cast<const ColumnArray *>(block[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.getByPosition(arguments[2]).column.get());
array = typeid_cast<const ColumnArray *>(block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};
@ -572,7 +572,7 @@ public:
{
auto col_to = ColumnVector<ToType>::create(input_rows_count);
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
@ -588,7 +588,7 @@ public:
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
private:
@ -599,7 +599,7 @@ private:
Block & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to) const
{
const ColumnAggregateFunction * column
= typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[0]).column.get());
= typeid_cast<const ColumnAggregateFunction *>(block[arguments[0]].column.get());
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateFunctionGroupBitmapData<T> & bitmap_data
@ -742,7 +742,7 @@ public:
{
auto col_to = ColumnVector<UInt8>::create(input_rows_count);
typename ColumnVector<UInt8>::Container & vec_to = col_to->getData();
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
@ -758,7 +758,7 @@ public:
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
private:
@ -773,7 +773,7 @@ private:
for (size_t i = 0; i < 2; ++i)
{
columns[i] = block.getByPosition(arguments[i]).column.get();
columns[i] = block[arguments[i]].column.get();
is_column_const[i] = isColumnConst(*columns[i]);
}
if (is_column_const[0])
@ -839,7 +839,7 @@ public:
{
auto col_to = ColumnVector<ToType>::create(input_rows_count);
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
@ -855,7 +855,7 @@ public:
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
private:
@ -867,14 +867,14 @@ private:
bool is_column_const[2];
for (size_t i = 0; i < 2; ++i)
{
if (auto argument_column_const = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[i]).column.get()))
if (auto argument_column_const = checkAndGetColumn<ColumnConst>(block[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.getByPosition(arguments[i]).column.get());
columns[i] = typeid_cast<const ColumnAggregateFunction*>(block[arguments[i]].column.get());
is_column_const[i] = false;
}
}
@ -972,7 +972,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
if (which.isUInt8())
@ -996,14 +996,14 @@ private:
bool is_column_const[2];
for (size_t i = 0; i < 2; ++i)
{
if (auto argument_column_const = typeid_cast<const ColumnConst *>(block.getByPosition(arguments[i]).column.get()))
if (auto argument_column_const = typeid_cast<const ColumnConst *>(block[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.getByPosition(arguments[i]).column.get());
columns[i] = typeid_cast<const ColumnAggregateFunction *>(block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};

View File

@ -90,7 +90,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const auto & col_type_name = block.getByPosition(arguments[0]);
const auto & col_type_name = block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -170,12 +170,12 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const auto & col_type_name = block.getByPosition(arguments[0]);
const auto & col_type_name = block[arguments[0]];
const ColumnPtr & column = col_type_name.column;
const auto & col_ipv6_zeroed_tail_bytes_type = block.getByPosition(arguments[1]);
const auto & col_ipv6_zeroed_tail_bytes_type = block[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.getByPosition(arguments[2]);
const auto & col_ipv4_zeroed_tail_bytes_type = block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -279,7 +279,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
if (const auto col_in = checkAndGetColumn<ColumnString>(column.get()))
{
@ -301,10 +301,10 @@ public:
src_offset = offsets_src[i];
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -341,7 +341,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
if (const ColumnUInt32 * col = typeid_cast<const ColumnUInt32 *>(column.get()))
{
@ -365,10 +365,10 @@ public:
vec_res.resize(pos - begin);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -409,7 +409,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
@ -428,10 +428,10 @@ public:
prev_offset = offsets_src[i];
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -462,7 +462,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const auto & col_type_name = block.getByPosition(arguments[0]);
const auto & col_type_name = block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -580,7 +580,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(column.get()))
{
@ -602,10 +602,10 @@ public:
offsets_res[i] = current_offset;
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -690,7 +690,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
@ -716,10 +716,10 @@ public:
prev_offset = current_offset;
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -757,7 +757,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & col_type_name = block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -859,7 +859,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & col_type_name = block[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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
}
};
@ -1189,8 +1189,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const IColumn * column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;
const IColumn * column = block[arguments[0]].column.get();
ColumnPtr & res_column = block[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.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -1257,7 +1257,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
@ -1290,11 +1290,11 @@ public:
out_vec.resize(pos - begin);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
{
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -1355,7 +1355,7 @@ public:
for (size_t idx = 0; idx < arguments.size(); ++idx)
{
//partial const column
columns_holder[idx] = block.getByPosition(arguments[idx]).column->convertToFullColumnIfConst();
columns_holder[idx] = block[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.getByPosition(arguments[idx]).column->getName()
throw Exception{"Illegal column " + block[arguments[idx]].column->getName()
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
}
block.getByPosition(result).column = std::move(col_str);
block[result].column = std::move(col_str);
}
private:
@ -1463,8 +1463,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const IColumn * in_column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & out_column = block.getByPosition(result).column;
const IColumn * in_column = block[arguments[0]].column.get();
ColumnPtr & out_column = block[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.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -1601,13 +1601,13 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const IColumn * column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;
const IColumn * column = block[arguments[0]].column.get();
ColumnPtr & res_column = block[result].column;
if (tryExecuteFixedString(column, res_column) || tryExecuteString(column, res_column))
return;
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -1670,13 +1670,13 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & col_type_name_ip = block.getByPosition(arguments[0]);
const auto & col_type_name_ip = block[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.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[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.getByPosition(arguments[1]);
const auto & col_type_name_cidr = block[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.getByPosition(arguments[1]).column->getName()
throw Exception("Illegal column " + block[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.getByPosition(result).column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
block[result].column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
}
};
@ -1774,23 +1774,23 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & col_type_name_ip = block.getByPosition(arguments[0]);
const auto & col_type_name_ip = block[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.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
const auto & col_type_name_cidr = block.getByPosition(arguments[1]);
const auto & col_type_name_cidr = block[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.getByPosition(arguments[1]).column->getName()
throw Exception("Illegal column " + block[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.getByPosition(result).column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
block[result].column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
}
};

View File

@ -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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
@ -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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(result).column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
block[result].column = DataTypeUInt8().createColumnConst(col_left->size(), toField(res));
return true;
}
@ -763,7 +763,7 @@ private:
if (!res)
return false;
block.getByPosition(result).column = ColumnConst::create(block.getByPosition(result).column, c0_const->size());
block[result].column = ColumnConst::create(block[result].column, c0_const->size());
return true;
}
else
@ -818,7 +818,7 @@ private:
+ " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(c_res);
block[result].column = std::move(c_res);
return true;
}
}
@ -846,7 +846,7 @@ private:
/// If not possible to convert, comparison with =, <, >, <=, >= yields to false and comparison with != yields to true.
if (converted.isNull())
{
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, IsOperation<Op>::not_equals);
block[result].column = DataTypeUInt8().createColumnConst(input_rows_count, IsOperation<Op>::not_equals);
}
else
{
@ -856,13 +856,13 @@ private:
{
{ left_const ? column_converted : col_left_untyped->getPtr(), type_to_compare, "" },
{ !left_const ? column_converted : col_right_untyped->getPtr(), type_to_compare, "" },
block.getByPosition(result)
block[result]
};
FunctionArguments tmp_block(tmp_block_columns);
executeImpl(tmp_block, {0, 1}, 2, input_rows_count);
block.getByPosition(result).column = std::move(tmp_block.getByPosition(2).column);
block[result].column = std::move(tmp_block[2].column);
}
return true;
@ -893,7 +893,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.getByPosition(result);
auto & res = block[result];
if (res.type->onlyNull())
{
res.column = res.type->createColumnConstWithDefaultValue(input_rows_count);
@ -967,7 +967,7 @@ private:
if (tuple_size == 1)
{
/// Do not call AND for single-element tuple.
block.getByPosition(result).column = tmp_block[2].column;
block[result].column = tmp_block[2].column;
return;
}
@ -981,7 +981,7 @@ private:
tmp_block.emplace_back(ColumnWithTypeAndName{ nullptr, impl->getReturnType(), "" });
impl->execute(tmp_block, convolution_args, tuple_size * 3, input_rows_count);
block.getByPosition(result).column = tmp_block[tuple_size * 3].column;
block[result].column = tmp_block[tuple_size * 3].column;
}
void executeTupleLessGreaterImpl(
@ -1064,7 +1064,7 @@ private:
}
block.getByPosition(result).column = tmp_block[tmp_block.size() - 1].column;
block[result].column = tmp_block[tmp_block.size() - 1].column;
}
void executeGenericIdenticalTypes(Block & block, size_t result, const IColumn * c0, const IColumn * c1) const
@ -1076,7 +1076,7 @@ private:
{
UInt8 res = 0;
GenericComparisonImpl<Op<int, int>>::constantConstant(*c0, *c1, res);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(c0->size(), toField(res));
block[result].column = DataTypeUInt8().createColumnConst(c0->size(), toField(res));
}
else
{
@ -1091,7 +1091,7 @@ private:
else
GenericComparisonImpl<Op<int, int>>::vectorVector(*c0, *c1, vec_res);
block.getByPosition(result).column = std::move(c_res);
block[result].column = std::move(c_res);
}
}
@ -1176,8 +1176,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & col_with_type_and_name_left = block.getByPosition(arguments[0]);
const auto & col_with_type_and_name_right = block.getByPosition(arguments[1]);
const auto & col_with_type_and_name_left = block[arguments[0]];
const auto & col_with_type_and_name_right = block[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();
@ -1195,12 +1195,12 @@ public:
|| IsOperation<Op>::less_or_equals
|| IsOperation<Op>::greater_or_equals)
{
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 1u);
block[result].column = DataTypeUInt8().createColumnConst(input_rows_count, 1u);
return;
}
else
{
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 0u);
block[result].column = DataTypeUInt8().createColumnConst(input_rows_count, 0u);
return;
}
}

View File

@ -67,7 +67,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
if (isColumnConst(*block.getByPosition(arguments[1]).column))
if (isColumnConst(*block[arguments[1]].column))
executeConstBuckets(block, arguments, result);
else
throw Exception(
@ -95,7 +95,7 @@ private:
void executeConstBuckets(Block & block, const ColumnNumbers & arguments, size_t result) const
{
Field buckets_field = (*block.getByPosition(arguments[1]).column)[0];
Field buckets_field = (*block[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.getByPosition(arguments[0]).column;
const IDataType * hash_type = block.getByPosition(arguments[0]).type.get();
const auto & hash_col = block[arguments[0]].column;
const IDataType * hash_type = block[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.getByPosition(result).column = std::move(res_col);
block[result].column = std::move(res_col);
}
template <typename CurrentHashType>

View File

@ -104,7 +104,7 @@ struct ConvertImpl
static void NO_SANITIZE_UNDEFINED execute(FunctionArguments & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/,
Additions additions [[maybe_unused]] = Additions())
{
const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & named_from = block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
@ -443,7 +443,7 @@ struct ConvertImpl<DataTypeEnum<FieldType>, DataTypeNumber<FieldType>, Name>
{
static void execute(FunctionArguments & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
block[result].column = block[arguments[0]].column;
}
};
@ -456,7 +456,7 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
static void execute(FunctionArguments & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
const auto & col_with_type_and_name = block.getByPosition(arguments[0]);
const auto & col_with_type_and_name = block[arguments[0]];
const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);
const DateLUTImpl * time_zone = nullptr;
@ -495,10 +495,10 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
}
write_buffer.finalize();
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + Name::name,
ErrorCodes::ILLEGAL_COLUMN);
}
@ -510,7 +510,7 @@ struct ConvertImplGenericToString
{
static void execute(FunctionArguments & block, const ColumnNumbers & arguments, size_t result)
{
const auto & col_with_type_and_name = block.getByPosition(arguments[0]);
const auto & col_with_type_and_name = block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};
@ -607,7 +607,7 @@ inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, Read
*/
[[noreturn]] inline void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, FunctionArguments & block, size_t result)
{
const IDataType & to_type = *block.getByPosition(result).type;
const IDataType & to_type = *block[result].type;
WriteBufferFromOwnString message_buf;
message_buf << "Cannot parse string " << quote << String(read_buffer.buffer().begin(), read_buffer.buffer().size())
@ -681,7 +681,7 @@ 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.getByPosition(result).type);
const auto result_type = removeNullable(block[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();
@ -694,7 +694,7 @@ struct ConvertThroughParsing
utc_time_zone = &DateLUT::instance("UTC");
}
const IColumn * col_from = block.getByPosition(arguments[0]).column.get();
const IColumn * col_from = block[arguments[0]].column.get();
const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(col_from);
const ColumnFixedString * col_from_fixed_string = checkAndGetColumn<ColumnFixedString>(col_from);
@ -847,9 +847,9 @@ struct ConvertThroughParsing
}
if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
block.getByPosition(result).column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
block[result].column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
else
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};
@ -867,10 +867,10 @@ struct ConvertImplGenericFromString
{
static void execute(FunctionArguments & block, const ColumnNumbers & arguments, size_t result)
{
const IColumn & col_from = *block.getByPosition(arguments[0]).column;
const IColumn & col_from = *block[arguments[0]].column;
size_t size = col_from.size();
const IDataType & data_type_to = *block.getByPosition(result).type;
const IDataType & data_type_to = *block[result].type;
if (const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(&col_from))
{
@ -897,10 +897,10 @@ struct ConvertImplGenericFromString
current_offset = offsets[i];
}
block.getByPosition(result).column = std::move(res);
block[result].column = std::move(res);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of conversion function from string",
ErrorCodes::ILLEGAL_COLUMN);
}
@ -922,7 +922,7 @@ struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
{
static void execute(FunctionArguments & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
block[result].column = block[arguments[0]].column;
}
};
@ -935,7 +935,7 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
{
static void execute(FunctionArguments & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(block[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
@ -964,10 +964,10 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
}
data_to.resize(offset_to);
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + Name::name,
ErrorCodes::ILLEGAL_COLUMN);
}
@ -1144,8 +1144,8 @@ public:
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
{
e.addMessage("Cannot parse "
+ block.getByPosition(result).type->getName() + " from "
+ block.getByPosition(arguments[0]).type->getName()
+ block[result].type->getName() + " from "
+ block[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.getByPosition(result).type->getName() + " from "
+ block.getByPosition(arguments[0]).type->getName());
+ block[result].type->getName() + " from "
+ block[arguments[0]].type->getName());
}
throw;
@ -1183,7 +1183,7 @@ private:
throw Exception{"Function " + getName() + " expects at least 1 arguments",
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
auto call = [&](const auto & types) -> bool
{
@ -1206,14 +1206,14 @@ private:
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
}
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
const ColumnWithTypeAndName & scale_column = block[arguments[1]];
UInt32 scale = extractToDecimalScale(scale_column);
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, 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.getByPosition(arguments[0]).type.get());
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());
}
else if constexpr (IsDataTypeDecimalOrNumber<LeftDataType> && IsDataTypeDecimalOrNumber<RightDataType>)
@ -1244,13 +1244,13 @@ private:
if (isDateTime64<Name, ToDataType>(block.getColumnsWithTypeAndName(), arguments))
{
/// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
const ColumnWithTypeAndName & scale_column = block[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.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return;
@ -1266,7 +1266,7 @@ private:
ConvertImplGenericToString::execute(block, arguments, result);
}
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
@ -1396,7 +1396,7 @@ public:
template <typename ConvertToDataType>
bool executeInternal(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, UInt32 scale = 0) const
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
if (checkAndGetDataType<DataTypeString>(from_type))
{
@ -1420,7 +1420,7 @@ public:
if constexpr (to_decimal)
ok = executeInternal<ToDataType>(block, arguments, result, input_rows_count,
assert_cast<const ToDataType &>(*removeNullable(block.getByPosition(result).type)).getScale());
assert_cast<const ToDataType &>(*removeNullable(block[result].type)).getScale());
else
{
if (isDateTime64<Name, ToDataType>(block.getColumnsWithTypeAndName(), arguments))
@ -1443,7 +1443,7 @@ public:
}
if (!ok)
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
throw Exception("Illegal type " + block[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);
@ -2088,7 +2088,7 @@ private:
return [nested_function, from_nested_type, to_nested_type](
Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
const auto & array_arg = block.getByPosition(arguments.front());
const auto & array_arg = block[arguments.front()];
if (const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(array_arg.column.get()))
{
@ -2104,7 +2104,7 @@ private:
nested_function(nested_block, {0}, 1, nested_block_columns.front().column->size());
/// set converted nested column to result
block.getByPosition(result).column = ColumnArray::create(nested_block.getByPosition(1).column, col_array->getOffsetsPtr());
block[result].column = ColumnArray::create(nested_block[1].column, col_array->getOffsetsPtr());
}
else
throw Exception{"Illegal column " + array_arg.column->getName() + " for function CAST AS Array", ErrorCodes::LOGICAL_ERROR};
@ -2143,7 +2143,7 @@ private:
return [element_wrappers, from_element_types, to_element_types]
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
const auto col = block.getByPosition(arguments.front()).column.get();
const auto col = block[arguments.front()].column.get();
/// copy tuple elements to a separate block
ColumnsWithTypeAndName element_block_columns;
@ -2171,9 +2171,9 @@ private:
Columns converted_columns(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
converted_columns[i] = element_block.getByPosition(tuple_size + i).column;
converted_columns[i] = element_block[tuple_size + i].column;
block.getByPosition(result).column = ColumnTuple::create(converted_columns);
block[result].column = ColumnTuple::create(converted_columns);
};
}
@ -2239,9 +2239,9 @@ private:
const char * function_name = name;
return [function_name, source_is_nullable] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
const auto first_col = block.getByPosition(arguments.front()).column.get();
const auto first_col = block[arguments.front()].column.get();
auto & col_with_type_and_name = block.getByPosition(result);
auto & col_with_type_and_name = block[result];
const auto & result_type = typeid_cast<const EnumType &>(*col_with_type_and_name.type);
const ColumnStringType * col = typeid_cast<const ColumnStringType *>(first_col);
@ -2252,7 +2252,7 @@ private:
throw Exception("Not enough columns", ErrorCodes::LOGICAL_ERROR);
size_t nullable_pos = block.columns() - 1;
nullable_col = typeid_cast<const ColumnNullable *>(block.getByPosition(nullable_pos).column.get());
nullable_col = typeid_cast<const ColumnNullable *>(block[nullable_pos].column.get());
if (col && nullable_col && nullable_col->size() != col->size())
throw Exception("ColumnNullable is not compatible with original", ErrorCodes::LOGICAL_ERROR);
}
@ -2291,7 +2291,7 @@ private:
{
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
block[result].column = block[arguments.front()].column;
};
}
@ -2301,7 +2301,7 @@ private:
return [res] (Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
{
/// Column of Nothing type is trivially convertible to any other column
block.getByPosition(result).column = res->cloneResized(input_rows_count)->convertToFullColumnIfConst();
block[result].column = res->cloneResized(input_rows_count)->convertToFullColumnIfConst();
};
}
@ -2319,7 +2319,7 @@ private:
return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
{
auto & res = block.getByPosition(result);
auto & res = block[result];
res.column = res.type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst();
};
}
@ -2337,8 +2337,8 @@ private:
return [wrapper, from_low_cardinality, to_low_cardinality, skip_not_null_check]
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
auto & arg = block.getByPosition(arguments[0]);
auto & res = block.getByPosition(result);
auto & arg = block[arguments[0]];
auto & res = block[result];
ColumnPtr res_indexes;
/// For some types default can't be casted (for example, String to Int). In that case convert column to full.
@ -2420,7 +2420,7 @@ private:
(Block & block, 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.getByPosition(result);
auto & res = block[result];
const auto & ret_type = res.type;
const auto & nullable_type = static_cast<const DataTypeNullable &>(*ret_type);
const auto & nested_type = nullable_type.getNestedType();
@ -2438,7 +2438,7 @@ private:
{
if (arguments.size() != 1)
throw Exception("Invalid number of arguments", ErrorCodes::LOGICAL_ERROR);
tmp_block_columns.emplace_back(block.getByPosition(arguments.front()));
tmp_block_columns.emplace_back(block[arguments.front()]);
}
FunctionArguments tmp_block(tmp_block_columns);
@ -2446,14 +2446,14 @@ private:
/// Perform the requested conversion.
wrapper(tmp_block, arguments, tmp_res_index, input_rows_count);
const auto & tmp_res = tmp_block.getByPosition(tmp_res_index);
const auto & tmp_res = tmp_block[tmp_res_index];
/// May happen in fuzzy tests. For debug purpose.
if (!tmp_res.column)
throw Exception("Couldn't convert " + block.getByPosition(arguments[0]).type->getName() + " to "
throw Exception("Couldn't convert " + block[arguments[0]].type->getName() + " to "
+ nested_type->getName() + " in " + " prepareRemoveNullable wrapper.", ErrorCodes::LOGICAL_ERROR);
res.column = wrapInNullable(tmp_res.column, {block.getByPosition(arguments[0]), tmp_res}, {0}, 1, input_rows_count);
res.column = wrapInNullable(tmp_res.column, {block[arguments[0]], tmp_res}, {0}, 1, input_rows_count);
};
}
else if (source_is_nullable)
@ -2469,7 +2469,7 @@ private:
/// In that case, correctness will be checked beforehand.
if (!skip_not_null_check)
{
const auto & col = block.getByPosition(arguments[0]).column;
const auto & col = block[arguments[0]].column;
const auto & nullable_col = assert_cast<const ColumnNullable &>(*col);
const auto & null_map = nullable_col.getNullMapData();
@ -2480,7 +2480,7 @@ private:
FunctionArguments tmp_block(tmp_block_columns);
wrapper(tmp_block, arguments, result, input_rows_count);
block.getByPosition(result).column = tmp_block.getByPosition(result).column;
block[result].column = tmp_block[result].column;
};
}
else

View File

@ -190,10 +190,10 @@ public:
if (arguments.size() == 2)
{
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!key_col)
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
throw Exception("Illegal column " + block[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.getByPosition(arguments[0]).column.get()))
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + name,
ErrorCodes::ILLEGAL_COLUMN);
}
@ -286,10 +286,10 @@ public:
if (arguments.size() == 3)
{
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[2]).column.get());
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block[arguments[2]].column.get());
if (!key_col)
throw Exception("Illegal column " + block.getByPosition(arguments[2]).column->getName()
throw Exception("Illegal column " + block[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.getByPosition(arguments[0]).column.get());
const ColumnVector<T> * col_vec2 = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[1]).column.get());
const ColumnConst * col_const1 = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get());
const ColumnConst * col_const2 = checkAndGetColumnConst<ColumnVector<T>>(block.getByPosition(arguments[1]).column.get());
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());
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.getByPosition(result).column = std::move(col_to);
block[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.getByPosition(result).column = std::move(col_to);
block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else if (col_const1 && col_const2)
{
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(col_const1->size(),
block[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.getByPosition(arguments[0]).column->getName()
+ " and " + block.getByPosition(arguments[1]).column->getName()
throw Exception("Illegal columns " + block[arguments[0]].column->getName()
+ " and " + block[arguments[1]].column->getName()
+ " of arguments of function " + name,
ErrorCodes::ILLEGAL_COLUMN);
}
@ -422,10 +422,10 @@ public:
if (arguments.size() == 2)
{
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const ColumnConst * key_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!key_col)
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
throw Exception("Illegal column " + block[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.getByPosition(arguments[0]).column.get()))
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[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.getByPosition(result).column = ColumnArray::create(std::move(col_values), std::move(col_offsets));
block[result].column = ColumnArray::create(std::move(col_values), std::move(col_offsets));
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + name,
ErrorCodes::ILLEGAL_COLUMN);
}
@ -627,17 +627,17 @@ public:
/// If the result language is specified
if (arguments.size() == 2)
{
if (const ColumnConst * col_language = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()))
if (const ColumnConst * col_language = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get()))
language = RegionsNames::getLanguageEnum(col_language->getValue<String>());
else
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
throw Exception("Illegal column " + block[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.getByPosition(arguments[0]).column.get()))
if (const ColumnUInt32 * col_from = typeid_cast<const ColumnUInt32 *>(block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of the first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -172,12 +172,12 @@ private:
*/
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
auto & elem = block[result];
elem.column = elem.type->createColumn();
return;
}
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
auto dict = helper.getDictionary(block[arguments[0]]);
if (!executeDispatchSimple<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatchSimple<DirectDictionary>(block, arguments, result, dict) &&
@ -209,14 +209,14 @@ private:
if (!dict)
return false;
const auto id_col_untyped = block.getByPosition(arguments[1]).column.get();
const auto id_col_untyped = block[arguments[1]].column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
{
const auto & ids = id_col->getData();
auto out = ColumnUInt8::create(ext::size(ids));
dict->has(ids, out->getData());
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
@ -232,7 +232,7 @@ private:
if (!dict)
return false;
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[1]);
const ColumnWithTypeAndName & key_col_with_type = block[arguments[1]];
const ColumnPtr & key_col = key_col_with_type.column;
if (checkColumn<ColumnTuple>(key_col.get()))
@ -242,7 +242,7 @@ private:
auto out = ColumnUInt8::create(key_col_with_type.column->size());
dict->has(key_columns, key_types, out->getData());
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
throw Exception{"Second argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH};
@ -330,12 +330,12 @@ private:
{
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
auto & elem = block[result];
elem.column = elem.type->createColumn();
return;
}
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
auto dict = helper.getDictionary(block[arguments[0]]);
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
@ -372,18 +372,18 @@ private:
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
const auto id_col_untyped = block[arguments[2]].column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
{
auto out = ColumnString::create();
dict->getString(attr_name, id_col->getData(), out.get());
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
@ -403,13 +403,13 @@ private:
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
const ColumnWithTypeAndName & key_col_with_type = block[arguments[2]];
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
@ -420,7 +420,7 @@ private:
auto out = ColumnString::create();
dict->getString(attr_name, key_columns, key_types, out.get());
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
throw Exception{"Third argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH};
@ -440,14 +440,14 @@ private:
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const auto & id_col_untyped = block.getByPosition(arguments[2]).column;
const auto & range_col_untyped = block.getByPosition(arguments[3]).column;
const auto & id_col_untyped = block[arguments[2]].column;
const auto & range_col_untyped = block[arguments[3]].column;
PaddedPODArray<UInt64> id_col_values_storage;
PaddedPODArray<Int64> range_col_values_storage;
@ -456,7 +456,7 @@ private:
auto out = ColumnString::create();
dict->getString(attr_name, id_col_values, range_col_values, out.get());
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
return true;
}
@ -516,12 +516,12 @@ private:
{
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
auto & elem = block[result];
elem.column = elem.type->createColumn();
return;
}
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
auto dict = helper.getDictionary(block[arguments[0]]);
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
@ -553,13 +553,13 @@ private:
if (!dict)
return false;
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
const auto id_col_untyped = block[arguments[2]].column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
@ -575,7 +575,7 @@ private:
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnUInt64 * id_col) const
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
const auto default_col_untyped = block[arguments[3]].column.get();
if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped))
{
@ -583,7 +583,7 @@ private:
auto out = ColumnString::create();
const auto & ids = id_col->getData();
dict->getString(attr_name, ids, default_col, out.get());
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
{
@ -592,7 +592,7 @@ private:
const auto & ids = id_col->getData();
String def = default_col_const->getValue<String>();
dict->getString(attr_name, ids, def, out.get());
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
throw Exception{"Fourth argument of function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN};
@ -603,7 +603,7 @@ private:
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnConst * id_col) const
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
const auto default_col_untyped = block[arguments[3]].column.get();
if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped))
{
@ -615,10 +615,10 @@ private:
{
auto out = ColumnString::create();
dict->getString(attr_name, ids, String(), out.get());
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
block[result].column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
}
else
block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column
block[result].column = block[arguments[3]].column; // reuse the default column
}
else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
{
@ -627,7 +627,7 @@ private:
auto out = ColumnString::create();
String def = default_col_const->getValue<String>();
dict->getString(attr_name, ids, def, out.get());
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
block[result].column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
}
else
throw Exception{"Fourth argument of function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN};
@ -641,13 +641,13 @@ private:
if (!dict)
return false;
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
const ColumnWithTypeAndName & key_col_with_type = block[arguments[2]];
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
@ -656,7 +656,7 @@ private:
auto out = ColumnString::create();
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
const auto default_col_untyped = block[arguments[3]].column.get();
if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped))
{
dict->getString(attr_name, key_columns, key_types, default_col, out.get());
@ -669,7 +669,7 @@ private:
else
throw Exception{"Fourth argument of function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN};
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
return true;
}
@ -858,12 +858,12 @@ private:
{
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
auto & elem = block[result];
elem.column = elem.type->createColumn();
return;
}
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
auto dict = helper.getDictionary(block[arguments[0]]);
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
@ -899,13 +899,13 @@ private:
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
const auto id_col_untyped = block[arguments[2]].column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
{
typename ColVec::MutablePtr out;
@ -916,7 +916,7 @@ private:
const auto & ids = id_col->getData();
auto & data = out->getData();
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
{
@ -926,7 +926,7 @@ private:
{
DecimalPaddedPODArray<Type> data(1, decimal_scale);
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
block.getByPosition(result).column =
block[result].column =
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
id_col_const->size(), toField(data.front(), decimal_scale));
}
@ -934,7 +934,7 @@ private:
{
PaddedPODArray<Type> data(1);
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col_const->size(), toField(data.front()));
block[result].column = DataTypeNumber<Type>().createColumnConst(id_col_const->size(), toField(data.front()));
}
}
else
@ -955,13 +955,13 @@ private:
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
const ColumnWithTypeAndName & key_col_with_type = block[arguments[2]];
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
@ -978,7 +978,7 @@ private:
out = ColVec::create(key_columns.front()->size());
auto & data = out->getData();
DictGetTraits<DataType>::get(dict, attr_name, key_columns, key_types, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
throw Exception{"Third argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH};
@ -998,14 +998,14 @@ private:
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const auto & id_col_untyped = block.getByPosition(arguments[2]).column;
const auto & range_col_untyped = block.getByPosition(arguments[3]).column;
const auto & id_col_untyped = block[arguments[2]].column;
const auto & range_col_untyped = block[arguments[3]].column;
PaddedPODArray<UInt64> id_col_values_storage;
PaddedPODArray<Int64> range_col_values_storage;
@ -1019,7 +1019,7 @@ private:
out = ColVec::create(id_col_untyped->size());
auto & data = out->getData();
DictGetTraits<DataType>::get(dict, attr_name, id_col_values, range_col_values, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
return true;
}
@ -1121,12 +1121,12 @@ private:
{
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
auto & elem = block[result];
elem.column = elem.type->createColumn();
return;
}
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
auto dict = helper.getDictionary(block[arguments[0]]);
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict) &&
@ -1157,13 +1157,13 @@ private:
if (!dict)
return false;
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
const auto id_col_untyped = block[arguments[2]].column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
@ -1179,7 +1179,7 @@ private:
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnUInt64 * id_col) const
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
const auto default_col_untyped = block[arguments[3]].column.get();
if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped))
{
@ -1193,7 +1193,7 @@ private:
auto & data = out->getData();
const auto & defs = default_col->getData();
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, defs, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped))
{
@ -1207,7 +1207,7 @@ private:
auto & data = out->getData();
const auto def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, def, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN};
@ -1218,7 +1218,7 @@ private:
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dict,
const std::string & attr_name, const ColumnConst * id_col) const
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
const auto default_col_untyped = block[arguments[3]].column.get();
if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped))
{
@ -1232,7 +1232,7 @@ private:
{
DecimalPaddedPODArray<Type> data(1, decimal_scale);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, Type(), data);
block.getByPosition(result).column =
block[result].column =
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
id_col->size(), toField(data.front(), decimal_scale));
}
@ -1240,11 +1240,11 @@ private:
{
PaddedPODArray<Type> data(1);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, Type(), data);
block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front()));
block[result].column = DataType().createColumnConst(id_col->size(), toField(data.front()));
}
}
else
block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column
block[result].column = block[arguments[3]].column; // reuse the default column
}
else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped))
{
@ -1256,7 +1256,7 @@ private:
DecimalPaddedPODArray<Type> data(1, decimal_scale);
const auto & def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, def, data);
block.getByPosition(result).column =
block[result].column =
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
id_col->size(), toField(data.front(), decimal_scale));
}
@ -1265,7 +1265,7 @@ private:
PaddedPODArray<Type> data(1);
const auto & def = default_col_const->template getValue<Type>();
DictGetTraits<DataType>::getOrDefault(dict, attr_name, ids, def, data);
block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front()));
block[result].column = DataType().createColumnConst(id_col->size(), toField(data.front()));
}
}
else
@ -1280,13 +1280,13 @@ private:
if (!dict)
return false;
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block[arguments[1]].column.get());
if (!attr_name_col)
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]);
const ColumnWithTypeAndName & key_col_with_type = block[arguments[2]];
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst();
@ -1303,7 +1303,7 @@ private:
out = ColVec::create(rows);
auto & data = out->getData();
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
const auto default_col_untyped = block[arguments[3]].column.get();
if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped))
{
/// const defaults
@ -1320,7 +1320,7 @@ private:
else
throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN};
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
return true;
}
@ -1696,12 +1696,12 @@ private:
{
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
auto & elem = block[result];
elem.column = elem.type->createColumn();
return;
}
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
auto dict = helper.getDictionary(block[arguments[0]]);
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict) &&
!executeDispatch<DirectDictionary>(block, arguments, result, dict) &&
@ -1781,14 +1781,14 @@ private:
}
};
const auto id_col_untyped = block.getByPosition(arguments[1]).column.get();
const auto id_col_untyped = block[arguments[1]].column.get();
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
{
const auto & in = id_col->getData();
auto backend = ColumnUInt64::create();
auto offsets = ColumnArray::ColumnOffsets::create();
get_hierarchies(in, backend->getData(), offsets->getData());
block.getByPosition(result).column = ColumnArray::create(std::move(backend), std::move(offsets));
block[result].column = ColumnArray::create(std::move(backend), std::move(offsets));
}
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
{
@ -1797,7 +1797,7 @@ private:
auto offsets = ColumnArray::ColumnOffsets::create();
get_hierarchies(in, backend->getData(), offsets->getData());
auto array = ColumnArray::create(std::move(backend), std::move(offsets));
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(id_col_const->size(), (*array)[0].get<Array>());
block[result].column = block[result].type->createColumnConst(id_col_const->size(), (*array)[0].get<Array>());
}
else
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
@ -1853,12 +1853,12 @@ private:
{
if (input_rows_count == 0)
{
auto & elem = block.getByPosition(result);
auto & elem = block[result];
elem.column = elem.type->createColumn();
return;
}
auto dict = helper.getDictionary(block.getByPosition(arguments[0]));
auto dict = helper.getDictionary(block[arguments[0]]);
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict)
&& !executeDispatch<DirectDictionary>(block, arguments, result, dict)
@ -1878,8 +1878,8 @@ private:
if (!dict->hasHierarchy())
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
const auto child_id_col_untyped = block.getByPosition(arguments[1]).column.get();
const auto ancestor_id_col_untyped = block.getByPosition(arguments[2]).column.get();
const auto child_id_col_untyped = block[arguments[1]].column.get();
const auto ancestor_id_col_untyped = block[arguments[2]].column.get();
if (const auto child_id_col = checkAndGetColumn<ColumnUInt64>(child_id_col_untyped))
execute(block, result, dict, child_id_col, ancestor_id_col_untyped);
@ -1907,7 +1907,7 @@ private:
data.resize(size);
dict->isInVectorVector(child_ids, ancestor_ids, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
{
@ -1920,7 +1920,7 @@ private:
data.resize(size);
dict->isInVectorConstant(child_ids, ancestor_id, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else
{
@ -1946,7 +1946,7 @@ private:
data.resize(size);
dict->isInConstantVector(child_id, ancestor_ids, data);
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
{
@ -1955,7 +1955,7 @@ private:
UInt8 res = 0;
dict->isInConstantConstant(child_id, ancestor_id, res);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(child_id_col->size(), res);
block[result].column = DataTypeUInt8().createColumnConst(child_id_col->size(), res);
}
else
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()

View File

@ -71,7 +71,7 @@ DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const ColumnsWithTypeAndNam
void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
{
const auto * name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
const auto * name_col = checkAndGetColumnConst<ColumnString>(block[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(Block & block, const ColumnNumbers & arg
columns.reserve(arguments.size());
for (auto arg : ext::range(1, arguments.size()))
{
auto & column = block.getByPosition(arguments[arg]).column;
auto & column = block[arguments[arg]].column;
columns.push_back(column.get());
if (auto full_column = column->convertToFullColumnIfConst())
{
@ -130,7 +130,7 @@ void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arg
res = ColumnNullable::create(res, null_map);
}
block.getByPosition(result).column = res;
block[result].column = res;
}
void registerFunctionsExternalModels(FunctionFactory & factory)

View File

@ -66,7 +66,7 @@ public:
|| executeType<Int16>(block, arguments, result)
|| executeType<Int32>(block, arguments, result)
|| executeType<Int64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -94,7 +94,7 @@ private:
template <typename T>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
@ -115,7 +115,7 @@ private:
}
buf_to.finalize();
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
{
@ -164,7 +164,7 @@ public:
|| executeType<Int64>(block, arguments, result)
|| executeType<Float32>(block, arguments, result)
|| executeType<Float64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -173,7 +173,7 @@ private:
template <typename T>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
@ -194,7 +194,7 @@ private:
}
buf_to.finalize();
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
return true;
}
@ -240,7 +240,7 @@ public:
|| executeType<Int64>(block, arguments, result)
|| executeType<Float32>(block, arguments, result)
|| executeType<Float64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -249,7 +249,7 @@ private:
template <typename T>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) const
{
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
{
auto col_to = ColumnString::create();
@ -270,7 +270,7 @@ private:
}
buf_to.finalize();
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
return true;
}

View File

@ -545,7 +545,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block[arguments[0]].column.get()))
{
auto col_to = ColumnFixedString::create(Impl::length);
@ -566,10 +566,10 @@ public:
current_offset = offsets[i];
}
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else if (
const ColumnFixedString * col_from_fix = checkAndGetColumn<ColumnFixedString>(block.getByPosition(arguments[0]).column.get()))
const ColumnFixedString * col_from_fix = checkAndGetColumn<ColumnFixedString>(block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -608,7 +608,7 @@ private:
{
using ColVecType = std::conditional_t<IsDecimalNumber<FromType>, ColumnDecimal<FromType>, ColumnVector<FromType>>;
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(block.getByPosition(arguments[0]).column.get()))
if (const ColVecType * col_from = checkAndGetColumn<ColVecType>(block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + Name::name,
ErrorCodes::ILLEGAL_COLUMN);
}
@ -649,7 +649,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const IDataType * from_type = block[arguments[0]].type.get();
WhichDataType which(from_type);
if (which.isUInt8()) executeType<UInt8>(block, arguments, result);
@ -665,7 +665,7 @@ public:
else if (which.isDecimal32()) executeType<Decimal32>(block, arguments, result);
else if (which.isDecimal64()) executeType<Decimal64>(block, arguments, result);
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
throw Exception("Illegal type " + block[arguments[0]].type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
@ -1059,11 +1059,11 @@ public:
bool is_first_argument = true;
for (size_t i = 0; i < arguments.size(); ++i)
{
const ColumnWithTypeAndName & col = block.getByPosition(arguments[i]);
const ColumnWithTypeAndName & col = block[arguments[i]];
executeForArgument(col.type.get(), col.column.get(), vec_to, is_first_argument);
}
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};
@ -1224,7 +1224,7 @@ public:
private:
void executeSingleArg(Block & block, const ColumnNumbers & arguments, const size_t result) const
{
const auto col_untyped = block.getByPosition(arguments.front()).column.get();
const auto col_untyped = block[arguments.front()].column.get();
if (const auto col_from = checkAndGetColumn<ColumnString>(col_untyped))
{
@ -1245,22 +1245,22 @@ private:
current_offset = offsets[i];
}
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
throw Exception{"Illegal column " + block[arguments[0]].column->getName() +
" of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
void executeTwoArgs(Block & block, const ColumnNumbers & arguments, const size_t result) const
{
const auto level_col = block.getByPosition(arguments.back()).column.get();
const auto level_col = block[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.getByPosition(arguments.front()).column.get();
const auto col_untyped = block[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.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
throw Exception{"Illegal column " + block[arguments[0]].column->getName() +
" of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
};

View File

@ -16,7 +16,7 @@ std::vector<FunctionJSONHelpers::Move> FunctionJSONHelpers::prepareMoves(const c
moves.reserve(num_index_arguments);
for (const auto i : ext::range(first_index_argument, first_index_argument + num_index_arguments))
{
const auto & column = block.getByPosition(arguments[i]);
const auto & column = block[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(),

View File

@ -57,13 +57,13 @@ public:
public:
static void run(FunctionArguments & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count)
{
MutableColumnPtr to{block.getByPosition(result_pos).type->createColumn()};
MutableColumnPtr to{block[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.getByPosition(arguments[0]);
const auto & first_column = block[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};
@ -131,7 +131,7 @@ public:
if (!added_to_column)
to->insertDefault();
}
block.getByPosition(result_pos).column = std::move(to);
block[result_pos].column = std::move(to);
}
};
@ -196,14 +196,14 @@ private:
}
case MoveType::Index:
{
Int64 index = (*block.getByPosition(arguments[j + 1]).column)[row].get<Int64>();
Int64 index = (*block[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.getByPosition(arguments[j + 1]).column).getDataAt(row)};
key = std::string_view{(*block[arguments[j + 1]].column).getDataAt(row)};
if (!moveToElementByKey<JSONParser>(res_element, key))
return false;
break;
@ -913,7 +913,7 @@ public:
void prepare(const char * function_name, const FunctionArguments & block, const ColumnNumbers &, size_t result_pos)
{
extract_tree = JSONExtractTree<JSONParser>::build(function_name, block.getByPosition(result_pos).type);
extract_tree = JSONExtractTree<JSONParser>::build(function_name, block[result_pos].type);
}
bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &)
@ -954,7 +954,7 @@ public:
void prepare(const char * function_name, const FunctionArguments & block, const ColumnNumbers &, size_t result_pos)
{
const auto & result_type = block.getByPosition(result_pos).type;
const auto & result_type = block[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);

View File

@ -516,9 +516,9 @@ void FunctionAnyArityLogical<Impl, Name>::executeImpl(
{
ColumnRawPtrs args_in;
for (const auto arg_index : arguments)
args_in.push_back(block.getByPosition(arg_index).column.get());
args_in.push_back(block[arg_index].column.get());
auto & result_info = block.getByPosition(result_index);
auto & result_info = block[result_index];
if (result_info.type->isNullable())
executeForTernaryLogicImpl<Impl>(std::move(args_in), result_info, input_rows_count);
else
@ -556,7 +556,7 @@ DataTypePtr FunctionUnaryLogical<Impl, Name>::getReturnTypeImpl(const DataTypes
template <template <typename> class Impl, typename T>
bool functionUnaryExecuteType(FunctionArguments & block, const ColumnNumbers & arguments, size_t result)
{
if (auto col = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
if (auto col = checkAndGetColumn<ColumnVector<T>>(block[arguments[0]].column.get()))
{
auto col_res = ColumnUInt8::create();
@ -564,7 +564,7 @@ bool functionUnaryExecuteType(FunctionArguments & block, const ColumnNumbers & a
vec_res.resize(col->getData().size());
UnaryOperationImpl<T, Impl<T>>::vector(col->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
@ -584,7 +584,7 @@ void FunctionUnaryLogical<Impl, Name>::executeImpl(Block & block, const ColumnNu
|| functionUnaryExecuteType<Impl, Int64>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Float32>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Float64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -40,14 +40,14 @@ public:
DB::Block expr_block;
for (size_t i = 0; i < arguments.size(); ++i)
{
const auto & argument = block.getByPosition(arguments[i]);
const auto & argument = block[arguments[i]];
/// Replace column name with value from argument_names.
expr_block.insert({argument.column, argument.type, signature->argument_names[i]});
}
expression_actions->execute(expr_block);
block.getByPosition(result).column = expr_block.getByName(signature->return_name).column;
block[result].column = expr_block.getByName(signature->return_name).column;
}
bool useDefaultImplementationForNulls() const override { return false; }
@ -140,12 +140,12 @@ public:
}
for (const auto & argument : arguments)
columns.push_back(block.getByPosition(argument));
columns.push_back(block[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.getByPosition(result).column = ColumnFunction::create(input_rows_count, std::move(function_adaptor), columns);
block[result].column = ColumnFunction::create(input_rows_count, std::move(function_adaptor), columns);
}
private:

View File

@ -70,11 +70,11 @@ public:
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr & column_haystack = block[arguments[0]].column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & num_ptr = block.getByPosition(arguments[1]).column;
const ColumnPtr & num_ptr = block[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.getByPosition(arguments[1]).column->getName()
"Illegal column " + block[arguments[1]].column->getName()
+ ". The number is not const or does not fit in UInt32",
ErrorCodes::ILLEGAL_COLUMN);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[2]).column;
const ColumnPtr & arr_ptr = block[arguments[2]].column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[2]).column->getName() + ". The array is not const",
"Illegal column " + block[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.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Illegal column " + block[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
if constexpr (Impl::is_column_array)
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
block[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
else
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
};

View File

@ -77,16 +77,16 @@ public:
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr & column_haystack = block[arguments[0]].column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnPtr & arr_ptr = block[arguments[1]].column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
"Illegal column " + block[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.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Illegal column " + block[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.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
block[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
}
};

View File

@ -79,16 +79,16 @@ public:
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr & column_haystack = block[arguments[0]].column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnPtr & arr_ptr = block[arguments[1]].column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
"Illegal column " + block[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.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Illegal column " + block[arguments[0]].column->getName(), ErrorCodes::ILLEGAL_COLUMN);
if constexpr (Impl::is_column_array)
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
block[result].column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
else
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
};

View File

@ -83,7 +83,7 @@ public:
vec_to.resize(size);
RandImpl::execute(reinterpret_cast<char *>(vec_to.data()), vec_to.size() * sizeof(ToType));
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
};

View File

@ -484,7 +484,7 @@ class Dispatcher
}
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
static void apply(FunctionArguments & block, const ColumnDecimal<T> * col, Int64 scale_arg, size_t result)
@ -497,7 +497,7 @@ class Dispatcher
if (!vec_res.empty())
DecimalRoundingImpl<T, rounding_mode, tie_breaking_mode>::apply(col->getData(), vec_res, scale_arg);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
public:
@ -549,7 +549,7 @@ public:
{
if (arguments.size() == 2)
{
const IColumn & scale_column = *block.getByPosition(arguments[1]).column;
const IColumn & scale_column = *block[arguments[1]].column;
if (!isColumnConst(scale_column))
throw Exception("Scale argument for rounding functions must be constant.", ErrorCodes::ILLEGAL_COLUMN);
@ -568,7 +568,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnWithTypeAndName & column = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & column = block[arguments[0]];
Int64 scale_arg = getScaleArg(block, arguments);
auto call = [&](const auto & types) -> bool
@ -648,21 +648,21 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) const override
{
auto in_column = block.getByPosition(arguments[0]).column;
const auto & in_type = block.getByPosition(arguments[0]).type;
auto in_column = block[arguments[0]].column;
const auto & in_type = block[arguments[0]].type;
auto array_column = block.getByPosition(arguments[1]).column;
const auto & array_type = block.getByPosition(arguments[1]).type;
auto array_column = block[arguments[1]].column;
const auto & array_type = block[arguments[1]].type;
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[result].type;
auto column_result = return_type->createColumn();
auto out = column_result.get();
if (!in_type->equals(*return_type))
in_column = castColumn(block.getByPosition(arguments[0]), return_type);
in_column = castColumn(block[arguments[0]], return_type);
if (!array_type->equals(*return_type))
array_column = castColumn(block.getByPosition(arguments[1]), std::make_shared<DataTypeArray>(return_type));
array_column = castColumn(block[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.getByPosition(result).column = std::move(column_result);
block[result].column = std::move(column_result);
}
private:

View File

@ -138,10 +138,10 @@ public:
void init(FunctionArguments & block, const ColumnNumbers & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get());
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block[arguments[0]].column.get());
if (!col)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
@ -206,10 +206,10 @@ public:
void init(FunctionArguments & block, const ColumnNumbers & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[0]).column.get());
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block[arguments[0]].column.get());
if (!col)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
@ -286,10 +286,10 @@ public:
/// Initialize by the function arguments.
void init(FunctionArguments & block, const ColumnNumbers & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[1]).column.get());
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block[arguments[1]].column.get());
if (!col)
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
throw Exception("Illegal column " + block[arguments[1]].column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
@ -367,9 +367,9 @@ public:
generator.init(block, arguments);
size_t array_argument_position = arguments[generator.getStringsArgumentPosition()];
const ColumnString * col_str = checkAndGetColumn<ColumnString>(block.getByPosition(array_argument_position).column.get());
const ColumnString * col_str = checkAndGetColumn<ColumnString>(block[array_argument_position].column.get());
const ColumnConst * col_const_str =
checkAndGetColumnConstStringOrFixedString(block.getByPosition(array_argument_position).column.get());
checkAndGetColumnConstStringOrFixedString(block[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.getByPosition(result).column = std::move(col_res);
block[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.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_const_str->size(), dst);
block[result].column = block[result].type->createColumnConst(col_const_str->size(), dst);
}
else
throw Exception("Illegal columns " + block.getByPosition(array_argument_position).column->getName()
+ ", " + block.getByPosition(array_argument_position).column->getName()
throw Exception("Illegal columns " + block[array_argument_position].column->getName()
+ ", " + block[array_argument_position].column->getName()
+ " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -541,14 +541,14 @@ public:
String delimiter;
if (arguments.size() == 2)
{
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[1]).column.get());
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(block[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.getByPosition(arguments[0]).column.get()))
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(block[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.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_const_arr->size(), dst_str);
block[result].column = block[result].type->createColumnConst(col_const_arr->size(), dst_str);
}
else
{
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*block.getByPosition(arguments[0]).column);
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
}
};

View File

@ -101,12 +101,12 @@ public:
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr & column_needle = block.getByPosition(arguments[1]).column;
const ColumnPtr & column_haystack = block[arguments[0]].column;
const ColumnPtr & column_needle = block[arguments[1]].column;
ColumnPtr column_start_pos = nullptr;
if (arguments.size() >= 3)
column_start_pos = block.getByPosition(arguments[2]).column;
column_start_pos = block[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.getByPosition(result).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(vec_res[0]));
block[result].column
= block[result].type->createColumnConst(col_haystack_const->size(), toField(vec_res[0]));
else
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return;
}
@ -175,11 +175,11 @@ public:
vec_res);
else
throw Exception(
"Illegal columns " + block.getByPosition(arguments[0]).column->getName() + " and "
+ block.getByPosition(arguments[1]).column->getName() + " of arguments of function " + getName(),
"Illegal columns " + block[arguments[0]].column->getName() + " and "
+ block[arguments[1]].column->getName() + " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
};

View File

@ -60,8 +60,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const ColumnPtr column = block[arguments[0]].column;
const ColumnPtr column_needle = block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};

View File

@ -55,8 +55,8 @@ public:
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr & column_needle = block.getByPosition(arguments[1]).column;
const ColumnPtr & column_haystack = block[arguments[0]].column;
const ColumnPtr & column_needle = block[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.getByPosition(result).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
block[result].column
= block[result].type->createColumnConst(col_haystack_const->size(), toField(res));
return;
}
@ -122,12 +122,12 @@ public:
else
{
throw Exception(
"Illegal columns " + block.getByPosition(arguments[0]).column->getName() + " and "
+ block.getByPosition(arguments[1]).column->getName() + " of arguments of function " + getName(),
"Illegal columns " + block[arguments[0]].column->getName() + " and "
+ block[arguments[1]].column->getName() + " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
};

View File

@ -30,36 +30,13 @@ class FunctionArguments
public:
explicit FunctionArguments(ColumnsWithTypeAndName & arguments) : data(arguments) {}
const ColumnWithTypeAndName & getByPosition(size_t position) const { return data[position]; }
ColumnWithTypeAndName & getByPosition(size_t position) { return data[position]; }
ColumnWithTypeAndName & safeGetByPosition(size_t position)
{
checkPosition(position);
return data[position];
}
const ColumnWithTypeAndName & safeGetByPosition(size_t position) const
{
checkPosition(position);
return data[position];
}
ColumnWithTypeAndName & operator[] (size_t position) { return data[position]; }
const ColumnWithTypeAndName & operator[] (size_t position) const { return data[position]; }
size_t columns() const { return data.size(); }
const ColumnsWithTypeAndName & getColumnsWithTypeAndName() const { return data; }
ColumnsWithTypeAndName & data;
private:
void checkPosition(size_t position) const
{
if (data.empty())
throw Exception("Arguments are empty", ErrorCodes::POSITION_OUT_OF_BOUND);
if (position >= data.size())
throw Exception("Position " + std::to_string(position)
+ " is out of bound in FunctionArguments::safeGetByPosition(), max position = "
+ std::to_string(data.size() - 1), ErrorCodes::POSITION_OUT_OF_BOUND);
}
};
/// Cache for functions result if it was executed on low cardinality column.

View File

@ -51,15 +51,15 @@ private:
size_t num_arguments = arguments.size();
if (1 == num_arguments)
{
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
block[result].column = block[arguments[0]].column;
return;
}
auto result_type = block.getByPosition(result).type;
auto result_type = block[result].type;
Columns converted_columns(num_arguments);
for (size_t arg = 0; arg < num_arguments; ++arg)
converted_columns[arg] = castColumn(block.getByPosition(arguments[arg]), result_type)->convertToFullColumnIfConst();
converted_columns[arg] = castColumn(block[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.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
};

View File

@ -53,13 +53,13 @@ struct FunctionPort : public IFunction
UInt16 default_port = 0;
if (arguments.size() == 2)
{
const auto * port_column = checkAndGetColumn<ColumnConst>(block.getByPosition(arguments[1]).column.get());
const auto * port_column = checkAndGetColumn<ColumnConst>(block[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.getByPosition(arguments[0]).column;
const ColumnPtr url_column = block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -220,7 +220,7 @@ public:
{
if (input_rows_count == 0)
{
block.getByPosition(result).column = ColumnString::create();
block[result].column = ColumnString::create();
return;
}
@ -229,7 +229,7 @@ public:
String dist;
bool higher_is_better;
if (const ColumnConst * col_dist = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnConst * col_dist = checkAndGetColumnConst<ColumnString>(block[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.getByPosition(arguments[1]).column.get()))
if (const ColumnConst * col_higher_is_better = checkAndGetColumnConst<ColumnUInt8>(block[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.getByPosition(arguments[2]).column.get()))
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(block[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.getByPosition(arguments[3]).column.get()))
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(block[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.getByPosition(arguments[4]).column.get()))
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(block[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.getByPosition(result).column = std::move(dst);
block[result].column = std::move(dst);
}
};

View File

@ -74,7 +74,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[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.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
private:

View File

@ -68,7 +68,7 @@ public:
{
const SymbolIndex & symbol_index = SymbolIndex::instance();
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
const ColumnUInt64 * column_concrete = checkAndGetColumn<ColumnUInt64>(column.get());
if (!column_concrete)
@ -85,7 +85,7 @@ public:
result_column->insertDefault();
}
block.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
};

View File

@ -57,8 +57,8 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const auto & column = block.getByPosition(arguments[0]).column;
const auto & column_char = block.getByPosition(arguments[1]).column;
const auto & column = block[arguments[0]].column;
const auto & column_char = block[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.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
else
throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
throw Exception{"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
};

View File

@ -139,7 +139,7 @@ public:
{
if (arguments.size() == 1)
{
ColumnPtr column_array_ptr = block.getByPosition(arguments[0]).column;
ColumnPtr column_array_ptr = block[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.getByPosition(result).column = Impl::execute(*column_array, column_array->getDataPtr());
block[result].column = Impl::execute(*column_array, column_array->getDataPtr());
}
else
{
const auto & column_with_type_and_name = block.getByPosition(arguments[0]);
const auto & column_with_type_and_name = block[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.getByPosition(arguments[i]);
const auto & array_with_type_and_name = block[arguments[i]];
ColumnPtr column_array_ptr = array_with_type_and_name.column;
const auto * column_array = checkAndGetColumn<ColumnArray>(column_array_ptr.get());
@ -229,7 +229,7 @@ public:
if (lambda_result->lowCardinality())
lambda_result = lambda_result->convertToFullColumnIfLowCardinality();
block.getByPosition(result).column = Impl::execute(*column_first_array, lambda_result);
block[result].column = Impl::execute(*column_first_array, lambda_result);
}
}
};

View File

@ -37,11 +37,11 @@ public:
if (num_elements == 0)
{
/// We should return constant empty array.
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
block[result].column = block[result].type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
const DataTypePtr & return_type = block.getByPosition(result).type;
const DataTypePtr & return_type = block[result].type;
const DataTypePtr & elem_type = static_cast<const DataTypeArray &>(*return_type).getNestedType();
size_t block_size = input_rows_count;
@ -57,7 +57,7 @@ public:
for (size_t i = 0; i < num_elements; ++i)
{
const auto & arg = block.getByPosition(arguments[i]);
const auto & arg = block[arguments[i]];
ColumnPtr preprocessed_column = arg.column;
@ -89,7 +89,7 @@ public:
out_offsets[i] = current_offset;
}
block.getByPosition(result).column = std::move(out);
block[result].column = std::move(out);
}
private:

View File

@ -51,11 +51,11 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const DataTypePtr & return_type = block.getByPosition(result).type;
const DataTypePtr & return_type = block[result].type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
block[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.getByPosition(arguments[i]);
const ColumnWithTypeAndName & arg = block[arguments[i]];
ColumnPtr preprocessed_column = arg.column;
if (!arg.type->equals(*return_type))
@ -95,7 +95,7 @@ public:
auto sink = GatherUtils::concat(sources);
block.getByPosition(result).column = std::move(sink);
block[result].column = std::move(sink);
}
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -87,10 +87,10 @@ private:
void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const
{
ColumnPtr array_ptr = block.getByPosition(arguments[0]).column;
ColumnPtr array_ptr = block[arguments[0]].column;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[result].type;
auto res_ptr = return_type->createColumn();
ColumnArray & res = assert_cast<ColumnArray &>(*res_ptr);
@ -127,7 +127,7 @@ void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arg
|| executeString(*inner_col, offsets, res_data, res_offsets, nullable_col)))
executeHashed(*inner_col, offsets, res_data, res_offsets, nullable_col);
block.getByPosition(result).column = std::move(res_ptr);
block[result].column = std::move(res_ptr);
}
template <typename T>

View File

@ -429,7 +429,7 @@ template <typename DataType>
bool FunctionArrayElement::executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
ArrayImpl::NullMapBuilder & builder)
{
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!col_array)
return false;
@ -450,7 +450,7 @@ bool FunctionArrayElement::executeNumberConst(Block & block, const ColumnNumbers
else
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
@ -458,7 +458,7 @@ template <typename IndexType, typename DataType>
bool FunctionArrayElement::executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder)
{
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!col_array)
return false;
@ -473,14 +473,14 @@ bool FunctionArrayElement::executeNumber(Block & block, const ColumnNumbers & ar
ArrayElementNumImpl<DataType>::template vector<IndexType>(
col_nested->getData(), col_array->getOffsets(), indices, col_res->getData(), builder);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
bool FunctionArrayElement::executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
ArrayImpl::NullMapBuilder & builder)
{
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!col_array)
return false;
@ -513,7 +513,7 @@ bool FunctionArrayElement::executeStringConst(Block & block, const ColumnNumbers
else
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
@ -521,7 +521,7 @@ template <typename IndexType>
bool FunctionArrayElement::executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder)
{
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!col_array)
return false;
@ -542,14 +542,14 @@ bool FunctionArrayElement::executeString(Block & block, const ColumnNumbers & ar
col_res->getOffsets(),
builder);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
bool FunctionArrayElement::executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
ArrayImpl::NullMapBuilder & builder)
{
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!col_array)
return false;
@ -566,7 +566,7 @@ bool FunctionArrayElement::executeGenericConst(Block & block, const ColumnNumber
else
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
@ -574,7 +574,7 @@ template <typename IndexType>
bool FunctionArrayElement::executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder)
{
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!col_array)
return false;
@ -585,7 +585,7 @@ bool FunctionArrayElement::executeGeneric(Block & block, const ColumnNumbers & a
ArrayElementGenericImpl::vector<IndexType>(
col_nested, col_array->getOffsets(), indices, *col_res, builder);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
@ -594,12 +594,12 @@ bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arg
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
size_t input_rows_count)
{
const ColumnArray * col_array = checkAndGetColumnConstData<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = checkAndGetColumnConstData<ColumnArray>(block[arguments[0]].column.get());
if (!col_array)
return false;
auto res = block.getByPosition(result).type->createColumn();
auto res = block[result].type->createColumn();
size_t rows = input_rows_count;
const IColumn & array_elements = col_array->getData();
@ -630,7 +630,7 @@ bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arg
}
}
block.getByPosition(result).column = std::move(res);
block[result].column = std::move(res);
return true;
}
@ -638,7 +638,7 @@ template <typename IndexType>
bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers & arguments, size_t result,
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count) const
{
auto index = checkAndGetColumn<ColumnVector<IndexType>>(block.getByPosition(arguments[1]).column.get());
auto index = checkAndGetColumn<ColumnVector<IndexType>>(block[arguments[1]].column.get());
if (!index)
return false;
@ -661,7 +661,7 @@ bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers &
|| 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.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
return true;
@ -669,7 +669,7 @@ bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers &
bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
{
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block[arguments[0]].column.get());
if (!col_array)
return false;
@ -683,7 +683,7 @@ bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arg
size_t tuple_size = tuple_columns.size();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(
*typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType()).getElements();
*typeid_cast<const DataTypeArray &>(*block[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.
@ -696,7 +696,7 @@ bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arg
* ...
*/
ColumnsWithTypeAndName temporary_results;
temporary_results.emplace_back(block.getByPosition(arguments[1]));
temporary_results.emplace_back(block[arguments[1]]);
/// results of taking elements by index for arrays from each element of the tuples;
Columns result_tuple_columns;
@ -719,7 +719,7 @@ bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arg
result_tuple_columns.emplace_back(std::move(temporary_results[i * 2 + 2].column));
}
block.getByPosition(result).column = ColumnTuple::create(result_tuple_columns);
block[result].column = ColumnTuple::create(result_tuple_columns);
return true;
}
@ -757,16 +757,16 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
const ColumnArray * col_array = nullptr;
const ColumnArray * col_const_array = nullptr;
col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
col_array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (col_array)
is_array_of_nullable = isColumnNullable(col_array->getData());
else
{
col_const_array = checkAndGetColumnConstData<ColumnArray>(block.getByPosition(arguments[0]).column.get());
col_const_array = checkAndGetColumnConstData<ColumnArray>(block[arguments[0]].column.get());
if (col_const_array)
is_array_of_nullable = isColumnNullable(col_const_array->getData());
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
}
@ -782,9 +782,9 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
ColumnsWithTypeAndName source_columns;
const DataTypePtr & input_type = typeid_cast<const DataTypeNullable &>(
*typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType()).getNestedType();
*typeid_cast<const DataTypeArray &>(*block[arguments[0]].type).getNestedType()).getNestedType();
DataTypePtr tmp_ret_type = removeNullable(block.getByPosition(result).type);
DataTypePtr tmp_ret_type = removeNullable(block[result].type);
if (col_array)
{
@ -799,7 +799,7 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
std::make_shared<DataTypeArray>(input_type),
""
},
block.getByPosition(arguments[1]),
block[arguments[1]],
{
nullptr,
tmp_ret_type,
@ -822,7 +822,7 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
std::make_shared<DataTypeArray>(input_type),
""
},
block.getByPosition(arguments[1]),
block[arguments[1]],
{
nullptr,
tmp_ret_type,
@ -837,8 +837,8 @@ void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & argu
perform(source_block, {0, 1}, 2, builder, input_rows_count);
/// Store the result.
const ColumnWithTypeAndName & source_col = source_block.getByPosition(2);
ColumnWithTypeAndName & dest_col = block.getByPosition(result);
const ColumnWithTypeAndName & source_col = source_block[2];
ColumnWithTypeAndName & dest_col = block[result];
dest_col.column = ColumnNullable::create(source_col.column, builder ? std::move(builder).getNullMapColumnPtr() : ColumnUInt8::create());
}
}
@ -849,7 +849,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument
if (executeTuple(block, arguments, result, input_rows_count))
{
}
else if (!isColumnConst(*block.getByPosition(arguments[1]).column))
else if (!isColumnConst(*block[arguments[1]].column))
{
if (!(executeArgument<UInt8>(block, arguments, result, builder, input_rows_count)
|| executeArgument<UInt16>(block, arguments, result, builder, input_rows_count)
@ -864,7 +864,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument
}
else
{
Field index = (*block.getByPosition(arguments[1]).column)[0];
Field index = (*block[arguments[1]].column)[0];
if (builder)
builder.initSink(input_rows_count);
@ -884,7 +884,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument
|| executeNumberConst<Float64>(block, arguments, result, index, builder)
|| executeStringConst (block, arguments, result, index, builder)
|| executeGenericConst (block, arguments, result, index, builder)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -47,7 +47,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) const override
{
if (const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get()))
{
const ColumnArray::Offsets & offsets = array->getOffsets();
@ -63,11 +63,11 @@ public:
prev_off = off;
}
block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), array->getOffsetsPtr());
block[result].column = ColumnArray::create(std::move(res_nested), array->getOffsetsPtr());
}
else
{
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -130,14 +130,14 @@ void FunctionArrayEnumerateExtended<Derived>::executeImpl(Block & block, const C
ColumnPtr offsets_column;
for (size_t i = 0; i < num_arguments; ++i)
{
const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column;
const ColumnPtr & array_ptr = block[arguments[i]].column;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
if (!array)
{
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(
block.getByPosition(arguments[i]).column.get());
block[arguments[i]].column.get());
if (!const_array)
throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName()
throw Exception("Illegal column " + block[arguments[i]].column->getName()
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
array_holders.emplace_back(const_array->convertToFullColumn());
@ -200,7 +200,7 @@ void FunctionArrayEnumerateExtended<Derived>::executeImpl(Block & block, const C
executeHashed(*offsets, data_columns, res_values);
}
block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_column);
block[result].column = ColumnArray::create(std::move(res_nested), offsets_column);
}
template <typename Derived>

View File

@ -160,7 +160,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
ColumnsWithTypeAndName args;
for (size_t i = 0; i < arguments.size(); ++i)
args.emplace_back(block.getByPosition(arguments[i]));
args.emplace_back(block[arguments[i]]);
const ArraysDepths arrays_depths = getArraysDepths(args);
@ -185,7 +185,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.getByPosition(arguments[i]).column.get());
const auto * array = get_array_column(block[arguments[i]].column.get());
if (!array)
continue;
@ -257,7 +257,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.getByPosition(result).column = result_nested_array;
block[result].column = result_nested_array;
}
/*

View File

@ -79,10 +79,10 @@ result offsets: 3, 4
result: Row 1: [1, 2, 3], Row2: [4]
*/
const ColumnArray * src_col = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * src_col = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!src_col)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " in argument of function 'arrayFlatten'",
throw Exception("Illegal column " + block[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.getByPosition(result).column = ColumnArray::create(
block[result].column = ColumnArray::create(
prev_data->getPtr(),
result_offsets_column ? std::move(result_offsets_column) : src_col->getOffsetsPtr());
}

View File

@ -397,7 +397,7 @@ public:
*/
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
ColumnPtr& ptr = block.getByPosition(arguments[0]).column;
ColumnPtr& ptr = block[arguments[0]].column;
/**
* The columns here have two general cases, either being Array(T) or Const(Array(T)).
@ -410,7 +410,7 @@ public:
if (col_array)
nullable = checkAndGetColumn<ColumnNullable>(col_array->getData());
auto & arg_column = block.getByPosition(arguments[1]).column;
auto & arg_column = block[arguments[1]].column;
const ColumnNullable * arg_nullable = checkAndGetColumn<ColumnNullable>(*arg_column);
if (!nullable && !arg_nullable)
@ -423,7 +423,7 @@ public:
* {0, 1, 2, 3, 4}
* {data (array) argument, "value" argument, data null map, "value" null map, function result}.
*/
ColumnsWithTypeAndName source_block = { {}, {}, {}, {}, {nullptr, block.getByPosition(result).type, ""} };
ColumnsWithTypeAndName source_block = { {}, {}, {}, {}, {nullptr, block[result].type, ""} };
if (nullable)
{
@ -435,7 +435,7 @@ public:
data.type = std::make_shared<DataTypeArray>(
static_cast<const DataTypeNullable &>(
*static_cast<const DataTypeArray &>(
*block.getByPosition(arguments[0]).type
*block[arguments[0]].type
).getNestedType()
).getNestedType());
@ -447,7 +447,7 @@ public:
else
{
auto & data = source_block[0];
data = block.getByPosition(arguments[0]);
data = block[arguments[0]];
}
if (arg_nullable)
@ -456,7 +456,7 @@ public:
arg.column = arg_nullable->getNestedColumnPtr();
arg.type =
static_cast<const DataTypeNullable &>(
*block.getByPosition(arguments[1]).type
*block[arguments[1]].type
).getNestedType();
auto & null_map = source_block[3];
@ -466,7 +466,7 @@ public:
else
{
auto & arg = source_block[1];
arg = block.getByPosition(arguments[1]);
arg = block[arguments[1]];
}
/// Now perform the function.
@ -475,7 +475,7 @@ public:
/// Move the result to its final position.
const ColumnWithTypeAndName & source_col = source_block[4];
ColumnWithTypeAndName & dest_col = block.getByPosition(result);
ColumnWithTypeAndName & dest_col = block[result];
dest_col.column = std::move(source_col.column);
}
}
@ -497,7 +497,7 @@ private:
NullMaps maps;
ResultColumnPtr result { ResultColumnType::create() };
inline void move_result() { block.getByPosition(result_pos).column = std::move(result); }
inline void move_result() { block[result_pos].column = std::move(result); }
};
static inline bool allowNested(const DataTypePtr & left, const DataTypePtr & right)
@ -581,7 +581,7 @@ private:
void executeOnNonNullable(Block & block, const ColumnNumbers & arguments, size_t result) const
{
if (const auto* const left_arr = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
if (const auto* const left_arr = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get()))
{
if (checkAndGetColumn<ColumnLowCardinality>(&left_arr->getData()))
{
@ -622,10 +622,10 @@ private:
const NullMap * null_map_data = nullptr;
const NullMap * null_map_item = nullptr;
if (const auto & data_map = block.getByPosition(arguments[2]).column; data_map)
if (const auto & data_map = block[arguments[2]].column; data_map)
null_map_data = &assert_cast<const ColumnUInt8 &>(*data_map).getData();
if (const auto & item_map = block.getByPosition(arguments[3]).column; item_map)
if (const auto & item_map = block[arguments[3]].column; item_map)
null_map_item = &assert_cast<const ColumnUInt8 &>(*item_map).getData();
return {null_map_data, null_map_item};
@ -639,12 +639,12 @@ private:
template <class ...Integral>
static inline bool executeIntegral(Block & block, const ColumnNumbers & arguments, size_t result_pos)
{
const ColumnArray * const left = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * const left = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!left)
return false;
const IColumn& right = *block.getByPosition(arguments[1]).column.get();
const IColumn& right = *block[arguments[1]].column.get();
ExecutionData data = {
left->getData(),
@ -730,7 +730,7 @@ private:
static bool executeLowCardinality(Block & block, const ColumnNumbers & arguments, size_t result)
{
const ColumnArray * const col_array = checkAndGetColumn<ColumnArray>(
block.getByPosition(arguments[0]).column.get());
block[arguments[0]].column.get());
if (!col_array)
return false;
@ -742,7 +742,7 @@ private:
const auto [null_map_data, null_map_item] = getNullMaps(block, arguments);
const IColumn& col_arg = *block.getByPosition(arguments[1]).column.get();
const IColumn& col_arg = *block[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.getByPosition(arguments[0]).type.get());
block[arguments[0]].type.get());
const DataTypePtr target_type_ptr = recursiveRemoveLowCardinality(array_type->getNestedType());
const ColumnPtr col_arg_cloned = use_cloned_arg
? castColumn(block.getByPosition(arguments[1]), target_type_ptr)
? castColumn(block[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.getByPosition(result).column = std::move(col_result);
block[result].column = std::move(col_result);
return true;
}
}
@ -795,7 +795,7 @@ private:
null_map_data,
null_map_item);
block.getByPosition(result).column = std::move(col_result);
block[result].column = std::move(col_result);
return true;
}
else if (col_lc->nestedIsNullable()) // LC(Nullable(T)) and U
@ -871,7 +871,7 @@ private:
static bool executeString(Block & block, const ColumnNumbers & arguments, size_t result_pos)
{
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!array)
return false;
@ -881,7 +881,7 @@ private:
if (!left)
return false;
const IColumn & right = *block.getByPosition(arguments[1]).column.get();
const IColumn & right = *block[arguments[1]].column.get();
ExecutionData data = {
*left, right, array->getOffsets(),
@ -957,14 +957,14 @@ private:
static bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result)
{
const ColumnConst * col_array = checkAndGetColumnConst<ColumnArray>(
block.getByPosition(arguments[0]).column.get());
block[arguments[0]].column.get());
if (!col_array)
return false;
Array arr = col_array->getValue<Array>();
const IColumn * item_arg = block.getByPosition(arguments[1]).column.get();
const IColumn * item_arg = block[arguments[1]].column.get();
if (isColumnConst(*item_arg))
{
@ -982,7 +982,7 @@ private:
break;
}
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
block[result].column = block[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.getByPosition(arguments[3]).column; col)
if (const auto & col = block[arguments[3]].column; col)
null_map = &assert_cast<const ColumnUInt8 &>(*col).getData();
const size_t size = item_arg->size();
@ -1025,7 +1025,7 @@ private:
}
}
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
}
return true;
@ -1033,13 +1033,13 @@ private:
static bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result)
{
const ColumnArray * col = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * col = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!col)
return false;
const IColumn & col_nested = col->getData();
const IColumn & item_arg = *block.getByPosition(arguments[1]).column;
const IColumn & item_arg = *block[arguments[1]].column;
auto col_res = ResultColumnType::create();
@ -1067,7 +1067,7 @@ private:
null_map_data,
null_map_item);
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
};

View File

@ -233,7 +233,7 @@ FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns(
for (size_t i = 0; i < num_args; ++i)
{
const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]);
const ColumnWithTypeAndName & arg = block[arguments[i]];
initial_columns[i] = arg;
columns[i] = arg;
auto & column = columns[i];
@ -385,7 +385,7 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(
void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
{
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[result].type;
const auto * return_type_array = checkAndGetDataType<DataTypeArray>(return_type.get());
if (!return_type_array)
@ -395,7 +395,7 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
if (typeid_cast<const DataTypeNothing *>(nested_return_type.get()))
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
@ -403,7 +403,7 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
DataTypes data_types;
data_types.reserve(num_args);
for (size_t i = 0; i < num_args; ++i)
data_types.push_back(block.getByPosition(arguments[i]).type);
data_types.push_back(block[arguments[i]].type);
auto return_type_with_nulls = getMostSubtype(data_types, true, true);
@ -446,7 +446,7 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
}
}
block.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
template <typename T, size_t>

View File

@ -40,15 +40,15 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[result].type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
const auto & array_column = block.getByPosition(arguments[0]).column;
const auto & array_column = block[arguments[0]].column;
std::unique_ptr<GatherUtils::IArraySource> source;
@ -66,7 +66,7 @@ public:
else
sink = GatherUtils::sliceFromLeftConstantOffsetBounded(*source, 0, -1);
block.getByPosition(result).column = std::move(sink);
block[result].column = std::move(sink);
}
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -48,25 +48,25 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[result].type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
auto result_column = return_type->createColumn();
auto array_column = block.getByPosition(arguments[0]).column;
auto appended_column = block.getByPosition(arguments[1]).column;
auto array_column = block[arguments[0]].column;
auto appended_column = block[arguments[1]].column;
if (!block.getByPosition(arguments[0]).type->equals(*return_type))
array_column = castColumn(block.getByPosition(arguments[0]), return_type);
if (!block[arguments[0]].type->equals(*return_type))
array_column = castColumn(block[arguments[0]], return_type);
const DataTypePtr & return_nested_type = typeid_cast<const DataTypeArray &>(*return_type).getNestedType();
if (!block.getByPosition(arguments[1]).type->equals(*return_nested_type))
appended_column = castColumn(block.getByPosition(arguments[1]), return_nested_type);
if (!block[arguments[1]].type->equals(*return_nested_type))
appended_column = castColumn(block[arguments[1]], return_nested_type);
std::unique_ptr<GatherUtils::IArraySource> array_source;
std::unique_ptr<GatherUtils::IValueSource> value_source;
@ -99,7 +99,7 @@ public:
GatherUtils::push(*array_source, *value_source, *sink, push_front);
block.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -120,7 +120,7 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
for (size_t i = 0; i < num_arguments_columns; ++i)
{
const IColumn * col = block.getByPosition(arguments[i + 1]).column.get();
const IColumn * col = block[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(Block & block, const ColumnNumbers & argum
}
const IColumn ** aggregate_arguments = aggregate_arguments_vec.data();
MutableColumnPtr result_holder = block.getByPosition(result).type->createColumn();
MutableColumnPtr result_holder = block[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(Block & block, const ColumnNumbers & argum
if (!res_col_aggregate_function && agg_func.isState())
throw Exception("State function " + agg_func.getName() + " inserts results into non-state column "
+ block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN);
+ block[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(Block & block, const ColumnNumbers & argum
agg_func.insertResultInto(places[i], res_col, arena.get());
else
res_col_aggregate_function->insertFrom(places[i]);
block.getByPosition(result).column = std::move(result_holder);
block[result].column = std::move(result_holder);
}

View File

@ -133,7 +133,7 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers
/// Handling ranges
const IColumn * ranges_col_array = block.getByPosition(arguments[1]).column.get();
const IColumn * ranges_col_array = block[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(Block & block, const ColumnNumbers
for (size_t i = 0; i < num_arguments_columns; ++i)
{
const IColumn * col = block.getByPosition(arguments[i + 2]).column.get();
const IColumn * col = block[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(Block & block, const ColumnNumbers
/// Handling results
MutableColumnPtr result_holder = block.getByPosition(result).type->createColumn();
MutableColumnPtr result_holder = block[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(Block & block, const ColumnNumbers
if (!res_col_aggregate_function && agg_func.isState())
throw Exception("State function " + agg_func.getName() + " inserts results into non-state column "
+ block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN);
+ block[result].type->getName(), ErrorCodes::ILLEGAL_COLUMN);
/// Perform the aggregation
@ -383,7 +383,7 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers
}
}
block.getByPosition(result).column = std::move(result_holder);
block[result].column = std::move(result_holder);
}

View File

@ -65,21 +65,21 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[result].type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
auto result_column = return_type->createColumn();
auto array_column = block.getByPosition(arguments[0]).column;
auto size_column = block.getByPosition(arguments[1]).column;
auto array_column = block[arguments[0]].column;
auto size_column = block[arguments[1]].column;
if (!block.getByPosition(arguments[0]).type->equals(*return_type))
array_column = castColumn(block.getByPosition(arguments[0]), return_type);
if (!block[arguments[0]].type->equals(*return_type))
array_column = castColumn(block[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.getByPosition(arguments[2]).column;
if (!block.getByPosition(arguments[2]).type->equals(*return_nested_type))
appended_column = castColumn(block.getByPosition(arguments[2]), return_nested_type);
appended_column = block[arguments[2]].column;
if (!block[arguments[2]].type->equals(*return_nested_type))
appended_column = castColumn(block[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.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -55,9 +55,9 @@ private:
void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) const
{
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!array)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
throw Exception("Illegal column " + block[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(Block & block, const ColumnNumbers & argu
+ " of null map of the first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(res_ptr);
block[result].column = std::move(res_ptr);
}

View File

@ -49,8 +49,8 @@ private:
template <typename T, typename U>
bool executeNumberNumber(Block & block, const ColumnNumbers & arguments, size_t result) const
{
ColumnPtr col1 = block.getByPosition(arguments[0]).column->convertToFullColumnIfConst();
ColumnPtr col2 = block.getByPosition(arguments[1]).column->convertToFullColumnIfConst();
ColumnPtr col1 = block[arguments[0]].column->convertToFullColumnIfConst();
ColumnPtr col2 = block[arguments[1]].column->convertToFullColumnIfConst();
if (!col1 || !col2)
return false;
@ -75,7 +75,7 @@ private:
col_array1->getOffsets(),
col_res->getData());
block.getByPosition(result).column = std::move(col_res);
block[result].column = std::move(col_res);
return true;
}
@ -135,7 +135,7 @@ public:
|| executeNumber<Int64>(block, arguments, result)
|| executeNumber<Float32>(block, arguments, result)
|| executeNumber<Float64>(block, arguments, result)))
throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function "
throw Exception{"Illegal column " + block[arguments[0]].column->getName() + " of first argument of function "
+ getName(),
ErrorCodes::ILLEGAL_COLUMN};
}

View File

@ -71,17 +71,17 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & return_type = block.getByPosition(result).type;
const auto & return_type = block[result].type;
if (return_type->onlyNull())
{
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
block[result].column = return_type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
auto & array_column = block.getByPosition(arguments[0]).column;
const auto & offset_column = block.getByPosition(arguments[1]).column;
const auto & length_column = arguments.size() > 2 ? block.getByPosition(arguments[2]).column : nullptr;
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;
std::unique_ptr<GatherUtils::IArraySource> source;
@ -105,7 +105,7 @@ public:
{
if (!length_column || length_column->onlyNull())
{
block.getByPosition(result).column = array_column;
block[result].column = array_column;
return;
}
else if (isColumnConst(*length_column))
@ -146,7 +146,7 @@ public:
sink = GatherUtils::sliceDynamicOffsetBounded(*source, *offset_column, *length_column);
}
block.getByPosition(result).column = std::move(sink);
block[result].column = std::move(sink);
}
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -131,14 +131,14 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
Columns array_holders;
for (size_t i = 0; i < num_arguments; ++i)
{
const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column;
const ColumnPtr & array_ptr = block[arguments[i]].column;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
if (!array)
{
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(
block.getByPosition(arguments[i]).column.get());
block[arguments[i]].column.get());
if (!const_array)
throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName()
throw Exception("Illegal column " + block[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(Block & block, const ColumnNumbers & argumen
executeHashed(*offsets, data_columns, res_values);
}
block.getByPosition(result).column = std::move(res);
block[result].column = std::move(res);
}
template <typename Method, bool has_null_map>

View File

@ -49,8 +49,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t num_rows) const override
{
const auto * col_num = block.getByPosition(arguments[0]).column.get();
const auto * col_value = block.getByPosition(arguments[1]).column.get();
const auto * col_num = block[arguments[0]].column.get();
const auto * col_value = block[arguments[1]].column.get();
auto offsets_col = ColumnArray::ColumnOffsets::create();
ColumnArray::Offsets & offsets = offsets_col->getData();
@ -72,7 +72,7 @@ public:
offsets.push_back(offset);
}
block.getByPosition(result).column = ColumnArray::create(col_value->replicate(offsets)->convertToFullColumnIfConst(), std::move(offsets_col));
block[result].column = ColumnArray::create(col_value->replicate(offsets)->convertToFullColumnIfConst(), std::move(offsets_col));
}
};

View File

@ -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.getByPosition(arguments[i]).column->convertToFullColumnIfConst();
ColumnPtr holder = block[arguments[i]].column->convertToFullColumnIfConst();
const ColumnArray * column_array = checkAndGetColumn<ColumnArray>(holder.get());
@ -86,7 +86,7 @@ public:
tuple_columns[i] = column_array->getDataPtr();
}
block.getByPosition(result).column = ColumnArray::create(
block[result].column = ColumnArray::create(
ColumnTuple::create(tuple_columns), static_cast<const ColumnArray &>(*first_array_column).getOffsetsPtr());
}
};

View File

@ -48,7 +48,7 @@ private:
{
using UnderlyingColumnType = typename TypeToColumnType<typename DataType::FieldType>::ColumnType;
block.getByPosition(result).column = ColumnArray::create(
block[result].column = ColumnArray::create(
UnderlyingColumnType::create(),
ColumnArray::ColumnOffsets::create(input_rows_count, 0));
}

View File

@ -55,18 +55,18 @@ namespace
{
bool executeConst(FunctionArguments & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(block[arguments[0]].column.get()))
{
if (const_array->getValue<Array>().empty())
{
auto nested_type = typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType();
auto nested_type = typeid_cast<const DataTypeArray &>(*block[arguments[0]].type).getNestedType();
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
block[result].column = block[result].type->createColumnConst(
input_rows_count,
Array{nested_type->getDefault()});
}
else
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
block[result].column = block[arguments[0]].column;
return true;
}
@ -374,9 +374,9 @@ void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers
if (FunctionEmptyArrayToSingleImpl::executeConst(block, arguments, result, input_rows_count))
return;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block[arguments[0]].column.get());
if (!array)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
throw Exception("Illegal column " + block[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(Block & block, const ColumnNumbers
else
FunctionEmptyArrayToSingleImpl::executeDispatch<false>(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map);
block.getByPosition(result).column = std::move(res_ptr);
block[result].column = std::move(res_ptr);
}

View File

@ -61,7 +61,7 @@ public:
DataTypes data_types;
data_types.reserve(arguments.size());
for (const auto & argument : arguments)
data_types.push_back(block.getByPosition(argument).type);
data_types.push_back(block[argument].type);
common_type = getLeastSupertype(data_types);
}
@ -73,7 +73,7 @@ public:
for (size_t i = 0; i < num_args; ++i)
{
const auto & argument = block.getByPosition(arguments[i]);
const auto & argument = block[arguments[i]];
ColumnPtr preprocessed_column = argument.column;
const auto argument_type = typeid_cast<const DataTypeArray *>(argument.type.get());
@ -108,7 +108,7 @@ public:
auto result_column_ptr = typeid_cast<ColumnUInt8 *>(result_column.get());
GatherUtils::sliceHas(*sources[0], *sources[1], search_type, *result_column_ptr);
block.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -199,7 +199,7 @@ private:
// same offsets as in keys
to_vals_arr.getOffsets().insert(to_keys_offset.begin(), to_keys_offset.end());
block.getByPosition(result).column = std::move(res_tuple);
block[result].column = std::move(res_tuple);
}
template <typename KeyType, bool is_str_key>
@ -228,7 +228,7 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) const override
{
const DataTypeTuple * tup_type = checkAndGetDataType<DataTypeTuple>((block.safeGetByPosition(arguments[0])).type.get());
const DataTypeTuple * tup_type = checkAndGetDataType<DataTypeTuple>((block[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.getByPosition(arg);
auto & col = block[arg];
const ColumnTuple * tup;
bool is_const = isColumnConst(*col.column);
if (is_const)
@ -274,7 +274,7 @@ private:
args.push_back({key_column, val_column, key_offsets, val_offsets, is_const});
}
size_t row_count = block.getByPosition(arguments[0]).column->size();
size_t row_count = block[arguments[0]].column->size();
auto key_type_id = key_array_type->getNestedType()->getTypeId();
switch (key_type_id)

View File

@ -213,7 +213,7 @@ private:
}
to_vals_arr.getOffsets().insert(to_keys_offsets.begin(), to_keys_offsets.end());
block.getByPosition(result).column = std::move(res_tuple);
block[result].column = std::move(res_tuple);
}
template <typename KeyType>
@ -255,7 +255,8 @@ private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) const override
{
auto col1 = block.safeGetByPosition(arguments[0]), col2 = block.safeGetByPosition(arguments[1]);
auto col1 = block[arguments[0]];
auto col2 = block[arguments[1]];
const auto * k = assert_cast<const DataTypeArray *>(col1.type.get());
const auto * v = assert_cast<const DataTypeArray *>(col2.type.get());
@ -269,7 +270,7 @@ private:
if (arguments.size() == 3)
{
/* max key provided */
max_key_column = block.safeGetByPosition(arguments[2]).column;
max_key_column = block[arguments[2]].column;
}
switch (k->getNestedType()->getTypeId())

View File

@ -94,7 +94,7 @@ private:
out_offsets[row_idx] = offset;
}
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
return true;
}
else
@ -157,7 +157,7 @@ private:
out_offsets[row_idx] = offset;
}
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
return true;
}
@ -219,7 +219,7 @@ private:
out_offsets[row_idx] = offset;
}
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
return true;
}
@ -281,7 +281,7 @@ private:
out_offsets[row_idx] = offset;
}
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
return true;
}
@ -347,7 +347,7 @@ private:
out_offsets[row_idx] = offset;
}
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
return true;
}
@ -355,7 +355,7 @@ private:
{
if (arguments.size() == 1)
{
const auto * col = block.getByPosition(arguments[0]).column.get();
const auto * col = block[arguments[0]].column.get();
if (!executeInternal<UInt8>(block, col, result) &&
!executeInternal<UInt16>(block, col, result) &&
!executeInternal<UInt32>(block, col, result) &&
@ -369,14 +369,14 @@ private:
Columns columns_holder(3);
ColumnRawPtrs columns(3);
const auto return_type = checkAndGetDataType<DataTypeArray>(block.getByPosition(result).type.get())->getNestedType();
const auto return_type = checkAndGetDataType<DataTypeArray>(block[result].type.get())->getNestedType();
for (size_t i = 0; i < arguments.size(); ++i)
{
if (i == 1)
columns_holder[i] = castColumn(block.getByPosition(arguments[i]), return_type)->convertToFullColumnIfConst();
columns_holder[i] = castColumn(block[arguments[i]], return_type)->convertToFullColumnIfConst();
else
columns_holder[i] = castColumn(block.getByPosition(arguments[i]), return_type);
columns_holder[i] = castColumn(block[arguments[i]], return_type);
columns[i] = columns_holder[i].get();
}

View File

@ -41,8 +41,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) const override
{
const ColumnPtr & col = block.getByPosition(arguments[0]).column;
ColumnPtr & res_col = block.getByPosition(result).column;
const ColumnPtr & col = block[arguments[0]].column;
ColumnPtr & res_col = block[result].column;
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(*col))
res_col = nullable_col->getNestedColumnPtr();

View File

@ -81,7 +81,7 @@ public:
if (max_width > 1000)
throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
const auto & src = *block.getByPosition(arguments[0]).column;
const auto & src = *block[arguments[0]].column;
auto res_column = ColumnString::create();
@ -96,11 +96,11 @@ public:
|| executeNumber<Float32>(src, *res_column, min, max, max_width)
|| executeNumber<Float64>(src, *res_column, min, max, max_width))
{
block.getByPosition(result).column = std::move(res_column);
block[result].column = std::move(res_column);
}
else
throw Exception(
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
"Illegal column " + block[arguments[0]].column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
@ -108,7 +108,7 @@ private:
template <typename T>
T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const
{
const auto & column = *block.getByPosition(arguments[argument_pos]).column;
const auto & column = *block[arguments[argument_pos]].column;
if (!isColumnConst(column))
throw Exception(

View File

@ -54,7 +54,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
size_t current_block_number = block_number++;
block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, current_block_number);
block[result].column = ColumnUInt64::create(input_rows_count, current_block_number);
}
};

View File

@ -35,9 +35,9 @@ public:
UInt64 size = 0;
for (auto arg_pos : arguments)
size += blockSerializedSizeOne(block.getByPosition(arg_pos));
size += blockSerializedSizeOne(block[arg_pos]);
block.getByPosition(result).column = DataTypeUInt64().createColumnConst(
block[result].column = DataTypeUInt64().createColumnConst(
input_rows_count, size)->convertToFullColumnIfConst();
}

View File

@ -45,7 +45,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
block.getByPosition(result).column = ColumnUInt64::create(input_rows_count, input_rows_count);
block[result].column = ColumnUInt64::create(input_rows_count, input_rows_count);
}
};

View File

@ -40,7 +40,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, SymbolIndex::instance().getBuildIDHex());
block[result].column = DataTypeString().createColumnConst(input_rows_count, SymbolIndex::instance().getBuildIDHex());
}
};

View File

@ -74,14 +74,14 @@ public:
if (i % 2)
{
src_array_args.push_back(args[i]);
src_array_elems.push_back(block.getByPosition(args[i]));
src_array_types.push_back(block.getByPosition(args[i]).type);
src_array_elems.push_back(block[args[i]]);
src_array_types.push_back(block[args[i]].type);
}
else
{
dst_array_args.push_back(args[i]);
dst_array_elems.push_back(block.getByPosition(args[i]));
dst_array_types.push_back(block.getByPosition(args[i]).type);
dst_array_elems.push_back(block[args[i]]);
dst_array_types.push_back(block[args[i]].type);
}
}
@ -108,7 +108,7 @@ public:
->execute(temp_block_columns, transform_args, result, input_rows_count);
/// Put the result into the original block.
block.getByPosition(result).column = std::move(temp_block_columns[result].column);
block[result].column = std::move(temp_block_columns[result].column);
}
private:

View File

@ -97,7 +97,7 @@ public:
filtered_args.reserve(arguments.size());
for (const auto & arg : arguments)
{
const auto & type = block.getByPosition(arg).type;
const auto & type = block[arg].type;
if (type->onlyNull())
continue;
@ -129,7 +129,7 @@ public:
{
temp_block_columns.emplace_back(ColumnWithTypeAndName{nullptr, std::make_shared<DataTypeUInt8>(), ""});
is_not_null->build({temp_block_columns[filtered_args[i]]})->execute(temp_block_columns, {filtered_args[i]}, res_pos, input_rows_count);
temp_block_columns.emplace_back(ColumnWithTypeAndName{nullptr, removeNullable(block.getByPosition(filtered_args[i]).type), ""});
temp_block_columns.emplace_back(ColumnWithTypeAndName{nullptr, removeNullable(block[filtered_args[i]].type), ""});
assume_not_null->build({temp_block_columns[filtered_args[i]]})->execute(temp_block_columns, {filtered_args[i]}, res_pos + 1, input_rows_count);
multi_if_args.push_back(res_pos);
@ -140,13 +140,13 @@ public:
/// If all arguments appeared to be NULL.
if (multi_if_args.empty())
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
block[result].column = block[result].type->createColumnConstWithDefaultValue(input_rows_count);
return;
}
if (multi_if_args.size() == 1)
{
block.getByPosition(result).column = block.getByPosition(multi_if_args.front()).column;
block[result].column = block[multi_if_args.front()].column;
return;
}
@ -160,7 +160,7 @@ public:
ColumnPtr res = std::move(temp_block_columns[result].column);
/// if last argument is not nullable, result should be also not nullable
if (!block.getByPosition(multi_if_args.back()).column->isNullable() && res->isNullable())
if (!block[multi_if_args.back()].column->isNullable() && res->isNullable())
{
if (const auto * column_lc = checkAndGetColumn<ColumnLowCardinality>(*res))
res = checkAndGetColumn<ColumnNullable>(*column_lc->convertToFullColumn())->getNestedColumnPtr();
@ -170,7 +170,7 @@ public:
res = checkAndGetColumn<ColumnNullable>(*res)->getNestedColumnPtr();
}
block.getByPosition(result).column = std::move(res);
block[result].column = std::move(res);
}
private:

View File

@ -89,8 +89,8 @@ private:
void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) const
{
const IColumn * c0 = block.getByPosition(arguments[0]).column.get();
const IColumn * c1 = block.getByPosition(arguments[1]).column.get();
const IColumn * c0 = block[arguments[0]].column.get();
const IColumn * c1 = block[arguments[1]].column.get();
const ColumnString * c0_string = checkAndGetColumn<ColumnString>(c0);
const ColumnString * c1_string = checkAndGetColumn<ColumnString>(c1);
@ -112,7 +112,7 @@ private:
return;
}
block.getByPosition(result).column = std::move(c_res);
block[result].column = std::move(c_res);
}
void executeFormatImpl(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) const
@ -129,7 +129,7 @@ private:
bool has_column_fixed_string = false;
for (size_t i = 0; i < num_arguments; ++i)
{
const ColumnPtr & column = block.getByPosition(arguments[i]).column;
const ColumnPtr & column = block[arguments[i]].column;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
{
has_column_string = true;
@ -169,7 +169,7 @@ private:
c_res->getOffsets(),
input_rows_count);
block.getByPosition(result).column = std::move(c_res);
block[result].column = std::move(c_res);
}
};

View File

@ -186,9 +186,9 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
{
const ColumnWithTypeAndName & arg_from = block.getByPosition(arguments[0]);
const ColumnWithTypeAndName & arg_charset_from = block.getByPosition(arguments[1]);
const ColumnWithTypeAndName & arg_charset_to = block.getByPosition(arguments[2]);
const ColumnWithTypeAndName & arg_from = block[arguments[0]];
const ColumnWithTypeAndName & arg_charset_from = block[arguments[1]];
const ColumnWithTypeAndName & arg_charset_to = block[arguments[2]];
const ColumnConst * col_charset_from = checkAndGetColumnConstStringOrFixedString(arg_charset_from.column.get());
const ColumnConst * col_charset_to = checkAndGetColumnConstStringOrFixedString(arg_charset_to.column.get());
@ -204,7 +204,7 @@ public:
{
auto col_to = ColumnString::create();
convert(charset_from, charset_to, col_from->getChars(), col_from->getOffsets(), col_to->getChars(), col_to->getOffsets());
block.getByPosition(result).column = std::move(col_to);
block[result].column = std::move(col_to);
}
else
throw Exception("Illegal column passed as first argument of function " + getName() + " (must be ColumnString).",

View File

@ -50,7 +50,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) const override
{
const auto & src_column = block.getByPosition(arguments[0]);
const auto & src_column = block[arguments[0]];
if (!src_column.column)
throw Exception("Illegal column while execute function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -76,7 +76,7 @@ public:
throw Exception("Wrong call for " + getName() + " with " + src_column.type->getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result_pos).column = std::move(result_column);
block[result_pos].column = std::move(result_column);
}
private:

View File

@ -43,7 +43,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, db_name);
block[result].column = DataTypeString().createColumnConst(input_rows_count, db_name);
}
};

View File

@ -43,7 +43,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, user_name);
block[result].column = DataTypeString().createColumnConst(input_rows_count, user_name);
}
};

View File

@ -85,14 +85,14 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto * unit_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
const auto * unit_column = checkAndGetColumnConst<ColumnString>(block[arguments[0]].column.get());
if (!unit_column)
throw Exception("First argument for function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN);
String unit = Poco::toLower(unit_column->getValue<String>());
const IColumn & x = *block.getByPosition(arguments[1]).column;
const IColumn & y = *block.getByPosition(arguments[2]).column;
const IColumn & x = *block[arguments[1]].column;
const IColumn & y = *block[arguments[2]].column;
size_t rows = input_rows_count;
auto res = ColumnInt64::create(rows);
@ -119,7 +119,7 @@ public:
else
throw Exception("Function " + getName() + " does not support '" + unit + "' unit", ErrorCodes::BAD_ARGUMENTS);
block.getByPosition(result).column = std::move(res);
block[result].column = std::move(res);
}
private:

View File

@ -143,7 +143,7 @@ public:
to_start_of_interval->execute(temp_block, {arguments[1], interval_pos, arguments[2]}, result, input_rows_count);
}
block.getByPosition(result).column = std::move(temp_block[result].column);
block[result].column = std::move(temp_block[result].column);
}
bool hasInformationAboutMonotonicity() const override

View File

@ -38,8 +38,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const IDataType & type = *block.getByPosition(arguments[0]).type;
block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault());
const IDataType & type = *block[arguments[0]].type;
block[result].column = type.createColumnConst(input_rows_count, type.getDefault());
}
ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments) const override

View File

@ -51,8 +51,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) const override
{
const IDataType & type = *block.getByPosition(result).type;
block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault());
const IDataType & type = *block[result].type;
block[result].column = type.createColumnConst(input_rows_count, type.getDefault());
}
};

View File

@ -63,7 +63,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnPtr & column = block[arguments[0]].column;
const ColumnString * column_concrete = checkAndGetColumn<ColumnString>(column.get());
if (!column_concrete)
@ -85,7 +85,7 @@ public:
}
}
block.getByPosition(result).column = std::move(result_column);
block[result].column = std::move(result_column);
}
};

View File

@ -38,11 +38,11 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const auto & elem = block.getByPosition(arguments[0]);
const auto & elem = block[arguments[0]];
/// Note that the result is not a constant, because it contains block size.
block.getByPosition(result).column
block[result].column
= DataTypeString().createColumnConst(input_rows_count,
elem.type->getName() + ", " + elem.column->dumpStructure())->convertToFullColumnIfConst();
}

View File

@ -67,7 +67,7 @@ public:
if (arguments.empty())
throw Exception("Function " + getName() + " requires at least one argument", ErrorCodes::BAD_ARGUMENTS);
const auto * model = block.getByPosition(arguments[0]).column.get();
const auto * model = block[arguments[0]].column.get();
if (const auto * column_with_states = typeid_cast<const ColumnConst *>(model))
model = column_with_states->getDataColumnPtr().get();
@ -75,10 +75,10 @@ public:
const auto * agg_function = typeid_cast<const ColumnAggregateFunction *>(model);
if (!agg_function)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
throw Exception("Illegal column " + block[arguments[0]].column->getName()
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = agg_function->predictValues(block.data, arguments, context);
block[result].column = agg_function->predictValues(block.data, arguments, context);
}
const Context & context;

View File

@ -73,8 +73,8 @@ public:
{
static const auto MAX_GROUPS_COUNT = 128;
const ColumnPtr column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const ColumnPtr column_haystack = block[arguments[0]].column;
const ColumnPtr column_needle = block[arguments[1]].column;
const auto needle = typeid_cast<const ColumnConst &>(*column_needle).getValue<String>();
@ -233,7 +233,7 @@ public:
ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(data_col), std::move(nested_offsets_col));
ColumnArray::MutablePtr root_array_col = ColumnArray::create(std::move(nested_array_col), std::move(root_offsets_col));
block.getByPosition(result).column = std::move(root_array_col);
block[result].column = std::move(root_array_col);
}
};

View File

@ -53,8 +53,8 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override
{
const ColumnPtr column_haystack = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
const ColumnPtr column_haystack = block[arguments[0]].column;
const ColumnPtr column_needle = block[arguments[1]].column;
const auto needle = typeid_cast<const ColumnConst &>(*column_needle).getValue<String>();
@ -100,7 +100,7 @@ public:
offsets_data[i] = current_offset;
}
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
block[result].column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
}
};

Some files were not shown because too many files have changed in this diff Show More