Avoid some logical errors in experimantal Object type

This commit is contained in:
avogar 2024-03-11 16:42:45 +00:00
parent b2f795601b
commit 74281b0a4e
4 changed files with 38 additions and 18 deletions

View File

@ -20,12 +20,12 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
extern const int DUPLICATE_COLUMN; extern const int DUPLICATE_COLUMN;
extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; extern const int NUMBER_OF_DIMENSIONS_MISMATCHED;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int ARGUMENT_OUT_OF_BOUND; extern const int ARGUMENT_OUT_OF_BOUND;
extern const int EXPERIMENTAL_FEATURE_ERROR;
} }
namespace namespace
@ -247,7 +247,7 @@ void ColumnObject::Subcolumn::checkTypes() const
prefix_types.push_back(current_type); prefix_types.push_back(current_type);
auto prefix_common_type = getLeastSupertype(prefix_types); auto prefix_common_type = getLeastSupertype(prefix_types);
if (!prefix_common_type->equals(*current_type)) if (!prefix_common_type->equals(*current_type))
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
"Data type {} of column at position {} cannot represent all columns from i-th prefix", "Data type {} of column at position {} cannot represent all columns from i-th prefix",
current_type->getName(), i); current_type->getName(), i);
} }
@ -635,7 +635,7 @@ void ColumnObject::checkConsistency() const
{ {
if (num_rows != leaf->data.size()) if (num_rows != leaf->data.size())
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of subcolumns are inconsistent in ColumnObject." throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Sizes of subcolumns are inconsistent in ColumnObject."
" Subcolumn '{}' has {} rows, but expected size is {}", " Subcolumn '{}' has {} rows, but expected size is {}",
leaf->path.getPath(), leaf->data.size(), num_rows); leaf->path.getPath(), leaf->data.size(), num_rows);
} }
@ -919,7 +919,7 @@ void ColumnObject::addSubcolumn(const PathInData & key, size_t new_size)
void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size) void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size)
{ {
if (!key.hasNested()) if (!key.hasNested())
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
"Cannot add Nested subcolumn, because path doesn't contain Nested"); "Cannot add Nested subcolumn, because path doesn't contain Nested");
bool inserted = false; bool inserted = false;

View File

@ -598,6 +598,7 @@
M(714, UNEXPECTED_CLUSTER) \ M(714, UNEXPECTED_CLUSTER) \
M(715, CANNOT_DETECT_FORMAT) \ M(715, CANNOT_DETECT_FORMAT) \
M(716, CANNOT_FORGET_PARTITION) \ M(716, CANNOT_FORGET_PARTITION) \
M(717, EXPERIMENTAL_FEATURE_ERROR) \
\ \
M(999, KEEPER_EXCEPTION) \ M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \ M(1000, POCO_EXCEPTION) \

View File

