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
5b32914089
commit
45805a367f
@ -83,7 +83,7 @@ public:
|
||||
|
||||
//MutableColumnPtr getDataMutablePtr() { return data->assumeMutable(); }
|
||||
const ColumnPtr & getDataPtr() const { return data; }
|
||||
//ColumnPtr & getDataPtr() { return data; }
|
||||
ColumnPtr & getDataPtr() { return data; }
|
||||
|
||||
//MutableColumnPtr getOffsetsMutablePtr() { return offsets->assumeMutable(); }
|
||||
const ColumnPtr & getOffsetsPtr() const { return offsets; }
|
||||
|
@ -75,7 +75,6 @@ struct ConvertImpl
|
||||
= checkAndGetColumn<ColumnVector<FromFieldType>>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToFieldType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container & vec_from = col_from->getData();
|
||||
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
|
||||
@ -84,6 +83,8 @@ struct ConvertImpl
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -217,7 +218,6 @@ struct ConvertImpl<FromDataType, typename std::enable_if<!std::is_same<FromDataT
|
||||
if (const auto col_from = checkAndGetColumn<ColumnVector<FromFieldType>>(col_with_type_and_name.column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container & vec_from = col_from->getData();
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
@ -243,6 +243,8 @@ struct ConvertImpl<FromDataType, typename std::enable_if<!std::is_same<FromDataT
|
||||
}
|
||||
|
||||
data_to.resize(write_buffer.count());
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -264,7 +266,6 @@ struct ConvertImplGenericToString
|
||||
size_t size = col_from.size();
|
||||
|
||||
auto col_to = ColumnString::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
ColumnString::Offsets & offsets_to = col_to->getOffsets();
|
||||
@ -282,6 +283,7 @@ struct ConvertImplGenericToString
|
||||
}
|
||||
|
||||
data_to.resize(write_buffer.count());
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
};
|
||||
|
||||
@ -335,7 +337,6 @@ struct ConvertImpl<typename std::enable_if<!std::is_same<ToDataType, DataTypeStr
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToFieldType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
|
||||
size_t size = col_from->size();
|
||||
@ -359,6 +360,8 @@ struct ConvertImpl<typename std::enable_if<!std::is_same<ToDataType, DataTypeStr
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -390,7 +393,6 @@ struct ConvertOrZeroImpl
|
||||
if (const ColumnString * col_from = checkAndGetColumn<ColumnString>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToFieldType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
typename ColumnVector<ToFieldType>::Container & vec_to = col_to->getData();
|
||||
size_t size = col_from->size();
|
||||
@ -411,6 +413,8 @@ struct ConvertOrZeroImpl
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -428,17 +432,16 @@ struct ConvertImplGenericFromString
|
||||
const IColumn & col_from = *block.getByPosition(arguments[0]).column;
|
||||
size_t size = col_from.size();
|
||||
|
||||
ColumnWithTypeAndName & column_type_name_to = block.getByPosition(result);
|
||||
const IDataType & data_type_to = *column_type_name_to.type;
|
||||
const IDataType & data_type_to = *block.getByPosition(result).type;
|
||||
|
||||
if (const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(&col_from))
|
||||
{
|
||||
column_type_name_to.column = data_type_to.createColumn();
|
||||
auto res = data_type_to.createColumn();
|
||||
|
||||
if (!size)
|
||||
return;
|
||||
|
||||
IColumn & column_to = *column_type_name_to.column;
|
||||
IColumn & column_to = *res;
|
||||
column_to.reserve(size);
|
||||
|
||||
const ColumnString::Chars_t & chars = col_from_string->getChars();
|
||||
@ -457,6 +460,8 @@ struct ConvertImplGenericFromString
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -504,7 +509,6 @@ struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
||||
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1);
|
||||
|
||||
auto col_to = ColumnVector<ToFieldType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const ColumnFixedString::Chars_t & data_from = col_from->getChars();
|
||||
size_t n = col_from->getN();
|
||||
@ -527,6 +531,8 @@ struct ConvertImpl<DataTypeFixedString, ToDataType, Name>
|
||||
throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
|
||||
}
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -547,7 +553,6 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto col_to = ColumnString::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const ColumnFixedString::Chars_t & data_from = col_from->getChars();
|
||||
ColumnString::Chars_t & data_to = col_to->getChars();
|
||||
@ -574,6 +579,7 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
|
||||
}
|
||||
|
||||
data_to.resize(offset_to);
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -839,8 +845,7 @@ public:
|
||||
|
||||
if (const auto column_string = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
const auto column_fixed = ColumnFixedString::create(n);
|
||||
ColumnPtr result_ptr = column_fixed;
|
||||
auto column_fixed = ColumnFixedString::create(n);
|
||||
|
||||
auto & out_chars = column_fixed->getChars();
|
||||
const auto & in_chars = column_string->getChars();
|
||||
@ -858,7 +863,7 @@ public:
|
||||
memcpy(&out_chars[i * n], &in_chars[off], len);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = result_ptr;
|
||||
block.getByPosition(result).column = std::move(column_fixed);
|
||||
}
|
||||
else if (const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
@ -868,8 +873,7 @@ public:
|
||||
"String too long for type FixedString(" + toString(n) + ")",
|
||||
ErrorCodes::TOO_LARGE_STRING_SIZE};
|
||||
|
||||
const auto column_fixed = ColumnFixedString::create(n);
|
||||
block.getByPosition(result).column = column_fixed;
|
||||
auto column_fixed = ColumnFixedString::create(n);
|
||||
|
||||
auto & out_chars = column_fixed->getChars();
|
||||
const auto & in_chars = column_fixed_string->getChars();
|
||||
@ -878,6 +882,8 @@ public:
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
memcpy(&out_chars[i * n], &in_chars[i * src_n], src_n);
|
||||
|
||||
block.getByPosition(result).column = std::move(column_fixed);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -1158,11 +1164,12 @@ private:
|
||||
};
|
||||
}
|
||||
|
||||
DataTypePtr from_nested_type, to_nested_type;
|
||||
DataTypePtr from_nested_type;
|
||||
DataTypePtr to_nested_type;
|
||||
auto from_type = checkAndGetDataType<DataTypeArray>(from_type_untyped.get());
|
||||
|
||||
/// get the most nested type
|
||||
while (from_type && to_type)
|
||||
if (from_type && to_type)
|
||||
{
|
||||
from_nested_type = from_type->getNestedType();
|
||||
to_nested_type = to_type->getNestedType();
|
||||
@ -1183,23 +1190,10 @@ private:
|
||||
return [nested_function, from_nested_type, to_nested_type](
|
||||
Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
auto array_arg = block.getByPosition(arguments.front());
|
||||
const auto & array_arg = block.getByPosition(arguments.front());
|
||||
|
||||
if (auto col_array = checkAndGetColumn<ColumnArray>(array_arg.column.get()))
|
||||
if (const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(array_arg.column.get()))
|
||||
{
|
||||
auto res = new ColumnArray(nullptr, col_array->getOffsetsPtr());
|
||||
block.getByPosition(result).column.reset(res);
|
||||
|
||||
/// get the most nested column
|
||||
while (const auto nested_col_array = checkAndGetColumn<ColumnArray>(col_array->getDataPtr().get()))
|
||||
{
|
||||
/// create new level of array, copy offsets
|
||||
res->getDataPtr() = ColumnArray::create(nullptr, nested_col_array->getOffsetsPtr());
|
||||
|
||||
res = static_cast<ColumnArray *>(&res->getData());
|
||||
col_array = nested_col_array;
|
||||
}
|
||||
|
||||
/// create block for converting nested column containing original and result columns
|
||||
Block nested_block
|
||||
{
|
||||
@ -1207,12 +1201,11 @@ private:
|
||||
{ nullptr, to_nested_type, "" }
|
||||
};
|
||||
|
||||
const auto nested_result = 1;
|
||||
/// convert nested column
|
||||
nested_function(nested_block, {0}, nested_result);
|
||||
nested_function(nested_block, {0}, 1);
|
||||
|
||||
/// set converted nested column to result
|
||||
res->getDataPtr() = nested_block.getByPosition(nested_result).column;
|
||||
block.getByPosition(result).column = ColumnArray::create(nested_block.getByPosition(1).column, col_array->getOffsetsPtr());
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
@ -1362,7 +1355,6 @@ private:
|
||||
const auto first_col = block.getByPosition(arguments.front()).column.get();
|
||||
|
||||
auto & col_with_type_and_name = block.getByPosition(result);
|
||||
auto & result_col = col_with_type_and_name.column;
|
||||
const auto & result_type = typeid_cast<EnumType &>(*col_with_type_and_name.type);
|
||||
|
||||
if (const auto col = typeid_cast<const ColumnStringType *>(first_col))
|
||||
@ -1376,7 +1368,7 @@ private:
|
||||
for (const auto i : ext::range(0, size))
|
||||
out_data[i] = result_type.getValue(col->getDataAt(i));
|
||||
|
||||
result_col = res;
|
||||
col_with_type_and_name.column = std::move(res);
|
||||
}
|
||||
else
|
||||
throw Exception{
|
||||
|
@ -555,8 +555,8 @@ struct DateTimeTransformImpl
|
||||
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
|
||||
{
|
||||
auto col_to = ColumnVector<ToType>::create();
|
||||
block.getByPosition(result).column = col_to;
|
||||
Op::vector(sources->getData(), col_to->getData(), time_zone);
|
||||
block.getByPosition(result).column = std::move(col_to);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1110,7 +1110,6 @@ public:
|
||||
if (const ColumnUInt32 * times = typeid_cast<const ColumnUInt32 *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
auto res = ColumnUInt32::create();
|
||||
ColumnPtr res_holder = res;
|
||||
ColumnUInt32::Container & res_vec = res->getData();
|
||||
const ColumnUInt32::Container & vec = times->getData();
|
||||
|
||||
@ -1120,7 +1119,7 @@ public:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
res_vec[i] = vec[i] / TIME_SLOT_SIZE * TIME_SLOT_SIZE;
|
||||
|
||||
block.getByPosition(result).column = res_holder;
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
@ -1244,23 +1243,22 @@ public:
|
||||
auto const_durations = checkAndGetColumnConst<ColumnUInt32>(block.getByPosition(arguments[1]).column.get());
|
||||
|
||||
auto res = ColumnArray::create(ColumnUInt32::create());
|
||||
ColumnPtr res_holder = res;
|
||||
ColumnUInt32::Container & res_values = typeid_cast<ColumnUInt32 &>(res->getData()).getData();
|
||||
|
||||
if (starts && durations)
|
||||
{
|
||||
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets());
|
||||
block.getByPosition(result).column = res_holder;
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
else if (starts && const_durations)
|
||||
{
|
||||
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getValue<UInt32>(), res_values, res->getOffsets());
|
||||
block.getByPosition(result).column = res_holder;
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
else if (const_starts && durations)
|
||||
{
|
||||
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getValue<UInt32>(), durations->getData(), res_values, res->getOffsets());
|
||||
block.getByPosition(result).column = res_holder;
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
else if (const_starts && const_durations)
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user