2021-06-12 15:10:25 +00:00
|
|
|
#include <Core/Field.h>
|
2021-04-23 12:53:38 +00:00
|
|
|
#include <Columns/ColumnObject.h>
|
2021-05-06 05:33:06 +00:00
|
|
|
#include <Columns/ColumnSparse.h>
|
|
|
|
#include <Columns/ColumnVector.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-05-27 16:54:27 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2021-06-12 15:10:25 +00:00
|
|
|
#include <DataTypes/FieldToDataType.h>
|
|
|
|
#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
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
#include <Common/FieldVisitorToString.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;
|
2021-06-12 15:10:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2021-09-11 00:20:54 +00:00
|
|
|
Array createEmptyArrayField(size_t num_dimensions)
|
|
|
|
{
|
|
|
|
Array array;
|
|
|
|
Array * current_array = &array;
|
|
|
|
for (size_t i = 1; i < num_dimensions; ++i)
|
|
|
|
{
|
|
|
|
current_array->push_back(Array());
|
|
|
|
current_array = ¤t_array->back().get<Array &>();
|
|
|
|
}
|
|
|
|
|
|
|
|
return array;
|
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
ColumnPtr recreateWithDefaultValues(const ColumnPtr & column)
|
|
|
|
{
|
|
|
|
if (const auto * column_array = checkAndGetColumn<ColumnArray>(column.get()))
|
|
|
|
return ColumnArray::create(recreateWithDefaultValues(column_array->getDataPtr()), IColumn::mutate(column_array->getOffsetsPtr()));
|
|
|
|
else
|
|
|
|
return column->cloneEmpty()->cloneResized(column->size());
|
|
|
|
}
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
class FieldVisitorReplaceNull : public StaticVisitor<Field>
|
|
|
|
{
|
|
|
|
public:
|
2021-09-11 00:20:54 +00:00
|
|
|
[[maybe_unused]] explicit FieldVisitorReplaceNull(
|
|
|
|
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
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
Field operator()(const T & x) const
|
|
|
|
{
|
2021-09-11 00:20:54 +00:00
|
|
|
if constexpr (std::is_same_v<T, Null>)
|
2021-06-12 15:10:25 +00:00
|
|
|
{
|
2021-09-11 00:20:54 +00:00
|
|
|
return num_dimensions
|
|
|
|
? createEmptyArrayField(num_dimensions)
|
|
|
|
: replacement;
|
|
|
|
}
|
|
|
|
else if constexpr (std::is_same_v<T, Array>)
|
|
|
|
{
|
|
|
|
assert(num_dimensions > 0);
|
2021-06-12 15:10:25 +00:00
|
|
|
const size_t size = x.size();
|
2021-09-11 00:20:54 +00:00
|
|
|
Array res(size);
|
2021-06-12 15:10:25 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
2021-09-11 00:20:54 +00:00
|
|
|
res[i] = applyVisitor(FieldVisitorReplaceNull(replacement, num_dimensions - 1), x[i]);
|
2021-06-12 15:10:25 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return x;
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
2021-09-11 00:20:54 +00:00
|
|
|
const Field & replacement;
|
|
|
|
size_t num_dimensions;
|
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; }
|
|
|
|
};
|
|
|
|
|
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 &)
|
|
|
|
{
|
|
|
|
auto field_type = Field::TypeToEnum<NearestFieldType<T>>::value;
|
|
|
|
field_types.insert(field_type);
|
|
|
|
type_indexes.insert(TypeId<NearestFieldType<T>>);
|
|
|
|
}
|
|
|
|
|
|
|
|
DataTypePtr getScalarType() const
|
|
|
|
{
|
|
|
|
|
|
|
|
auto res = getLeastSupertype(type_indexes, true);
|
|
|
|
if (have_nulls)
|
|
|
|
return makeNullable(res);
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
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-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-06-12 15:10:25 +00:00
|
|
|
auto column_dim = getNumberOfDimensions(*least_common_type);
|
2021-09-13 14:02:38 +00:00
|
|
|
auto value_dim = applyVisitor(FieldVisitorToNumberOfDimensions(), field);
|
|
|
|
|
|
|
|
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-06-12 15:10:25 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATHED,
|
2021-09-14 14:02:32 +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);
|
|
|
|
|
|
|
|
FieldVisitorToScalarType to_scalar_type_visitor;
|
|
|
|
applyVisitor(to_scalar_type_visitor, field);
|
|
|
|
|
|
|
|
auto base_type = to_scalar_type_visitor.getScalarType();
|
|
|
|
if (isNothing(base_type))
|
|
|
|
{
|
|
|
|
insertDefault();
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2021-07-23 23:15:44 +00:00
|
|
|
if (is_nullable && !base_type->isNullable())
|
|
|
|
base_type = makeNullable(base_type);
|
|
|
|
|
2021-09-14 14:02:32 +00:00
|
|
|
DataTypePtr value_type;
|
2021-07-23 23:15:44 +00:00
|
|
|
if (!is_nullable && base_type->isNullable())
|
2021-06-12 15:10:25 +00:00
|
|
|
{
|
|
|
|
base_type = removeNullable(base_type);
|
|
|
|
if (isNothing(base_type))
|
|
|
|
{
|
|
|
|
insertDefault();
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2021-09-11 00:20:54 +00:00
|
|
|
value_type = createArrayOfType(base_type, value_dim);
|
2021-12-01 02:58:24 +00:00
|
|
|
auto default_value = base_type->getDefault();
|
2021-09-11 00:20:54 +00:00
|
|
|
field = applyVisitor(FieldVisitorReplaceNull(default_value, value_dim), std::move(field));
|
2021-06-12 15:10:25 +00:00
|
|
|
}
|
2021-09-14 14:02:32 +00:00
|
|
|
else
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (type_changed || to_scalar_type_visitor.needConvertField())
|
|
|
|
{
|
|
|
|
auto converted_field = convertFieldToTypeOrThrow(std::move(field), *value_type);
|
|
|
|
data.back()->insert(std::move(converted_field));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
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();
|
|
|
|
|
|
|
|
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-12-01 02:58:24 +00:00
|
|
|
Field ColumnObject::Subcolumn::getLastField() const
|
|
|
|
{
|
|
|
|
if (data.empty())
|
|
|
|
return Field();
|
|
|
|
|
|
|
|
const auto & last_part = data.back();
|
|
|
|
assert(!last_part.empty());
|
|
|
|
return (*last_part)[last_part->size() - 1];
|
|
|
|
}
|
|
|
|
|
|
|
|
|
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-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_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnObject::ColumnObject(SubcolumnsMap && subcolumns_, bool is_nullable_)
|
2021-05-06 00:40:17 +00:00
|
|
|
: subcolumns(std::move(subcolumns_))
|
2021-07-23 23:15:44 +00:00
|
|
|
, is_nullable(is_nullable_)
|
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-05-06 00:40:17 +00:00
|
|
|
size_t first_size = subcolumns.begin()->second.size();
|
2021-12-01 02:58:24 +00:00
|
|
|
for (const auto & [key, column] : subcolumns)
|
2021-04-23 12:53:38 +00:00
|
|
|
{
|
2021-06-12 15:10:25 +00:00
|
|
|
if (first_size != column.size())
|
2021-04-23 12:53:38 +00:00
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of subcolumns are inconsistent in ColumnObject."
|
|
|
|
" Subcolumn '{}' has {} rows, subcolumn '{}' has {} rows",
|
2021-12-01 02:58:24 +00:00
|
|
|
subcolumns.begin()->first.getPath(), first_size, key.getPath(), column.size());
|
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
|
|
|
|
return subcolumns.empty() ? 0 : subcolumns.begin()->second.size();
|
|
|
|
}
|
|
|
|
|
2021-04-23 12:53:38 +00:00
|
|
|
MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const
|
|
|
|
{
|
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;
|
|
|
|
for (const auto & [_, column] : subcolumns)
|
2021-06-12 15:10:25 +00:00
|
|
|
res += column.byteSize();
|
2021-04-23 12:53:38 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t ColumnObject::allocatedBytes() const
|
|
|
|
{
|
|
|
|
size_t res = 0;
|
|
|
|
for (const auto & [_, column] : subcolumns)
|
2021-06-12 15:10:25 +00:00
|
|
|
res += column.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");
|
|
|
|
|
|
|
|
for (auto & [_, column] : subcolumns)
|
|
|
|
callback(column.data.back());
|
|
|
|
}
|
|
|
|
|
|
|
|
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
|
|
|
{
|
2021-12-01 02:58:24 +00:00
|
|
|
Path key(key_str);
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
|
|
|
for (auto & [key, subcolumn] : subcolumns)
|
2021-12-01 02:58:24 +00:00
|
|
|
if (!inserted.has(key.getPath()))
|
2021-08-20 21:11:22 +00:00
|
|
|
subcolumn.insertDefault();
|
|
|
|
}
|
|
|
|
|
|
|
|
void ColumnObject::insertDefault()
|
|
|
|
{
|
|
|
|
for (auto & [_, subcolumn] : subcolumns)
|
|
|
|
subcolumn.insertDefault();
|
|
|
|
}
|
|
|
|
|
|
|
|
Field ColumnObject::operator[](size_t n) const
|
|
|
|
{
|
|
|
|
if (!isFinalized())
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get Field from non-finalized ColumnObject");
|
|
|
|
|
|
|
|
Object object;
|
|
|
|
for (const auto & [key, subcolumn] : subcolumns)
|
2021-12-01 02:58:24 +00:00
|
|
|
object[key.getPath()] = (*subcolumn.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 &>();
|
|
|
|
for (const auto & [key, subcolumn] : subcolumns)
|
|
|
|
{
|
2021-12-01 02:58:24 +00:00
|
|
|
auto it = object.try_emplace(key.getPath()).first;
|
2021-08-20 21:11:22 +00:00
|
|
|
subcolumn.data.back()->get(n, it->second);
|
|
|
|
}
|
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);
|
|
|
|
|
|
|
|
for (auto & [name, subcolumn] : subcolumns)
|
|
|
|
{
|
|
|
|
if (src_object.hasSubcolumn(name))
|
|
|
|
subcolumn.insertRangeFrom(src_object.getSubcolumn(name), start, length);
|
|
|
|
else
|
|
|
|
subcolumn.insertManyDefaults(length);
|
|
|
|
}
|
|
|
|
|
|
|
|
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-01 02:58:24 +00:00
|
|
|
for (const auto & [key, subcolumn] : subcolumns)
|
|
|
|
{
|
|
|
|
auto replicated_data = subcolumn.data.back()->replicate(offsets)->assumeMutable();
|
|
|
|
res_column->addSubcolumn(key, std::move(replicated_data), is_nullable);
|
|
|
|
}
|
2021-08-20 21:11:22 +00:00
|
|
|
|
|
|
|
return res_column;
|
|
|
|
}
|
|
|
|
|
|
|
|
void ColumnObject::popBack(size_t length)
|
|
|
|
{
|
|
|
|
if (!isFinalized())
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot popBack from non-finalized ColumnObject");
|
|
|
|
|
|
|
|
for (auto & [_, subcolumn] : subcolumns)
|
|
|
|
subcolumn.data.back()->popBack(length);
|
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
const ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const Path & key) const
|
2021-04-23 12:53:38 +00:00
|
|
|
{
|
|
|
|
auto it = subcolumns.find(key);
|
|
|
|
if (it != subcolumns.end())
|
2021-05-06 00:40:17 +00:00
|
|
|
return it->second;
|
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
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const Path & key)
|
2021-04-23 12:53:38 +00:00
|
|
|
{
|
|
|
|
auto it = subcolumns.find(key);
|
|
|
|
if (it != subcolumns.end())
|
2021-05-06 00:40:17 +00:00
|
|
|
return it->second;
|
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());
|
|
|
|
}
|
|
|
|
|
|
|
|
std::optional<Path> ColumnObject::findSubcolumnForNested(const Path & key, size_t expected_size) const
|
|
|
|
{
|
|
|
|
for (const auto & [other_key, other_subcolumn] : subcolumns)
|
|
|
|
{
|
|
|
|
if (key == other_key || expected_size != other_subcolumn.size())
|
|
|
|
continue;
|
|
|
|
|
|
|
|
auto split_lhs = Nested::splitName(key.getPath(), true);
|
|
|
|
auto split_rhs = Nested::splitName(other_key.getPath(), true);
|
|
|
|
|
|
|
|
if (!split_lhs.first.empty() && split_lhs.first == split_rhs.first)
|
|
|
|
return other_key;
|
|
|
|
}
|
|
|
|
|
|
|
|
return {};
|
2021-04-23 12:53:38 +00:00
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
bool ColumnObject::hasSubcolumn(const Path & key) const
|
2021-04-23 12:53:38 +00:00
|
|
|
{
|
|
|
|
return subcolumns.count(key) != 0;
|
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
void ColumnObject::addSubcolumn(const Path & key, size_t new_size, bool check_size)
|
2021-04-23 12:53:38 +00:00
|
|
|
{
|
2021-05-06 00:40:17 +00:00
|
|
|
if (subcolumns.count(key))
|
2021-12-01 02:58:24 +00:00
|
|
|
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath());
|
2021-05-06 00:40:17 +00:00
|
|
|
|
|
|
|
if (check_size && new_size != size())
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
|
|
|
"Cannot add subcolumn '{}' with {} rows to ColumnObject with {} rows",
|
2021-12-01 02:58:24 +00:00
|
|
|
key.getPath(), new_size, size());
|
2021-05-06 00:40:17 +00:00
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
if (key.hasNested())
|
|
|
|
{
|
|
|
|
auto nested_key = findSubcolumnForNested(key, new_size);
|
|
|
|
if (nested_key)
|
|
|
|
{
|
|
|
|
auto & nested_subcolumn = subcolumns[*nested_key];
|
|
|
|
nested_subcolumn.finalize();
|
|
|
|
auto default_column = recreateWithDefaultValues(nested_subcolumn.getFinalizedColumnPtr());
|
|
|
|
subcolumns[key] = Subcolumn(default_column->assumeMutable(), is_nullable);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
subcolumns[key] = Subcolumn(new_size, is_nullable);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
subcolumns[key] = Subcolumn(new_size, is_nullable);
|
|
|
|
}
|
2021-05-06 00:40:17 +00:00
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
void ColumnObject::addSubcolumn(const Path & key, MutableColumnPtr && subcolumn, bool check_size)
|
2021-05-06 00:40:17 +00:00
|
|
|
{
|
|
|
|
if (subcolumns.count(key))
|
2021-12-01 02:58:24 +00:00
|
|
|
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath());
|
2021-05-06 00:40:17 +00:00
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
if (check_size && subcolumn->size() != size())
|
2021-04-23 12:53:38 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
|
|
|
"Cannot add subcolumn '{}' with {} rows to ColumnObject with {} rows",
|
2021-12-01 02:58:24 +00:00
|
|
|
key.getPath(), subcolumn->size(), size());
|
2021-04-23 12:53:38 +00:00
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
subcolumns[key] = Subcolumn(std::move(subcolumn), is_nullable);
|
2021-04-23 12:53:38 +00:00
|
|
|
}
|
|
|
|
|
2021-12-01 02:58:24 +00:00
|
|
|
Paths ColumnObject::getKeys() const
|
2021-06-08 09:33:04 +00:00
|
|
|
{
|
2021-12-01 02:58:24 +00:00
|
|
|
Paths keys;
|
2021-08-10 01:33:57 +00:00
|
|
|
keys.reserve(subcolumns.size());
|
|
|
|
for (const auto & [key, _] : subcolumns)
|
|
|
|
keys.emplace_back(key);
|
|
|
|
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(),
|
|
|
|
[](const auto & elem) { return elem.second.isFinalized(); });
|
|
|
|
}
|
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();
|
|
|
|
SubcolumnsMap new_subcolumns;
|
|
|
|
for (auto && [name, subcolumn] : subcolumns)
|
2021-05-06 05:33:06 +00:00
|
|
|
{
|
2021-06-12 15:10:25 +00:00
|
|
|
const auto & least_common_type = subcolumn.getLeastCommonType();
|
|
|
|
if (isNothing(getBaseTypeOfArray(least_common_type)))
|
2021-05-06 05:33:06 +00:00
|
|
|
continue;
|
|
|
|
|
2021-06-12 15:10:25 +00:00
|
|
|
subcolumn.finalize();
|
2021-05-27 16:54:27 +00:00
|
|
|
new_subcolumns[name] = std::move(subcolumn);
|
2021-05-06 05:33:06 +00:00
|
|
|
}
|
|
|
|
|
2021-05-27 16:54:27 +00:00
|
|
|
if (new_subcolumns.empty())
|
2021-12-01 02:58:24 +00:00
|
|
|
new_subcolumns[Path(COLUMN_NAME_DUMMY)] = Subcolumn{ColumnUInt8::create(old_size), 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
|
|
|
}
|