mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Replace getByPosition to []
This commit is contained in:
parent
1e2b8a748b
commit
d28325a353
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
};
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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)});
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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};
|
||||
}
|
||||
};
|
||||
|
@ -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(),
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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:
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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};
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -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:
|
||||
|
@ -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; }
|
||||
|
@ -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>
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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>
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -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());
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
};
|
||||
|
@ -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>
|
||||
|
@ -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; }
|
||||
|
@ -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; }
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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; }
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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};
|
||||
}
|
||||
|
@ -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; }
|
||||
|
@ -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>
|
||||
|
@ -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));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
};
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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; }
|
||||
|
@ -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)
|
||||
|
@ -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())
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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(
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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:
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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).",
|
||||
|
@ -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:
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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());
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user