Better semantic of sharing columns: development [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-12-16 07:59:32 +03:00
parent 5b32914089
commit 45805a367f
3 changed files with 36 additions and 46 deletions

View File

@ -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; }

View File

@ -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{

View File

@ -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)
{