@ -28,9 +28,9 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
extern const int LOGICAL_ERROR;
extern const int INCOMPATIBLE_COLUMNS; extern const int INCOMPATIBLE_COLUMNS;
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int EXPERIMENTAL_FEATURE_ERROR;
} }
size_t getNumberOfDimensions(const IDataType & type) size_t getNumberOfDimensions(const IDataType & type)
@ -92,7 +92,7 @@ ColumnPtr createArrayOfColumn(ColumnPtr column, size_t num_dimensions)
Array createEmptyArrayField(size_t num_dimensions) Array createEmptyArrayField(size_t num_dimensions)
{ {
if (num_dimensions == 0) if (num_dimensions == 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create array field with 0 dimensions"); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Cannot create array field with 0 dimensions");
Array array; Array array;
Array * current_array = &array; Array * current_array = &array;
@ -231,7 +231,7 @@ static std::pair<ColumnPtr, DataTypePtr> recursivlyConvertDynamicColumnToTuple(
}; };
} }
throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type->getName()); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Type {} unexpectedly has dynamic columns", type->getName());
} }
void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & storage_snapshot) void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & storage_snapshot)
@ -247,7 +247,7 @@ void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & sto
GetColumnsOptions options(GetColumnsOptions::AllPhysical); GetColumnsOptions options(GetColumnsOptions::AllPhysical);
auto storage_column = storage_snapshot->tryGetColumn(options, column.name); auto storage_column = storage_snapshot->tryGetColumn(options, column.name);
if (!storage_column) if (!storage_column)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column '{}' not found in storage", column.name); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Column '{}' not found in storage", column.name);
auto storage_column_concrete = storage_snapshot->getColumn(options.withExtendedObjects(), column.name); auto storage_column_concrete = storage_snapshot->getColumn(options.withExtendedObjects(), column.name);
@ -315,7 +315,7 @@ static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool che
{ {
const auto * type_tuple = typeid_cast<const DataTypeTuple *>(type.get()); const auto * type_tuple = typeid_cast<const DataTypeTuple *>(type.get());
if (!type_tuple) if (!type_tuple)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
"Least common type for object can be deduced only from tuples, but {} given", type->getName()); "Least common type for object can be deduced only from tuples, but {} given", type->getName());
auto [tuple_paths, tuple_types] = flattenTuple(type); auto [tuple_paths, tuple_types] = flattenTuple(type);
@ -427,7 +427,7 @@ static DataTypePtr getLeastCommonTypeForDynamicColumnsImpl(
if (const auto * type_tuple = typeid_cast<const DataTypeTuple *>(type_in_storage.get())) if (const auto * type_tuple = typeid_cast<const DataTypeTuple *>(type_in_storage.get()))
return getLeastCommonTypeForTuple(*type_tuple, concrete_types, check_ambiguos_paths); return getLeastCommonTypeForTuple(*type_tuple, concrete_types, check_ambiguos_paths);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName()); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName());
} }
DataTypePtr getLeastCommonTypeForDynamicColumns( DataTypePtr getLeastCommonTypeForDynamicColumns(
@ -481,7 +481,7 @@ DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage
return recreateTupleWithElements(*type_tuple, new_elements); return recreateTupleWithElements(*type_tuple, new_elements);
} }
throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName()); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName());
} }
bool hasDynamicSubcolumns(const ColumnsDescription & columns) bool hasDynamicSubcolumns(const ColumnsDescription & columns)
@ -613,7 +613,7 @@ DataTypePtr reduceNumberOfDimensions(DataTypePtr type, size_t dimensions_to_redu
{ {
const auto * type_array = typeid_cast<const DataTypeArray *>(type.get()); const auto * type_array = typeid_cast<const DataTypeArray *>(type.get());
if (!type_array) if (!type_array)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not enough dimensions to reduce"); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Not enough dimensions to reduce");
type = type_array->getNestedType(); type = type_array->getNestedType();
} }
@ -627,7 +627,7 @@ ColumnPtr reduceNumberOfDimensions(ColumnPtr column, size_t dimensions_to_reduce
{ {
const auto * column_array = typeid_cast<const ColumnArray *>(column.get()); const auto * column_array = typeid_cast<const ColumnArray *>(column.get());
if (!column_array) if (!column_array)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not enough dimensions to reduce"); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Not enough dimensions to reduce");
column = column_array->getDataPtr(); column = column_array->getDataPtr();
} }
@ -705,6 +705,7 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
size_t num_elements = tuple_columns.size(); size_t num_elements = tuple_columns.size();
Columns tuple_elements_columns(num_elements); Columns tuple_elements_columns(num_elements);
DataTypes tuple_elements_types(num_elements); DataTypes tuple_elements_types(num_elements);
size_t last_offset = assert_cast<const ColumnArray::ColumnOffsets &>(*offsets_columns.back()).getData().back();
/// Reduce extra array dimensions to get columns and types of Nested elements. /// Reduce extra array dimensions to get columns and types of Nested elements.
for (size_t i = 0; i < num_elements; ++i) for (size_t i = 0; i < num_elements; ++i)
@ -712,6 +713,14 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
assert(tuple_columns[i].array_dimensions == tuple_columns[0].array_dimensions); assert(tuple_columns[i].array_dimensions == tuple_columns[0].array_dimensions);
tuple_elements_columns[i] = reduceNumberOfDimensions(tuple_columns[i].column, tuple_columns[i].array_dimensions); tuple_elements_columns[i] = reduceNumberOfDimensions(tuple_columns[i].column, tuple_columns[i].array_dimensions);
tuple_elements_types[i] = reduceNumberOfDimensions(tuple_columns[i].type, tuple_columns[i].array_dimensions); tuple_elements_types[i] = reduceNumberOfDimensions(tuple_columns[i].type, tuple_columns[i].array_dimensions);
if (tuple_elements_columns[i]->size() != last_offset)
throw Exception(
ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
"Cannot create a type for subcolumn {} in Object data type: offsets_column has data inconsistent with nested_column. "
"Data size: {}, last offset: {}",
node.path.getPath(),
tuple_elements_columns[i]->size(),
last_offset);
} }
auto result_column = ColumnArray::create(ColumnTuple::create(tuple_elements_columns), offsets_columns.back()); auto result_column = ColumnArray::create(ColumnTuple::create(tuple_elements_columns), offsets_columns.back());
@ -720,6 +729,16 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
/// Recreate result Array type and Array column. /// Recreate result Array type and Array column.
for (auto it = offsets_columns.rbegin() + 1; it != offsets_columns.rend(); ++it) for (auto it = offsets_columns.rbegin() + 1; it != offsets_columns.rend(); ++it)
{ {
last_offset = assert_cast<const ColumnArray::ColumnOffsets &>((**it)).getData().back();
if (result_column->size() != last_offset)
throw Exception(
ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
"Cannot create a type for subcolumn {} in Object data type: offsets_column has data inconsistent with nested_column. "
"Data size: {}, last offset: {}",
node.path.getPath(),
result_column->size(),
last_offset);
result_column = ColumnArray::create(result_column, *it); result_column = ColumnArray::create(result_column, *it);
result_type = std::make_shared<DataTypeArray>(result_type); result_type = std::make_shared<DataTypeArray>(result_type);
} }
@ -822,7 +841,7 @@ std::pair<ColumnPtr, DataTypePtr> unflattenTuple(
assert(paths.size() == tuple_columns.size()); assert(paths.size() == tuple_columns.size());
if (paths.empty()) if (paths.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot unflatten empty Tuple"); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Cannot unflatten empty Tuple");
/// We add all paths to the subcolumn tree and then create a type from it. /// We add all paths to the subcolumn tree and then create a type from it.
/// The tree stores column, type and number of array dimensions /// The tree stores column, type and number of array dimensions
@ -841,7 +860,7 @@ std::pair<ColumnPtr, DataTypePtr> unflattenTuple(
tree.add(paths[i], [&](Node::Kind kind, bool exists) -> std::shared_ptr<Node> tree.add(paths[i], [&](Node::Kind kind, bool exists) -> std::shared_ptr<Node>
{ {
if (pos >= num_parts) if (pos >= num_parts)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
"Not enough name parts for path {}. Expected at least {}, got {}", "Not enough name parts for path {}. Expected at least {}, got {}",
paths[i].getPath(), pos + 1, num_parts); paths[i].getPath(), pos + 1, num_parts);

View File

@ -29,7 +29,7 @@ namespace ErrorCodes
extern const int INCORRECT_DATA; extern const int INCORRECT_DATA;
extern const int CANNOT_READ_ALL_DATA; extern const int CANNOT_READ_ALL_DATA;
extern const int ARGUMENT_OUT_OF_BOUND; extern const int ARGUMENT_OUT_OF_BOUND;
extern const int LOGICAL_ERROR; extern const int EXPERIMENTAL_FEATURE_ERROR;
} }
template <typename Parser> template <typename Parser>
@ -177,7 +177,7 @@ void SerializationObject<Parser>::serializeBinaryBulkStatePrefix(
auto * stream = settings.getter(settings.path); auto * stream = settings.getter(settings.path);
if (!stream) if (!stream)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for kind of binary serialization"); throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Missing stream for kind of binary serialization");
auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object);
@ -288,7 +288,7 @@ void SerializationObject<Parser>::serializeBinaryBulkWithMultipleStreams(
if (!state_object->nested_type->equals(*tuple_type)) if (!state_object->nested_type->equals(*tuple_type))
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
"Types of internal column of Object mismatched. Expected: {}, Got: {}", "Types of internal column of Object mismatched. Expected: {}, Got: {}",
state_object->nested_type->getName(), tuple_type->getName()); state_object->nested_type->getName(), tuple_type->getName());
} }