ClickHouse/src/Columns/ColumnObject.cpp

726 lines
20 KiB
C++
Raw Normal View History

#include <Core/Field.h>
2021-04-23 12:53:38 +00:00
#include <Columns/ColumnObject.h>
#include <Columns/ColumnSparse.h>
#include <Columns/ColumnVector.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/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.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
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;
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
{
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 = &current_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());
}
class FieldVisitorReplaceNull : public StaticVisitor<Field>
{
public:
[[maybe_unused]] explicit FieldVisitorReplaceNull(
const Field & replacement_, size_t num_dimensions_)
: replacement(replacement_)
, num_dimensions(num_dimensions_)
{
}
template <typename T>
Field operator()(const T & x) const
{
if constexpr (std::is_same_v<T, Null>)
{
return num_dimensions
? createEmptyArrayField(num_dimensions)
: replacement;
}
else if constexpr (std::is_same_v<T, Array>)
{
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;
}
else
return x;
}
private:
const Field & replacement;
size_t num_dimensions;
};
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; }
};
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 &)
{
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_)
: 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
{
auto column_dim = getNumberOfDimensions(*least_common_type);
auto value_dim = applyVisitor(FieldVisitorToNumberOfDimensions(), field);
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-09-14 14:02:32 +00:00
"Dimension of types mismatched between inserted value and column."
"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;
}
if (is_nullable && !base_type->isNullable())
base_type = makeNullable(base_type);
2021-09-14 14:02:32 +00:00
DataTypePtr value_type;
if (!is_nullable && base_type->isNullable())
{
base_type = removeNullable(base_type);
if (isNothing(base_type))
{
insertDefault();
return;
}
value_type = createArrayOfType(base_type, value_dim);
2021-12-01 02:58:24 +00:00
auto default_value = base_type->getDefault();
field = applyVisitor(FieldVisitorReplaceNull(default_value, value_dim), std::move(field));
}
2021-09-14 14:02:32 +00:00
else
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;
}
}
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);
}
}
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();
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
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];
}
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);
}
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_)
{
}
ColumnObject::ColumnObject(SubcolumnsMap && subcolumns_, bool is_nullable_)
: subcolumns(std::move(subcolumns_))
, is_nullable(is_nullable_)
{
2021-04-23 12:53:38 +00:00
checkConsistency();
}
void ColumnObject::checkConsistency() const
{
if (subcolumns.empty())
return;
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
{
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
}
}
}
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
{
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;
for (const auto & [_, column] : subcolumns)
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)
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-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-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())
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())
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
{
if (subcolumns.count(key))
2021-12-01 02:58:24 +00:00
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath());
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-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-12-01 02:58:24 +00:00
void ColumnObject::addSubcolumn(const Path & key, MutableColumnPtr && subcolumn, bool check_size)
{
if (subcolumns.count(key))
2021-12-01 02:58:24 +00:00
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Subcolumn '{}' already exists", key.getPath());
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-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;
}
bool ColumnObject::isFinalized() const
{
return std::all_of(subcolumns.begin(), subcolumns.end(),
[](const auto & elem) { return elem.second.isFinalized(); });
}
void ColumnObject::finalize()
{
size_t old_size = size();
SubcolumnsMap new_subcolumns;
for (auto && [name, subcolumn] : subcolumns)
{
const auto & least_common_type = subcolumn.getLeastCommonType();
if (isNothing(getBaseTypeOfArray(least_common_type)))
continue;
subcolumn.finalize();
new_subcolumns[name] = std::move(subcolumn);
}
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};
std::swap(subcolumns, new_subcolumns);
2021-08-10 01:33:57 +00:00
checkObjectHasNoAmbiguosPaths(getKeys());
}
2021-04-23 12:53:38 +00:00
}