2021-06-12 15:10:25 +00:00
|
|
|
#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>
|
2021-05-06 05:33:06 +00:00
|
|
|
#include <DataTypes/ObjectUtils.h>
|
|
|
|
#include <DataTypes/getLeastSupertype.h>
|
|
|
|
#include <DataTypes/DataTypeNothing.h>
|
2021-06-12 15:10:25 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
2021-12-01 02:58:24 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2021-05-06 05:33:06 +00:00
|
|
|
#include <Interpreters/castColumn.h>
|
2021-06-12 15:10:25 +00:00
|
|
|
#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;
|
2021-05-06 00:40:17 +00:00
|
|
|
extern const int DUPLICATE_COLUMN;
|
2021-06-12 15:10:25 +00:00
|
|
|
extern const int NUMBER_OF_DIMENSIONS_MISMATHED;
|
2021-09-14 14:02:32 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2022-03-16 15:28:09 +00:00
|
|
|
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
2021-06-12 15:10:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2022-03-10 21:24:19 +00:00
|
|
|
/// Recreates column 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.
|
2021-06-12 15:10:25 +00:00
|
|
|
class FieldVisitorReplaceNull : public StaticVisitor<Field>
|
|
|
|
{
|
|
|
|
public:
|
2022-02-17 19:00:25 +00:00
|
|
|
explicit FieldVisitorReplaceNull(
|
2021-09-11 00:20:54 +00:00
|
|
|
const Field & replacement_, size_t num_dimensions_)
|
2021-06-12 15:10:25 +00:00
|
|
|
: replacement(replacement_)
|
2021-09-11 00:20:54 +00:00
|
|
|
, num_dimensions(num_dimensions_)
|
2021-06-12 15:10:25 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2022-02-17 19:00:25 +00:00
|
|
|
Field operator()(const Null &) const
|
2021-06-12 15:10:25 +00:00
|
|
|
{
|
2022-02-17 19:00:25 +00:00
|
|
|
return num_dimensions
|
|
|
|
? createEmptyArrayField(num_dimensions)
|
|
|
|
: replacement;
|
2021-06-12 15:10:25 +00:00
|
|
|
}
|
|
|
|
|
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; }
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
private:
|
2021-09-11 00:20:54 +00:00
|
|
|
const Field & replacement;
|
|
|
|
size_t num_dimensions;
|
2021-06-12 15:10:25 +00:00
|
|
|
};
|
|
|
|
|
2022-02-17 19:00:25 +00:00
|
|
|
/// Calculates number of dimensions in array field.
|
|
|
|
/// Returns 0 for scalar fields.
|
2021-06-12 15:10:25 +00:00
|
|
|
class FieldVisitorToNumberOfDimensions : public StaticVisitor<size_t>
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
size_t operator()(const Array & x) const
|
|
|
|
{
|
|
|
|
const size_t size = x.size();
|
2021-09-11 00:20:54 +00:00
|
|
|
std::optional<size_t> dimensions;
|
|
|
|
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
2021-06-12 15:10:25 +00:00
|
|
|
{
|
2021-09-11 00:20:54 +00:00
|
|
|
/// Do not count Nulls, because they will be replaced by default
|
|
|
|
/// values with proper number of dimensions.
|
|
|
|
if (x[i].isNull())
|
|
|
|
continue;
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
size_t current_dimensions = applyVisitor(*this, x[i]);
|
2021-09-11 00:20:54 +00:00
|
|
|
if (!dimensions)
|
|
|
|
dimensions = current_dimensions;
|
|
|
|
else if (current_dimensions != *dimensions)
|
2021-06-12 15:10:25 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED,
|
|
|
|
"Number of dimensions mismatched among array elements");
|
|
|
|
}
|
|
|
|
|
2021-09-11 00:20:54 +00:00
|
|
|
return 1 + dimensions.value_or(0);
|
2021-06-12 15:10:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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<>
|
2021-06-12 15:10:25 +00:00
|
|
|
{
|
|
|
|
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);
|
2022-01-21 17:11:18 +00:00
|
|
|
type_indexes.insert(TypeToTypeIndex<NearestFieldType<T>>);
|
2021-06-12 15:10:25 +00:00
|
|
|
}
|
|
|
|
|
2021-12-13 11:37:23 +00:00
|
|
|
DataTypePtr getScalarType() const { return getLeastSupertype(type_indexes, true); }
|
|
|
|
bool haveNulls() const { return have_nulls; }
|
2021-06-12 15:10:25 +00:00
|
|
|
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_)
|
2021-06-12 15:10:25 +00:00
|
|
|
: least_common_type(getDataTypeByColumn(*data_))
|
2021-12-01 02:58:24 +00:00
|
|
|
, is_nullable(is_nullable_)
|
2021-06-12 15:10:25 +00:00
|
|
|
{
|
|
|
|
data.push_back(std::move(data_));
|
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
ColumnObject::Subcolumn::Subcolumn(
|
|
|
|
size_t size_, bool is_nullable_)
|
2021-06-12 15:10:25 +00:00
|
|
|
: least_common_type(std::make_shared<DataTypeNothing>())
|
2021-07-23 23:15:44 +00:00
|
|
|
, is_nullable(is_nullable_)
|
2021-06-12 15:10:25 +00:00
|
|
|
, 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",
|
2021-06-12 15:10:25 +00:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
auto column_dim = getNumberOfDimensions(*least_common_type);
|
2021-12-17 15:08:34 +00:00
|
|
|
auto value_dim = info.num_dimensions;
|
2021-09-13 14:02:38 +00:00
|
|
|
|
|
|
|
if (isNothing(least_common_type))
|
|
|
|
column_dim = value_dim;
|
|
|
|
|
|
|
|
if (field.isNull())
|
|
|
|
value_dim = column_dim;
|
2021-06-12 15:10:25 +00:00
|
|
|
|
2021-09-13 14:02:38 +00:00
|
|
|
if (value_dim != column_dim)
|
2021-12-17 16:02:29 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED,
|
2021-12-13 11:37:23 +00:00
|
|
|
"Dimension of types mismatched between inserted value and column. "
|
2021-06-12 15:10:25 +00:00
|
|
|
"Dimension of value: {}. Dimension of column: {}",
|
|
|
|
value_dim, column_dim);
|
|
|
|
|
2021-12-13 11:37:23 +00:00
|
|
|
if (is_nullable)
|
2021-07-23 23:15:44 +00:00
|
|
|
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));
|
2021-06-12 15:10:25 +00:00
|
|
|
|
2022-02-17 19:00:25 +00:00
|
|
|
auto value_type = createArrayOfType(base_type, value_dim);
|
2021-06-12 15:10:25 +00:00
|
|
|
bool type_changed = false;
|
2021-07-23 23:15:44 +00:00
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
void ColumnObject::Subcolumn::finalize()
|
2021-04-23 12:53:38 +00:00
|
|
|
{
|
2021-08-10 01:33:57 +00:00
|
|
|
if (isFinalized() || data.empty())
|
2021-06-12 15:10:25 +00:00
|
|
|
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).
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
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-05-06 00:40:17 +00:00
|
|
|
}
|
2021-04-23 12:53:38 +00:00
|
|
|
|
2021-05-06 00:40:17 +00:00
|
|
|
void ColumnObject::Subcolumn::insertDefault()
|
|
|
|
{
|
2021-06-12 15:10:25 +00:00
|
|
|
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)
|
|
|
|
{
|
2022-02-15 23:14:09 +00:00
|
|
|
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();
|
2022-02-15 23:14:09 +00:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
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-05-06 00:40:17 +00:00
|
|
|
}
|
2021-04-23 12:53:38 +00:00
|
|
|
|
2021-07-23 23:15:44 +00:00
|
|
|
ColumnObject::ColumnObject(bool is_nullable_)
|
|
|
|
: is_nullable(is_nullable_)
|
2021-12-17 15:08:34 +00:00
|
|
|
, num_rows(0)
|
2021-07-23 23:15:44 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
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-05-06 00:40:17 +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
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-05-29 01:19:20 +00:00
|
|
|
size_t ColumnObject::size() const
|
|
|
|
{
|
|
|
|
#ifndef NDEBUG
|
|
|
|
checkConsistency();
|
|
|
|
#endif
|
2021-12-17 15:08:34 +00:00
|
|
|
return num_rows;
|
2021-05-29 01:19:20 +00:00
|
|
|
}
|
|
|
|
|
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().
|
2021-05-06 05:33:06 +00:00
|
|
|
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
|
|
|
|
2021-07-23 23:15:44 +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-05-27 16:54:27 +00:00
|
|
|
{
|
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
|
|
|
{
|
2022-02-01 11:14:27 +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-05-27 16:54:27 +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
|
|
|
}
|
|
|
|
|
2022-02-01 11:14:27 +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
|
|
|
}
|
|
|
|
|
2022-02-01 11:14:27 +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());
|
|
|
|
}
|
|
|
|
|
2022-02-01 11:14:27 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
2022-02-01 11:14:27 +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;
|
2022-03-14 17:29:18 +00:00
|
|
|
else if (new_size != num_rows)
|
|
|
|
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH,
|
|
|
|
"Size of subcolumn {} ({}) is inconsistent with column size ({})",
|
|
|
|
key.getPath(), new_size, num_rows);
|
2021-12-23 12:28:40 +00:00
|
|
|
}
|
2021-12-17 15:08:34 +00:00
|
|
|
|
2022-02-01 11:14:27 +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;
|
2022-03-14 17:29:18 +00:00
|
|
|
else if (new_size != num_rows)
|
|
|
|
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH,
|
2022-03-16 15:28:09 +00:00
|
|
|
"Required size of subcolumn {} ({}) is inconsistent with column size ({})",
|
2022-03-14 17:29:18 +00:00
|
|
|
key.getPath(), new_size, num_rows);
|
2021-12-23 12:28:40 +00:00
|
|
|
}
|
|
|
|
|
2022-02-01 11:14:27 +00:00
|
|
|
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())
|
2021-05-06 00:40:17 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
2021-12-23 12:28:40 +00:00
|
|
|
"Cannot add Nested subcolumn, because path doesn't contain Nested");
|
2021-05-06 00:40:17 +00:00
|
|
|
|
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);
|
2022-02-14 19:20:52 +00:00
|
|
|
|
|
|
|
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-14 19:20:52 +00:00
|
|
|
{
|
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));
|
2022-02-14 19:20:52 +00:00
|
|
|
}
|
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-05-06 00:40:17 +00:00
|
|
|
|
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
|
|
|
}
|
|
|
|
|
2022-02-01 11:14:27 +00:00
|
|
|
PathsInData ColumnObject::getKeys() const
|
2021-06-08 09:33:04 +00:00
|
|
|
{
|
2022-02-01 11:14:27 +00:00
|
|
|
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;
|
2021-06-08 09:33:04 +00:00
|
|
|
}
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
bool ColumnObject::isFinalized() const
|
2021-05-06 05:33:06 +00:00
|
|
|
{
|
2021-06-12 15:10:25 +00:00
|
|
|
return std::all_of(subcolumns.begin(), subcolumns.end(),
|
2022-01-27 00:24:34 +00:00
|
|
|
[](const auto & entry) { return entry->data.isFinalized(); });
|
2021-06-12 15:10:25 +00:00
|
|
|
}
|
2021-05-06 05:33:06 +00:00
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
void ColumnObject::finalize()
|
|
|
|
{
|
2021-05-27 16:54:27 +00:00
|
|
|
size_t old_size = size();
|
2021-12-13 11:37:23 +00:00
|
|
|
SubcolumnsTree new_subcolumns;
|
|
|
|
for (auto && entry : subcolumns)
|
2021-05-06 05:33:06 +00:00
|
|
|
{
|
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.
|
2021-06-12 15:10:25 +00:00
|
|
|
if (isNothing(getBaseTypeOfArray(least_common_type)))
|
2021-05-06 05:33:06 +00:00
|
|
|
continue;
|
|
|
|
|
2022-01-27 00:24:34 +00:00
|
|
|
entry->data.finalize();
|
|
|
|
new_subcolumns.add(entry->path, std::move(entry->data));
|
2021-05-06 05:33:06 +00:00
|
|
|
}
|
|
|
|
|
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.
|
2021-05-27 16:54:27 +00:00
|
|
|
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});
|
2021-05-27 16:54:27 +00:00
|
|
|
|
|
|
|
std::swap(subcolumns, new_subcolumns);
|
2021-08-10 01:33:57 +00:00
|
|
|
checkObjectHasNoAmbiguosPaths(getKeys());
|
2021-05-06 05:33:06 +00:00
|
|
|
}
|
|
|
|
|
2021-04-23 12:53:38 +00:00
|
|
|
}
|