ClickHouse/src/Columns/ColumnObject.cpp

772 lines
22 KiB
C++
Raw Normal View History

#include <Core/Field.h>
2021-04-23 12:53:38 +00:00
#include <Columns/ColumnObject.h>
2022-02-17 19:00:25 +00:00
#include <Columns/ColumnsNumber.h>
2021-12-01 02:58:24 +00:00
#include <Columns/ColumnArray.h>
#include <DataTypes/ObjectUtils.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeFactory.h>
2021-12-01 02:58:24 +00:00
#include <DataTypes/NestedUtils.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/convertFieldToType.h>
2021-08-20 21:11:22 +00:00
#include <Common/HashTable/HashSet.h>
2021-04-23 12:53:38 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN;
extern const int DUPLICATE_COLUMN;
extern const int NUMBER_OF_DIMENSIONS_MISMATHED;
2021-09-14 14:02:32 +00:00
extern const int NOT_IMPLEMENTED;
}
namespace
{
2022-02-17 19:00:25 +00:00
/// Recreates scolumn with default scalar values and keeps sizes of arrays.
2021-12-23 12:28:40 +00:00
ColumnPtr recreateColumnWithDefaultValues(
const ColumnPtr & column, const DataTypePtr & scalar_type, size_t num_dimensions)
2021-12-01 02:58:24 +00:00
{
2021-12-23 12:28:40 +00:00
const auto * column_array = checkAndGetColumn<ColumnArray>(column.get());
if (column_array && num_dimensions)
{
2021-12-13 11:37:23 +00:00
return ColumnArray::create(
2021-12-23 12:28:40 +00:00
recreateColumnWithDefaultValues(
column_array->getDataPtr(), scalar_type, num_dimensions - 1),
2021-12-13 11:37:23 +00:00
IColumn::mutate(column_array->getOffsetsPtr()));
2021-12-23 12:28:40 +00:00
}
return createArrayOfType(scalar_type, num_dimensions)->createColumn()->cloneResized(column->size());
2021-12-01 02:58:24 +00:00
}
2022-02-17 19:00:25 +00:00
/// Replaces NULL fields to given field or empty array.
class FieldVisitorReplaceNull : public StaticVisitor<Field>
{
public:
2022-02-17 19:00:25 +00:00
explicit FieldVisitorReplaceNull(
const Field & replacement_, size_t num_dimensions_)
: replacement(replacement_)
, num_dimensions(num_dimensions_)
{
}
2022-02-17 19:00:25 +00:00
Field operator()(const Null &) const
{
2022-02-17 19:00:25 +00:00
return num_dimensions
? createEmptyArrayField(num_dimensions)
: replacement;
}
2022-02-17 19:00:25 +00:00
Field operator()(const Array & x) const
{
assert(num_dimensions > 0);
const size_t size = x.size();
Array res(size);
for (size_t i = 0; i < size; ++i)
res[i] = applyVisitor(FieldVisitorReplaceNull(replacement, num_dimensions - 1), x[i]);
return res;
}
template <typename T>
Field operator()(const T & x) const { return x; }
private:
const Field & replacement;
size_t num_dimensions;
};
2022-02-17 19:00:25 +00:00
/// Calculates number of dimensions in array field.
/// Returns 0 for scalar fields.
class FieldVisitorToNumberOfDimensions : public StaticVisitor<size_t>
{
public:
size_t operator()(const Array & x) const
{
const size_t size = x.size();
std::optional<size_t> dimensions;
for (size_t i = 0; i < size; ++i)
{
/// Do not count Nulls, because they will be replaced by default
/// values with proper number of dimensions.
if (x[i].isNull())
continue;
size_t current_dimensions = applyVisitor(*this, x[i]);
if (!dimensions)
dimensions = current_dimensions;
else if (current_dimensions != *dimensions)
throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED,
"Number of dimensions mismatched among array elements");
}
return 1 + dimensions.value_or(0);
}
template <typename T>
size_t operator()(const T &) const { return 0; }
};
2022-02-17 19:00:25 +00:00
/// Visitor that allows to get type of scalar field
/// or least common type of scalars in array.
/// More optimized version of FieldToDataType.
2021-12-01 02:58:24 +00:00
class FieldVisitorToScalarType : public StaticVisitor<>
{
public:
using FieldType = Field::Types::Which;
void operator()(const Array & x)
{
size_t size = x.size();
for (size_t i = 0; i < size; ++i)
applyVisitor(*this, x[i]);
}
void operator()(const UInt64 & x)
{
field_types.insert(FieldType::UInt64);
if (x <= std::numeric_limits<UInt8>::max())
type_indexes.insert(TypeIndex::UInt8);
else if (x <= std::numeric_limits<UInt16>::max())
type_indexes.insert(TypeIndex::UInt16);
else if (x <= std::numeric_limits<UInt32>::max())
type_indexes.insert(TypeIndex::UInt32);
else
type_indexes.insert(TypeIndex::UInt64);
}
void operator()(const Int64 & x)
{
field_types.insert(FieldType::Int64);
if (x <= std::numeric_limits<Int8>::max() && x >= std::numeric_limits<Int8>::min())
type_indexes.insert(TypeIndex::Int8);
else if (x <= std::numeric_limits<Int16>::max() && x >= std::numeric_limits<Int16>::min())
type_indexes.insert(TypeIndex::Int16);
else if (x <= std::numeric_limits<Int32>::max() && x >= std::numeric_limits<Int32>::min())
type_indexes.insert(TypeIndex::Int32);
else
type_indexes.insert(TypeIndex::Int64);
}
void operator()(const Null &)
{
have_nulls = true;
}
template <typename T>
void operator()(const T &)
{
2022-02-17 19:00:25 +00:00
field_types.insert(Field::TypeToEnum<NearestFieldType<T>>::value);
type_indexes.insert(TypeToTypeIndex<NearestFieldType<T>>);
}
2021-12-13 11:37:23 +00:00
DataTypePtr getScalarType() const { return getLeastSupertype(type_indexes, true); }
bool haveNulls() const { return have_nulls; }
bool needConvertField() const { return field_types.size() > 1; }
private:
TypeIndexSet type_indexes;
std::unordered_set<FieldType> field_types;
bool have_nulls = false;
};
2021-04-23 12:53:38 +00:00
}
2021-12-17 15:08:34 +00:00
FieldInfo getFieldInfo(const Field & field)
{
FieldVisitorToScalarType to_scalar_type_visitor;
applyVisitor(to_scalar_type_visitor, field);
return
{
to_scalar_type_visitor.getScalarType(),
to_scalar_type_visitor.haveNulls(),
to_scalar_type_visitor.needConvertField(),
applyVisitor(FieldVisitorToNumberOfDimensions(), field),
};
}
2021-12-01 02:58:24 +00:00
ColumnObject::Subcolumn::Subcolumn(MutableColumnPtr && data_, bool is_nullable_)
: least_common_type(getDataTypeByColumn(*data_))
2021-12-01 02:58:24 +00:00
, is_nullable(is_nullable_)
{
data.push_back(std::move(data_));
}
2021-12-01 02:58:24 +00:00
ColumnObject::Subcolumn::Subcolumn(
size_t size_, bool is_nullable_)
: least_common_type(std::make_shared<DataTypeNothing>())
, is_nullable(is_nullable_)
, num_of_defaults_in_prefix(size_)
{
}
size_t ColumnObject::Subcolumn::Subcolumn::size() const
{
size_t res = num_of_defaults_in_prefix;
for (const auto & part : data)
res += part->size();
return res;
}
size_t ColumnObject::Subcolumn::Subcolumn::byteSize() const
{
size_t res = 0;
for (const auto & part : data)
res += part->byteSize();
return res;
}
size_t ColumnObject::Subcolumn::Subcolumn::allocatedBytes() const
{
size_t res = 0;
for (const auto & part : data)
res += part->allocatedBytes();
return res;
}
void ColumnObject::Subcolumn::checkTypes() const
{
DataTypes prefix_types;
prefix_types.reserve(data.size());
for (size_t i = 0; i < data.size(); ++i)
{
auto current_type = getDataTypeByColumn(*data[i]);
prefix_types.push_back(current_type);
auto prefix_common_type = getLeastSupertype(prefix_types);
if (!prefix_common_type->equals(*current_type))
throw Exception(ErrorCodes::LOGICAL_ERROR,
2021-09-14 14:02:32 +00:00
"Data type {} of column at position {} cannot represent all columns from i-th prefix",
current_type->getName(), i);
}
}
2021-08-20 21:11:22 +00:00
void ColumnObject::Subcolumn::insert(Field field)
2021-04-23 12:53:38 +00:00
{
2021-12-17 15:08:34 +00:00
auto info = getFieldInfo(field);
insert(std::move(field), std::move(info));
}
2021-12-13 11:37:23 +00:00
2021-12-17 15:08:34 +00:00
void ColumnObject::Subcolumn::insert(Field field, FieldInfo info)
{
auto base_type = info.scalar_type;
2021-12-13 11:37:23 +00:00
2022-01-21 14:55:26 +00:00
if (isNothing(base_type) && info.num_dimensions == 0)
2021-12-13 11:37:23 +00:00
{
insertDefault();
return;
}
auto column_dim = getNumberOfDimensions(*least_common_type);
2021-12-17 15:08:34 +00:00
auto value_dim = info.num_dimensions;
if (isNothing(least_common_type))
column_dim = value_dim;
if (field.isNull())
value_dim = column_dim;
if (value_dim != column_dim)
throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED,
2021-12-13 11:37:23 +00:00
"Dimension of types mismatched between inserted value and column. "
"Dimension of value: {}. Dimension of column: {}",
value_dim, column_dim);
2021-12-13 11:37:23 +00:00
if (is_nullable)
base_type = makeNullable(base_type);
2021-12-17 15:08:34 +00:00
if (!is_nullable && info.have_nulls)
2022-02-17 19:00:25 +00:00
field = applyVisitor(FieldVisitorReplaceNull(base_type->getDefault(), value_dim), std::move(field));
2022-02-17 19:00:25 +00:00
auto value_type = createArrayOfType(base_type, value_dim);
bool type_changed = false;
if (data.empty())
{
data.push_back(value_type->createColumn());
least_common_type = value_type;
}
else if (!least_common_type->equals(*value_type))
{
value_type = getLeastSupertype(DataTypes{value_type, least_common_type}, true);
type_changed = true;
if (!least_common_type->equals(*value_type))
{
data.push_back(value_type->createColumn());
least_common_type = value_type;
}
}
2021-12-17 15:08:34 +00:00
if (type_changed || info.need_convert)
2022-02-17 19:00:25 +00:00
field = convertFieldToTypeOrThrow(std::move(field), *value_type);
data.back()->insert(std::move(field));
2021-04-23 12:53:38 +00:00
}
2021-08-10 01:33:57 +00:00
void ColumnObject::Subcolumn::insertRangeFrom(const Subcolumn & src, size_t start, size_t length)
{
assert(src.isFinalized());
const auto & src_column = src.data.back();
const auto & src_type = src.least_common_type;
if (data.empty())
{
least_common_type = src_type;
data.push_back(src_type->createColumn());
data.back()->insertRangeFrom(*src_column, start, length);
}
else if (least_common_type->equals(*src_type))
{
data.back()->insertRangeFrom(*src_column, start, length);
}
else
{
2021-08-20 21:11:22 +00:00
auto new_least_common_type = getLeastSupertype(DataTypes{least_common_type, src_type}, true);
2021-08-10 01:33:57 +00:00
auto casted_column = castColumn({src_column, src_type, ""}, new_least_common_type);
if (!least_common_type->equals(*new_least_common_type))
{
least_common_type = new_least_common_type;
data.push_back(least_common_type->createColumn());
}
data.back()->insertRangeFrom(*casted_column, start, length);
}
}
void ColumnObject::Subcolumn::finalize()
2021-04-23 12:53:38 +00:00
{
2021-08-10 01:33:57 +00:00
if (isFinalized() || data.empty())
return;
const auto & to_type = least_common_type;
auto result_column = to_type->createColumn();
if (num_of_defaults_in_prefix)
result_column->insertManyDefaults(num_of_defaults_in_prefix);
for (auto & part : data)
{
auto from_type = getDataTypeByColumn(*part);
size_t part_size = part->size();
if (!from_type->equals(*to_type))
{
auto offsets = ColumnUInt64::create();
auto & offsets_data = offsets->getData();
2022-02-17 19:00:25 +00:00
/// We need to convert only non-default values and then recreate column
/// with default value of new type, because default values (which represents misses in data)
/// may be inconsistent between types (e.g "0" in UInt64 and empty string in String).
part->getIndicesOfNonDefaultRows(offsets_data, 0, part_size);
if (offsets->size() == part_size)
{
part = castColumn({part, from_type, ""}, to_type);
}
else
{
auto values = part->index(*offsets, offsets->size());
values = castColumn({values, from_type, ""}, to_type);
part = values->createWithOffsets(offsets_data, to_type->getDefault(), part_size, /*shift=*/ 0);
}
}
result_column->insertRangeFrom(*part, 0, part_size);
}
data = { std::move(result_column) };
num_of_defaults_in_prefix = 0;
}
2021-04-23 12:53:38 +00:00
void ColumnObject::Subcolumn::insertDefault()
{
if (data.empty())
++num_of_defaults_in_prefix;
else
data.back()->insertDefault();
}
2021-08-10 01:33:57 +00:00
void ColumnObject::Subcolumn::insertManyDefaults(size_t length)
{
if (data.empty())
num_of_defaults_in_prefix += length;
else
data.back()->insertManyDefaults(length);
}
2021-12-13 11:37:23 +00:00
void ColumnObject::Subcolumn::popBack(size_t n)
{
assert(n <= size());
2021-12-13 11:37:23 +00:00
size_t num_removed = 0;
for (auto it = data.rbegin(); it != data.rend(); ++it)
{
if (n == 0)
break;
auto & column = *it;
if (n < column->size())
{
column->popBack(n);
n = 0;
}
else
{
++num_removed;
n -= column->size();
}
}
data.resize(data.size() - num_removed);
num_of_defaults_in_prefix -= n;
}
Field ColumnObject::Subcolumn::getLastField() const
{
if (data.empty())
return Field();
const auto & last_part = data.back();
assert(!last_part->empty());
2021-12-13 11:37:23 +00:00
return (*last_part)[last_part->size() - 1];
}
2021-12-23 12:28:40 +00:00
ColumnObject::Subcolumn ColumnObject::Subcolumn::recreateWithDefaultValues(const FieldInfo & field_info) const
2021-12-13 11:37:23 +00:00
{
2022-02-17 19:00:25 +00:00
auto scalar_type = field_info.scalar_type;
if (is_nullable)
scalar_type = makeNullable(scalar_type);
2021-12-13 11:37:23 +00:00
Subcolumn new_subcolumn;
2022-02-17 19:00:25 +00:00
new_subcolumn.least_common_type = createArrayOfType(scalar_type, field_info.num_dimensions);
2021-12-17 15:08:34 +00:00
new_subcolumn.is_nullable = is_nullable;
2021-12-13 11:37:23 +00:00
new_subcolumn.num_of_defaults_in_prefix = num_of_defaults_in_prefix;
new_subcolumn.data.reserve(data.size());
for (const auto & part : data)
2021-12-23 12:28:40 +00:00
new_subcolumn.data.push_back(recreateColumnWithDefaultValues(
2022-01-21 14:55:26 +00:00
part, scalar_type, field_info.num_dimensions));
2021-12-13 11:37:23 +00:00
return new_subcolumn;
}
IColumn & ColumnObject::Subcolumn::getFinalizedColumn()
{
assert(isFinalized());
return *data[0];
}
const IColumn & ColumnObject::Subcolumn::getFinalizedColumn() const
{
assert(isFinalized());
return *data[0];
}
const ColumnPtr & ColumnObject::Subcolumn::getFinalizedColumnPtr() const
{
assert(isFinalized());
return data[0];
}
2021-04-23 12:53:38 +00:00
ColumnObject::ColumnObject(bool is_nullable_)
: is_nullable(is_nullable_)
2021-12-17 15:08:34 +00:00
, num_rows(0)
{
}
2021-12-13 11:37:23 +00:00
ColumnObject::ColumnObject(SubcolumnsTree && subcolumns_, bool is_nullable_)
2021-12-17 15:08:34 +00:00
: is_nullable(is_nullable_)
, subcolumns(std::move(subcolumns_))
2022-01-27 00:24:34 +00:00
, num_rows(subcolumns.empty() ? 0 : (*subcolumns.begin())->data.size())
2021-12-17 15:08:34 +00:00
{
2021-04-23 12:53:38 +00:00
checkConsistency();
}
void ColumnObject::checkConsistency() const
{
if (subcolumns.empty())
return;
2021-12-13 11:37:23 +00:00
for (const auto & leaf : subcolumns)
2021-04-23 12:53:38 +00:00
{
2022-01-27 00:24:34 +00:00
if (num_rows != leaf->data.size())
2021-04-23 12:53:38 +00:00
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of subcolumns are inconsistent in ColumnObject."
2021-12-17 15:08:34 +00:00
" Subcolumn '{}' has {} rows, but expected size is {}",
2022-01-27 00:24:34 +00:00
leaf->path.getPath(), leaf->data.size(), num_rows);
2021-04-23 12:53:38 +00:00
}
}
}
size_t ColumnObject::size() const
{
#ifndef NDEBUG
checkConsistency();
#endif
2021-12-17 15:08:34 +00:00
return num_rows;
}
2021-04-23 12:53:38 +00:00
MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const
{
2022-02-17 19:00:25 +00:00
/// cloneResized with new_size == 0 is used for cloneEmpty().
if (new_size != 0)
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"ColumnObject doesn't support resize to non-zero length");
2021-04-23 12:53:38 +00:00
return ColumnObject::create(is_nullable);
2021-04-23 12:53:38 +00:00
}
size_t ColumnObject::byteSize() const
{
size_t res = 0;
2021-12-13 11:37:23 +00:00
for (const auto & entry : subcolumns)
2022-01-27 00:24:34 +00:00
res += entry->data.byteSize();
2021-04-23 12:53:38 +00:00
return res;
}
size_t ColumnObject::allocatedBytes() const
{
size_t res = 0;
2021-12-13 11:37:23 +00:00
for (const auto & entry : subcolumns)
2022-01-27 00:24:34 +00:00
res += entry->data.allocatedBytes();
2021-04-23 12:53:38 +00:00
return res;
}
2021-08-20 21:11:22 +00:00
void ColumnObject::forEachSubcolumn(ColumnCallback callback)
{
2021-08-20 21:11:22 +00:00
if (!isFinalized())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot iterate over non-finalized ColumnObject");
2021-12-13 11:37:23 +00:00
for (auto & entry : subcolumns)
2022-01-27 00:24:34 +00:00
callback(entry->data.data.back());
2021-08-20 21:11:22 +00:00
}
void ColumnObject::insert(const Field & field)
{
2021-09-14 14:02:32 +00:00
const auto & object = field.get<const Object &>();
2021-08-20 21:11:22 +00:00
HashSet<StringRef, StringRefHash> inserted;
size_t old_size = size();
2021-12-01 02:58:24 +00:00
for (const auto & [key_str, value] : object)
2021-08-20 21:11:22 +00:00
{
PathInData key(key_str);
2021-12-01 02:58:24 +00:00
inserted.insert(key_str);
2021-08-20 21:11:22 +00:00
if (!hasSubcolumn(key))
addSubcolumn(key, old_size);
auto & subcolumn = getSubcolumn(key);
subcolumn.insert(value);
}
2021-12-13 11:37:23 +00:00
for (auto & entry : subcolumns)
if (!inserted.has(entry->path.getPath()))
2022-01-27 00:24:34 +00:00
entry->data.insertDefault();
2021-12-23 12:28:40 +00:00
++num_rows;
2021-08-20 21:11:22 +00:00
}
void ColumnObject::insertDefault()
{
2021-12-13 11:37:23 +00:00
for (auto & entry : subcolumns)
2022-01-27 00:24:34 +00:00
entry->data.insertDefault();
2021-12-23 12:28:40 +00:00
++num_rows;
2021-08-20 21:11:22 +00:00
}
Field ColumnObject::operator[](size_t n) const
{
if (!isFinalized())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get Field from non-finalized ColumnObject");
Object object;
2021-12-13 11:37:23 +00:00
for (const auto & entry : subcolumns)
2022-01-27 00:24:34 +00:00
object[entry->path.getPath()] = (*entry->data.data.back())[n];
2021-08-20 21:11:22 +00:00
return object;
}
void ColumnObject::get(size_t n, Field & res) const
{
if (!isFinalized())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get Field from non-finalized ColumnObject");
auto & object = res.get<Object &>();
2021-12-13 11:37:23 +00:00
for (const auto & entry : subcolumns)
2021-08-20 21:11:22 +00:00
{
2021-12-13 11:37:23 +00:00
auto it = object.try_emplace(entry->path.getPath()).first;
2022-01-27 00:24:34 +00:00
entry->data.data.back()->get(n, it->second);
2021-08-20 21:11:22 +00:00
}
}
2021-08-10 01:33:57 +00:00
void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
const auto & src_object = assert_cast<const ColumnObject &>(src);
2021-12-13 11:37:23 +00:00
for (auto & entry : subcolumns)
2021-08-10 01:33:57 +00:00
{
2021-12-13 11:37:23 +00:00
if (src_object.hasSubcolumn(entry->path))
2022-01-27 00:24:34 +00:00
entry->data.insertRangeFrom(src_object.getSubcolumn(entry->path), start, length);
2021-08-10 01:33:57 +00:00
else
2022-01-27 00:24:34 +00:00
entry->data.insertManyDefaults(length);
2021-08-10 01:33:57 +00:00
}
2021-12-23 12:28:40 +00:00
num_rows += length;
2021-08-10 01:33:57 +00:00
finalize();
}
2021-08-20 21:11:22 +00:00
ColumnPtr ColumnObject::replicate(const Offsets & offsets) const
{
if (!isFinalized())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot replicate non-finalized ColumnObject");
auto res_column = ColumnObject::create(is_nullable);
2021-12-13 11:37:23 +00:00
for (const auto & entry : subcolumns)
2021-12-01 02:58:24 +00:00
{
2022-01-27 00:24:34 +00:00
auto replicated_data = entry->data.data.back()->replicate(offsets)->assumeMutable();
2021-12-23 12:28:40 +00:00
res_column->addSubcolumn(entry->path, std::move(replicated_data));
2021-12-01 02:58:24 +00:00
}
2021-08-20 21:11:22 +00:00
return res_column;
}
void ColumnObject::popBack(size_t length)
{
2021-12-13 11:37:23 +00:00
for (auto & entry : subcolumns)
2022-01-27 00:24:34 +00:00
entry->data.popBack(length);
2021-12-23 12:28:40 +00:00
num_rows -= length;
2021-08-20 21:11:22 +00:00
}
const ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key) const
2021-04-23 12:53:38 +00:00
{
2021-12-13 11:37:23 +00:00
if (const auto * node = subcolumns.findLeaf(key))
2022-01-27 00:24:34 +00:00
return node->data;
2021-04-23 12:53:38 +00:00
2021-12-01 02:58:24 +00:00
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in ColumnObject", key.getPath());
2021-04-23 12:53:38 +00:00
}
ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key)
2021-04-23 12:53:38 +00:00
{
2021-12-13 11:37:23 +00:00
if (const auto * node = subcolumns.findLeaf(key))
2022-02-17 19:00:25 +00:00
return const_cast<SubcolumnsTree::Node *>(node)->data;
2021-04-23 12:53:38 +00:00
2021-12-01 02:58:24 +00:00
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in ColumnObject", key.getPath());
}
bool ColumnObject::hasSubcolumn(const PathInData & key) const
2021-04-23 12:53:38 +00:00
{
2021-12-13 11:37:23 +00:00
return subcolumns.findLeaf(key) != nullptr;
2021-04-23 12:53:38 +00:00
}
void ColumnObject::addSubcolumn(const PathInData & key, MutableColumnPtr && subcolumn)
2021-04-23 12:53:38 +00:00
{
2021-12-23 12:28:40 +00:00
size_t new_size = subcolumn->size();
bool inserted = subcolumns.add(key, Subcolumn(std::move(subcolumn), is_nullable));
2022-01-27 00:24:34 +00:00
2021-12-23 12:28:40 +00:00
if (!inserted)
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath());
2021-12-17 15:08:34 +00:00
if (num_rows == 0)
num_rows = new_size;
2021-12-23 12:28:40 +00:00
}
2021-12-17 15:08:34 +00:00
void ColumnObject::addSubcolumn(const PathInData & key, size_t new_size)
2021-12-23 12:28:40 +00:00
{
bool inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable));
if (!inserted)
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath());
if (num_rows == 0)
num_rows = new_size;
}
void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size)
2021-12-23 12:28:40 +00:00
{
if (!key.hasNested())
throw Exception(ErrorCodes::LOGICAL_ERROR,
2021-12-23 12:28:40 +00:00
"Cannot add Nested subcolumn, because path doesn't contain Nested");
2021-12-23 12:28:40 +00:00
bool inserted = false;
2022-02-17 19:00:25 +00:00
/// We find node that represents the same Nested type as @key.
2021-12-23 12:28:40 +00:00
const auto * nested_node = subcolumns.findBestMatch(key);
if (nested_node)
2021-12-01 02:58:24 +00:00
{
2022-02-17 19:00:25 +00:00
/// Find any leaf of Nested subcolumn.
2022-02-16 14:18:03 +00:00
const auto * leaf = subcolumns.findLeaf(nested_node, [&](const auto &) { return true; });
2021-12-23 12:28:40 +00:00
assert(leaf);
2021-12-13 11:37:23 +00:00
2022-02-17 19:00:25 +00:00
/// Recreate subcolumn with default values and the same sizes of arrays.
2022-01-27 00:24:34 +00:00
auto new_subcolumn = leaf->data.recreateWithDefaultValues(field_info);
2022-02-17 19:00:25 +00:00
/// It's possible that we have already inserted value from current row
/// to this subcolumn. So, adjust size to expected.
2021-12-23 12:28:40 +00:00
if (new_subcolumn.size() > new_size)
new_subcolumn.popBack(new_subcolumn.size() - new_size);
2021-12-13 11:37:23 +00:00
2022-02-17 19:00:25 +00:00
assert(new_subcolumn.size() == new_size);
2021-12-23 12:28:40 +00:00
inserted = subcolumns.add(key, new_subcolumn);
2021-12-01 02:58:24 +00:00
}
2021-12-23 12:28:40 +00:00
else
{
2022-02-17 19:00:25 +00:00
/// If node was not found just add subcolumn with empty arrays.
2021-12-13 11:37:23 +00:00
inserted = subcolumns.add(key, Subcolumn(new_size, is_nullable));
}
2021-12-13 11:37:23 +00:00
2021-12-23 12:28:40 +00:00
if (!inserted)
2021-12-13 11:37:23 +00:00
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath());
2021-12-17 15:08:34 +00:00
if (num_rows == 0)
2021-12-23 12:28:40 +00:00
num_rows = new_size;
2021-04-23 12:53:38 +00:00
}
PathsInData ColumnObject::getKeys() const
{
PathsInData keys;
2021-08-10 01:33:57 +00:00
keys.reserve(subcolumns.size());
2021-12-13 11:37:23 +00:00
for (const auto & entry : subcolumns)
keys.emplace_back(entry->path);
2021-08-10 01:33:57 +00:00
return keys;
}
bool ColumnObject::isFinalized() const
{
return std::all_of(subcolumns.begin(), subcolumns.end(),
2022-01-27 00:24:34 +00:00
[](const auto & entry) { return entry->data.isFinalized(); });
}
void ColumnObject::finalize()
{
size_t old_size = size();
2021-12-13 11:37:23 +00:00
SubcolumnsTree new_subcolumns;
for (auto && entry : subcolumns)
{
2022-01-27 00:24:34 +00:00
const auto & least_common_type = entry->data.getLeastCommonType();
2022-02-17 19:00:25 +00:00
/// Do not add subcolumns, which consists only from NULLs.
if (isNothing(getBaseTypeOfArray(least_common_type)))
continue;
2022-01-27 00:24:34 +00:00
entry->data.finalize();
new_subcolumns.add(entry->path, std::move(entry->data));
}
2022-02-17 19:00:25 +00:00
/// If all subcolumns were skipped add a dummy subcolumn,
/// because Tuple type must have at least one element.
if (new_subcolumns.empty())
2022-03-03 21:50:36 +00:00
new_subcolumns.add(PathInData{COLUMN_NAME_DUMMY}, Subcolumn{ColumnUInt8::create(old_size, 0), is_nullable});
std::swap(subcolumns, new_subcolumns);
2021-08-10 01:33:57 +00:00
checkObjectHasNoAmbiguosPaths(getKeys());
}
2021-04-23 12:53:38 +00:00
}