mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Better semantic of sharing columns: development [#CLICKHOUSE-2].
This commit is contained in:
parent
45805a367f
commit
14b0397500
@ -648,23 +648,23 @@ private:
|
||||
if (auto col_right = checkAndGetColumn<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::vector_vector(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
else if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::vector_constant(col_left->getData(), col_right->template getValue<T1>(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -678,12 +678,12 @@ private:
|
||||
if (auto col_right = checkAndGetColumn<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
auto & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->size());
|
||||
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::constant_vector(col_left->template getValue<T0>(), col_right->getData(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
else if (auto col_right = checkAndGetColumnConst<ColumnVector<T1>>(block.getByPosition(arguments[1]).column.get()))
|
||||
@ -1306,7 +1306,7 @@ private:
|
||||
out[i] = Impl::apply(val[i], mask[i]);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = out_col;
|
||||
block.getByPosition(result).column = std::move(out_col);
|
||||
return true;
|
||||
}
|
||||
else if (const auto value_col = checkAndGetColumnConst<ColumnVector<T>>(value_col_untyped))
|
||||
@ -1330,7 +1330,7 @@ private:
|
||||
for (const auto i : ext::range(0, size))
|
||||
out[i] = Impl::apply(val, mask[i]);
|
||||
|
||||
block.getByPosition(result).column = out_col;
|
||||
block.getByPosition(result).column = std::move(out_col);
|
||||
}
|
||||
|
||||
return true;
|
||||
|
@ -789,8 +789,7 @@ private:
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
const auto col_res = std::make_shared<ResultColumnType>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
auto col_res = std::make_shared<ResultColumnType>();
|
||||
|
||||
/// Null maps of the 1st and second function arguments,
|
||||
/// if it applies.
|
||||
@ -822,6 +821,7 @@ private:
|
||||
else
|
||||
return false;
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -837,8 +837,7 @@ private:
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
const auto col_res = std::make_shared<ResultColumnType>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
auto col_res = std::make_shared<ResultColumnType>();
|
||||
|
||||
/// Null maps of the 1st and second function arguments,
|
||||
/// if it applies.
|
||||
@ -872,6 +871,7 @@ private:
|
||||
else
|
||||
return false;
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -916,8 +916,7 @@ private:
|
||||
}
|
||||
|
||||
const auto size = item_arg->size();
|
||||
const auto col_res = std::make_shared<ResultColumnType>(size);
|
||||
block.getByPosition(result).column = col_res;
|
||||
auto col_res = ResultColumnType::create(size);
|
||||
|
||||
auto & data = col_res->getData();
|
||||
|
||||
@ -945,6 +944,8 @@ private:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
|
||||
return true;
|
||||
@ -960,8 +961,7 @@ private:
|
||||
const IColumn & col_nested = col_array->getData();
|
||||
const IColumn & item_arg = *block.getByPosition(arguments[1]).column;
|
||||
|
||||
const auto col_res = std::make_shared<ResultColumnType>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
auto col_res = ResultColumnType::create();
|
||||
|
||||
/// Null maps of the 1st and second function arguments,
|
||||
/// if it applies.
|
||||
@ -999,6 +999,7 @@ private:
|
||||
null_map_data, null_map_item);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -851,7 +851,6 @@ struct DateTimeAddIntervalImpl
|
||||
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const IColumn & delta_column = *block.getByPosition(arguments[1]).column;
|
||||
|
||||
@ -859,12 +858,14 @@ struct DateTimeAddIntervalImpl
|
||||
Op::vector_constant(sources->getData(), col_to->getData(), delta_const_column->getField().get<Int64>(), time_zone);
|
||||
else
|
||||
Op::vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else if (const auto * sources = checkAndGetColumnConst<ColumnVector<FromType>>(source_col.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
Op::constant_vector(sources->template getValue<FromType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -241,7 +241,6 @@ public:
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<T>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
|
||||
typename ColumnVector<T>::Container & vec_to = col_to->getData();
|
||||
@ -250,6 +249,8 @@ 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);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -339,7 +340,6 @@ public:
|
||||
if (col_vec1 && col_vec2)
|
||||
{
|
||||
auto col_to = ColumnUInt8::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from1 = col_vec1->getData();
|
||||
const typename ColumnVector<T>::Container & vec_from2 = col_vec2->getData();
|
||||
@ -349,11 +349,12 @@ 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);
|
||||
}
|
||||
else if (col_vec1 && col_const2)
|
||||
{
|
||||
auto col_to = ColumnUInt8::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from1 = col_vec1->getData();
|
||||
const T const_from2 = col_const2->template getValue<T>();
|
||||
@ -363,11 +364,12 @@ 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);
|
||||
}
|
||||
else if (col_const1 && col_vec2)
|
||||
{
|
||||
auto col_to = ColumnUInt8::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const T const_from1 = col_const1->template getValue<T>();
|
||||
const typename ColumnVector<T>::Container & vec_from2 = col_vec2->getData();
|
||||
@ -377,6 +379,8 @@ 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);
|
||||
}
|
||||
else if (col_const1 && col_const2)
|
||||
{
|
||||
@ -735,7 +739,6 @@ public:
|
||||
if (const ColumnUInt32 * col_from = typeid_cast<const ColumnUInt32 *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const ColumnUInt32::Container & region_ids = col_from->getData();
|
||||
|
||||
@ -744,6 +747,8 @@ public:
|
||||
const StringRef & name_ref = dict.getRegionName(region_ids[i], language);
|
||||
col_to->insertDataWithTerminatingZero(name_ref.data, name_ref.size + 1);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
|
@ -129,10 +129,9 @@ private:
|
||||
{
|
||||
const auto & ids = id_col->getData();
|
||||
|
||||
const auto out = ColumnUInt8::create(ext::size(ids));
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnUInt8::create(ext::size(ids));
|
||||
dict->has(ids, out->getData());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
|
||||
{
|
||||
@ -171,10 +170,9 @@ private:
|
||||
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col).getColumns();
|
||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||
|
||||
const auto out = ColumnUInt8::create(key_col_with_type.column->size());
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
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);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
@ -324,9 +322,9 @@ private:
|
||||
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
|
||||
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
|
||||
{
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnString::create();
|
||||
dict->getString(attr_name, id_col->getData(), out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
|
||||
{
|
||||
@ -380,10 +378,9 @@ private:
|
||||
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col).getColumns();
|
||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnString::create();
|
||||
dict->getString(attr_name, key_columns, key_types, out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
@ -438,17 +435,16 @@ private:
|
||||
{
|
||||
if (const auto date_col = checkAndGetColumn<ColumnUInt16>(date_col_untyped))
|
||||
{
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnString::create();
|
||||
dictionary->getString(attr_name, id_col->getData(), date_col->getData(), out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto date_col = checkAndGetColumnConst<ColumnVector<UInt16>>(date_col_untyped))
|
||||
{
|
||||
auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
const PaddedPODArray<UInt16> dates(id_col->size(), date_col->getValue<UInt64>());
|
||||
dictionary->getString(attr_name, id_col->getData(), dates, out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -465,11 +461,10 @@ private:
|
||||
{
|
||||
if (const auto date_col = checkAndGetColumn<ColumnUInt16>(date_col_untyped))
|
||||
{
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnString::create();
|
||||
const PaddedPODArray<UInt64> ids(date_col->size(), id_col->getValue<UInt64>());
|
||||
dictionary->getString(attr_name, ids, date_col->getData(), out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto date_col = checkAndGetColumnConst<ColumnVector<UInt16>>(date_col_untyped))
|
||||
{
|
||||
@ -478,7 +473,6 @@ private:
|
||||
|
||||
auto out = std::make_unique<ColumnString>();
|
||||
dictionary->getString(attr_name, ids, dates, out.get());
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
|
||||
}
|
||||
else
|
||||
@ -603,23 +597,19 @@ private:
|
||||
if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped))
|
||||
{
|
||||
/// vector ids, vector defaults
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnString::create();
|
||||
const auto & ids = id_col->getData();
|
||||
|
||||
dictionary->getString(attr_name, ids, default_col, out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto default_col = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
|
||||
{
|
||||
/// vector ids, const defaults
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnString::create();
|
||||
const auto & ids = id_col->getData();
|
||||
String def = default_col->getValue<String>();
|
||||
|
||||
dictionary->getString(attr_name, ids, def, out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
@ -639,21 +629,17 @@ private:
|
||||
/// const ids, vector defaults
|
||||
/// @todo avoid materialization
|
||||
const PaddedPODArray<UInt64> ids(id_col->size(), id_col->getValue<UInt64>());
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnString::create();
|
||||
dictionary->getString(attr_name, ids, default_col, out.get());
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto default_col = checkAndGetColumnConstStringOrFixedString(default_col_untyped))
|
||||
{
|
||||
/// const ids, const defaults
|
||||
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
|
||||
auto out = std::make_unique<ColumnString>();
|
||||
|
||||
String def = default_col->getValue<String>();
|
||||
|
||||
dictionary->getString(attr_name, ids, def, out.get());
|
||||
|
||||
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
|
||||
}
|
||||
else
|
||||
@ -688,8 +674,7 @@ private:
|
||||
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_col).getColumns();
|
||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||
|
||||
const auto out = ColumnString::create();
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnString::create();
|
||||
|
||||
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
||||
if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped))
|
||||
@ -706,6 +691,7 @@ private:
|
||||
"Fourth argument of function " + getName() + " must be String",
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -884,20 +870,17 @@ private:
|
||||
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
|
||||
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
|
||||
{
|
||||
const auto out = ColumnVector<Type>::create(id_col->size());
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::create(id_col->size());
|
||||
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);
|
||||
}
|
||||
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
|
||||
{
|
||||
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
|
||||
PaddedPODArray<Type> data(1);
|
||||
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
|
||||
|
||||
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
|
||||
}
|
||||
else
|
||||
@ -944,12 +927,10 @@ private:
|
||||
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col).getColumns();
|
||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||
|
||||
const auto out = ColumnVector<Type>::create(key_columns.front()->size());
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::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);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
@ -1008,11 +989,10 @@ private:
|
||||
const auto & ids = id_col->getData();
|
||||
const auto & dates = date_col->getData();
|
||||
|
||||
const auto out = ColumnVector<Type>::create(size);
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::create(size);
|
||||
auto & data = out->getData();
|
||||
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto date_col = checkAndGetColumnConst<ColumnVector<UInt16>>(date_col_untyped))
|
||||
{
|
||||
@ -1020,11 +1000,10 @@ private:
|
||||
const auto & ids = id_col->getData();
|
||||
const PaddedPODArray<UInt16> dates(size, date_col->getValue<UInt16>());
|
||||
|
||||
const auto out = ColumnVector<Type>::create(size);
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::create(size);
|
||||
auto & data = out->getData();
|
||||
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1045,11 +1024,10 @@ private:
|
||||
const PaddedPODArray<UInt64> ids(size, id_col->getValue<UInt64>());
|
||||
const auto & dates = date_col->getData();
|
||||
|
||||
const auto out = ColumnVector<Type>::create(size);
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::create(size);
|
||||
auto & data = out->getData();
|
||||
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto date_col = checkAndGetColumnConst<ColumnVector<UInt16>>(date_col_untyped))
|
||||
{
|
||||
@ -1057,7 +1035,6 @@ private:
|
||||
const PaddedPODArray<UInt16> dates(1, date_col->getValue<UInt16>());
|
||||
PaddedPODArray<Type> data(1);
|
||||
DictGetTraits<DataType>::get(dictionary, attr_name, ids, dates, data);
|
||||
|
||||
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
|
||||
}
|
||||
else
|
||||
@ -1219,26 +1196,22 @@ private:
|
||||
if (const auto default_col = checkAndGetColumn<ColumnVector<Type>>(default_col_untyped))
|
||||
{
|
||||
/// vector ids, vector defaults
|
||||
const auto out = ColumnVector<Type>::create(id_col->size());
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::create(id_col->size());
|
||||
const auto & ids = id_col->getData();
|
||||
auto & data = out->getData();
|
||||
const auto & defs = default_col->getData();
|
||||
|
||||
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto default_col = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
|
||||
{
|
||||
/// vector ids, const defaults
|
||||
const auto out = ColumnVector<Type>::create(id_col->size());
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::create(id_col->size());
|
||||
const auto & ids = id_col->getData();
|
||||
auto & data = out->getData();
|
||||
const auto def = default_col->template getValue<Type>();
|
||||
|
||||
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
@ -1259,13 +1232,11 @@ private:
|
||||
/// @todo avoid materialization
|
||||
const PaddedPODArray<UInt64> ids(id_col->size(), id_col->getValue<UInt64>());
|
||||
|
||||
const auto out = ColumnVector<Type>::create(id_col->size());
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
auto out = ColumnVector<Type>::create(id_col->size());
|
||||
auto & data = out->getData();
|
||||
const auto & defs = default_col->getData();
|
||||
|
||||
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto default_col = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
|
||||
{
|
||||
@ -1273,9 +1244,7 @@ private:
|
||||
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
|
||||
PaddedPODArray<Type> data(1);
|
||||
const auto & def = default_col->template getValue<Type>();
|
||||
|
||||
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
|
||||
|
||||
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
|
||||
}
|
||||
else
|
||||
@ -1312,9 +1281,7 @@ private:
|
||||
|
||||
/// @todo detect when all key columns are constant
|
||||
const auto rows = key_col->size();
|
||||
const auto out = ColumnVector<Type>::create(rows);
|
||||
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnVector<Type>::create(rows);
|
||||
auto & data = out->getData();
|
||||
|
||||
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
||||
@ -1336,6 +1303,7 @@ private:
|
||||
"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()),
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -1638,8 +1606,7 @@ private:
|
||||
{
|
||||
if (const auto ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
||||
{
|
||||
const auto out = ColumnUInt8::create();
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnUInt8::create();
|
||||
|
||||
const auto & child_ids = child_id_col->getData();
|
||||
const auto & ancestor_ids = ancestor_id_col->getData();
|
||||
@ -1648,11 +1615,11 @@ private:
|
||||
data.resize(size);
|
||||
|
||||
dictionary->isInVectorVector(child_ids, ancestor_ids, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto ancestor_id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
|
||||
{
|
||||
const auto out = ColumnUInt8::create();
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnUInt8::create();
|
||||
|
||||
const auto & child_ids = child_id_col->getData();
|
||||
const auto ancestor_id = ancestor_id_col->getValue<UInt64>();
|
||||
@ -1661,6 +1628,7 @@ private:
|
||||
data.resize(size);
|
||||
|
||||
dictionary->isInVectorConstant(child_ids, ancestor_id, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1679,8 +1647,7 @@ private:
|
||||
{
|
||||
if (const auto ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
||||
{
|
||||
const auto out = ColumnUInt8::create();
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnUInt8::create();
|
||||
|
||||
const auto child_id = child_id_col->getValue<UInt64>();
|
||||
const auto & ancestor_ids = ancestor_id_col->getData();
|
||||
@ -1689,6 +1656,7 @@ private:
|
||||
data.resize(size);
|
||||
|
||||
dictionary->isInConstantVector(child_id, ancestor_ids, data);
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
else if (const auto ancestor_id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
|
||||
{
|
||||
@ -1697,7 +1665,6 @@ private:
|
||||
UInt8 res = 0;
|
||||
|
||||
dictionary->isInConstantConstant(child_id, ancestor_id, res);
|
||||
|
||||
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(child_id_col->size(), UInt64(res));
|
||||
}
|
||||
else
|
||||
|
@ -140,7 +140,7 @@ public:
|
||||
|
||||
executeImplTyped(in_untyped, out_untyped, centroids_array_untyped);
|
||||
|
||||
block.getByPosition(result).column = column_result;
|
||||
block.getByPosition(result).column = std::move(column_result);
|
||||
}
|
||||
|
||||
protected:
|
||||
|
@ -92,7 +92,6 @@ private:
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
@ -110,6 +109,8 @@ private:
|
||||
offsets_to[i] = buf_to.count();
|
||||
}
|
||||
data_to.resize(buf_to.count());
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -170,7 +171,6 @@ private:
|
||||
if (const ColumnVector<T> * col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
@ -188,13 +188,12 @@ private:
|
||||
offsets_to[i] = buf_to.count();
|
||||
}
|
||||
data_to.resize(buf_to.count());
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
return true;
|
||||
}
|
||||
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -136,7 +136,6 @@ private:
|
||||
else
|
||||
{
|
||||
const auto dst = ColumnVector<Float64>::create();
|
||||
block.getByPosition(result).column = dst;
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(size);
|
||||
Float64 vals[instrs.size()];
|
||||
@ -153,6 +152,7 @@ private:
|
||||
}
|
||||
dst_data[row] = greatCircleDistance(vals[0], vals[1], vals[2], vals[3]);
|
||||
}
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -274,7 +274,6 @@ private:
|
||||
const auto col_vec_y = static_cast<const ColumnVector<Float64> *> (col_y);
|
||||
|
||||
const auto dst = ColumnVector<UInt8>::create();
|
||||
block.getByPosition(result).column = dst;
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(size);
|
||||
|
||||
@ -283,6 +282,8 @@ private:
|
||||
{
|
||||
dst_data[row] = isPointInEllipses(col_vec_x->getData()[row], col_vec_y->getData()[row], ellipses, ellipses_count, start_index);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
}
|
||||
else if (const_cnt == 2)
|
||||
{
|
||||
|
@ -206,7 +206,6 @@ public:
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnUInt64::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnString::Chars_t & data = col_from->getChars();
|
||||
const typename ColumnString::Offsets & offsets = col_from->getOffsets();
|
||||
@ -218,6 +217,8 @@ public:
|
||||
vec_to[i] = Impl::apply(
|
||||
reinterpret_cast<const char *>(&data[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -257,7 +258,6 @@ public:
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnFixedString::create(Impl::length);
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnString::Chars_t & data = col_from->getChars();
|
||||
const typename ColumnString::Offsets & offsets = col_from->getOffsets();
|
||||
@ -270,6 +270,8 @@ public:
|
||||
reinterpret_cast<const char *>(&data[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]),
|
||||
&chars_to[i * Impl::length]);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -295,7 +297,6 @@ private:
|
||||
if (auto col_from = checkAndGetColumn<ColumnVector<FromType>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromType>::Container & vec_from = col_from->getData();
|
||||
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
|
||||
@ -304,6 +305,8 @@ private:
|
||||
vec_to.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Impl::apply(vec_from[i]);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -587,7 +590,6 @@ public:
|
||||
{
|
||||
size_t rows = block.rows();
|
||||
auto col_to = ColumnUInt64::create(rows);
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
ColumnUInt64::Container & vec_to = col_to->getData();
|
||||
|
||||
@ -620,6 +622,8 @@ public:
|
||||
|
||||
if (all_constants && block.rows() > 0)
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(1, (*block.getByPosition(result).column)[0]);
|
||||
else
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -759,7 +763,6 @@ private:
|
||||
{
|
||||
const auto size = col_from->size();
|
||||
const auto col_to = ColumnUInt64::create(size);
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const auto & chars = col_from->getChars();
|
||||
const auto & offsets = col_from->getOffsets();
|
||||
@ -769,6 +772,8 @@ private:
|
||||
out[i] = URLHashImpl::apply(
|
||||
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
@ -792,7 +797,6 @@ private:
|
||||
{
|
||||
const auto size = col_from->size();
|
||||
const auto col_to = ColumnUInt64::create(size);
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const auto & chars = col_from->getChars();
|
||||
const auto & offsets = col_from->getOffsets();
|
||||
@ -802,6 +806,8 @@ private:
|
||||
out[i] = URLHierarchyHashImpl::apply(level,
|
||||
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
|
@ -293,8 +293,7 @@ public:
|
||||
{
|
||||
if (!in.empty())
|
||||
const_val = Impl<UInt8>::apply(const_val, 0);
|
||||
auto col_res = DataTypeUInt8().createColumnConst(rows, toField(const_val));
|
||||
block.getByPosition(result).column = col_res;
|
||||
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(rows, toField(const_val));
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -92,7 +92,6 @@ private:
|
||||
if (const auto col = checkAndGetColumn<ColumnVector<FieldType>>(arg))
|
||||
{
|
||||
const auto dst = ColumnVector<Float64>::create();
|
||||
block.getByPosition(result).column = dst;
|
||||
|
||||
const auto & src_data = col->getData();
|
||||
const auto src_size = src_data.size();
|
||||
@ -117,6 +116,7 @@ private:
|
||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -222,7 +222,6 @@ private:
|
||||
if (const auto right_arg_typed = checkAndGetColumn<ColumnVector<RightType>>(right_arg))
|
||||
{
|
||||
const auto dst = ColumnVector<Float64>::create();
|
||||
block.getByPosition(result).column = dst;
|
||||
|
||||
LeftType left_src_data[Impl::rows_per_iteration];
|
||||
std::fill(std::begin(left_src_data), std::end(left_src_data), left_arg->template getValue<LeftType>());
|
||||
@ -249,6 +248,7 @@ private:
|
||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -262,7 +262,6 @@ private:
|
||||
if (const auto right_arg_typed = checkAndGetColumn<ColumnVector<RightType>>(right_arg))
|
||||
{
|
||||
const auto dst = ColumnVector<Float64>::create();
|
||||
block.getByPosition(result).column = dst;
|
||||
|
||||
const auto & left_src_data = left_arg->getData();
|
||||
const auto & right_src_data = right_arg_typed->getData();
|
||||
@ -291,12 +290,12 @@ private:
|
||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
else if (const auto right_arg_typed = checkAndGetColumnConst<ColumnVector<RightType>>(right_arg))
|
||||
{
|
||||
const auto dst = ColumnVector<Float64>::create();
|
||||
block.getByPosition(result).column = dst;
|
||||
|
||||
const auto & left_src_data = left_arg->getData();
|
||||
RightType right_src_data[Impl::rows_per_iteration];
|
||||
@ -323,6 +322,7 @@ private:
|
||||
memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64));
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -455,7 +455,7 @@ public:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
data[i] = i;
|
||||
|
||||
block.getByPosition(result).column = column;
|
||||
block.getByPosition(result).column = std::move(column);
|
||||
}
|
||||
};
|
||||
|
||||
@ -547,7 +547,7 @@ public:
|
||||
for (size_t i = 0; i < rows_in_block; ++i)
|
||||
data[i] = current_row_number + i;
|
||||
|
||||
block.getByPosition(result).column = column;
|
||||
block.getByPosition(result).column = std::move(column);
|
||||
}
|
||||
};
|
||||
|
||||
@ -1152,7 +1152,6 @@ public:
|
||||
const auto & src = *block.getByPosition(arguments[0]).column;
|
||||
|
||||
auto res_column = ColumnString::create();
|
||||
block.getByPosition(result).column = res_column;
|
||||
|
||||
if (executeNumber<UInt8>(src, *res_column, min, max, max_width) || executeNumber<UInt16>(src, *res_column, min, max, max_width)
|
||||
|| executeNumber<UInt32>(src, *res_column, min, max, max_width)
|
||||
@ -1164,6 +1163,7 @@ 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);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
@ -1283,8 +1283,7 @@ public:
|
||||
{
|
||||
const auto size = in->size();
|
||||
|
||||
const auto out = ColumnUInt8::create(size);
|
||||
block.getByPosition(result).column = out;
|
||||
auto out = ColumnUInt8::create(size);
|
||||
|
||||
const auto & in_data = in->getData();
|
||||
auto & out_data = out->getData();
|
||||
@ -1292,6 +1291,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);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -1534,8 +1534,7 @@ public:
|
||||
|
||||
std::unique_ptr<Arena> arena = agg_func.allocatesMemoryInArena() ? std::make_unique<Arena>() : nullptr;
|
||||
|
||||
ColumnPtr result_column_ptr = agg_func.getReturnType()->createColumn();
|
||||
block.getByPosition(result).column = result_column_ptr;
|
||||
auto result_column_ptr = agg_func.getReturnType()->createColumn();
|
||||
IColumn & result_column = *result_column_ptr;
|
||||
result_column.reserve(column_with_states->size());
|
||||
|
||||
@ -1546,6 +1545,8 @@ public:
|
||||
agg_func.merge(place.get(), state_to_add, arena.get());
|
||||
agg_func.insertResultInto(place.get(), result_column);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column_ptr);
|
||||
}
|
||||
};
|
||||
|
||||
@ -1782,7 +1783,6 @@ void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & argu
|
||||
|
||||
auto res_col = ColumnUInt64::create(size);
|
||||
auto & res_data = static_cast<ColumnUInt64 &>(*res_col).getData();
|
||||
block.getByPosition(result).column = res_col;
|
||||
|
||||
/// For simplicity reasons, function is implemented by serializing into temporary buffer.
|
||||
|
||||
@ -1796,6 +1796,8 @@ void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & argu
|
||||
|
||||
res_data[i] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(tmp.data()), tmp.size());
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(res_col);
|
||||
}
|
||||
|
||||
|
||||
|
@ -216,13 +216,13 @@ void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & argument
|
||||
|
||||
FunctionMultiIf{context}.execute(temp_block, multi_if_args, result);
|
||||
|
||||
auto res = std::move(temp_block.getByPosition(result).column);
|
||||
auto res = std::move(temp_block.getByPosition(result).column)->mutate();
|
||||
|
||||
/// if last argument is not nullable, result should be also not nullable
|
||||
if (!block.getByPosition(multi_if_args.back()).column->isColumnNullable() && res->isColumnNullable())
|
||||
res = static_cast<ColumnNullable &>(*res).getNestedColumnPtr();
|
||||
|
||||
block.getByPosition(result).column = res;
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
|
||||
/// Implementation of ifNull.
|
||||
|
@ -153,13 +153,13 @@ public:
|
||||
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
typename ColumnVector<ToType>::Container & vec_to = col_to->getData();
|
||||
|
||||
size_t size = block.rows();
|
||||
vec_to.resize(size);
|
||||
Impl::execute(&vec_to[0], vec_to.size());
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -56,7 +56,6 @@ public:
|
||||
if (auto col_from = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<T>::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
@ -80,6 +79,8 @@ public:
|
||||
offsets_to[i] = pos;
|
||||
}
|
||||
data_to.resize(pos);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -142,7 +143,6 @@ public:
|
||||
if (ColumnString * col_from = typeid_cast<ColumnString *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ToFieldType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnString::Chars_t & data_from = col_from->getChars();
|
||||
ColumnString::Offsets & offsets_from = col_from->getOffsets();
|
||||
@ -158,11 +158,12 @@ public:
|
||||
vec_res[i] = value;
|
||||
offset = offsets_from[i];
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else if (ColumnFixedString * col_from = typeid_cast<ColumnFixedString *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ToFieldType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnString::Chars_t & data_from = col_from->getChars();
|
||||
size_t step = col_from->getN();
|
||||
@ -179,6 +180,8 @@ public:
|
||||
vec_res[i] = value;
|
||||
offset += step;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -491,13 +491,15 @@ struct Dispatcher
|
||||
}
|
||||
|
||||
auto col_res = ColumnVector<T>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<T>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(col->getData().size());
|
||||
|
||||
if (vec_res.empty())
|
||||
{
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return;
|
||||
}
|
||||
|
||||
if (scale_arg == 0)
|
||||
{
|
||||
@ -514,6 +516,8 @@ struct Dispatcher
|
||||
scale = pow(10, -scale_arg);
|
||||
FunctionRoundingImpl<T, rounding_mode, ScaleMode::Negative>::apply(col->getData(), scale, vec_res);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -542,11 +542,12 @@ public:
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(col->size());
|
||||
Impl::vector(col->getChars(), col->getOffsets(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
@ -561,21 +562,23 @@ public:
|
||||
else
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(col->size());
|
||||
Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
}
|
||||
else if (const ColumnArray * col = checkAndGetColumn<ColumnArray>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(col->size());
|
||||
Impl::array(col->getOffsets(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
@ -743,7 +746,7 @@ private:
|
||||
return;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = c_res;
|
||||
block.getByPosition(result).column = std::move(c_res);
|
||||
}
|
||||
|
||||
void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
@ -756,7 +759,7 @@ private:
|
||||
|
||||
auto c_res = ColumnString::create();
|
||||
concat(sources, StringSink(*c_res, block.rows()));
|
||||
block.getByPosition(result).column = c_res;
|
||||
block.getByPosition(result).column = std::move(c_res);
|
||||
}
|
||||
};
|
||||
|
||||
@ -1029,7 +1032,6 @@ private:
|
||||
if (const auto col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
const auto & src_data = col->getChars();
|
||||
const auto & src_offsets = col->getOffsets();
|
||||
@ -1062,6 +1064,7 @@ private:
|
||||
}
|
||||
|
||||
dst_data.resize_assume_reserved(dst_offset);
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
|
@ -91,7 +91,6 @@ public:
|
||||
}
|
||||
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(column_haystack->size());
|
||||
@ -115,6 +114,8 @@ public:
|
||||
+ " of arguments of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user