diff --git a/src/DataTypes/DataTypeCustom.h b/src/DataTypes/DataTypeCustom.h index cf1e943d8e9..675673ee1fd 100644 --- a/src/DataTypes/DataTypeCustom.h +++ b/src/DataTypes/DataTypeCustom.h @@ -22,6 +22,8 @@ public: virtual ~IDataTypeCustomName() = default; virtual String getName() const = 0; + + virtual bool identical(const IDataTypeCustomName & rhs) const = 0; }; using DataTypeCustomNamePtr = std::unique_ptr; @@ -51,6 +53,12 @@ private: public: explicit DataTypeCustomFixedName(String name_) : name(name_) {} String getName() const override { return name; } + bool identical(const IDataTypeCustomName & rhs_) const override + { + if (const auto * rhs = typeid_cast(&rhs_)) + return name == rhs->getName(); + return false; + } }; } diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index 4e50be0a0cc..42e94291094 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -167,4 +167,19 @@ void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory) factory.registerDataTypeCustom("SimpleAggregateFunction", create); } +bool DataTypeCustomSimpleAggregateFunction::identical(const IDataTypeCustomName & rhs_) const +{ + if (const auto * rhs = typeid_cast(&rhs_)) + { + if (parameters != rhs->parameters) + return false; + if (argument_types.size() != rhs->argument_types.size()) + return false; + for (size_t i = 0; i < argument_types.size(); ++i) + if (!argument_types[i]->identical(*rhs->argument_types[i])) + return false; + return function->getName() == rhs->function->getName(); + } + return false; +} } diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h index 926dfd9cc82..dd68756d82e 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h @@ -36,6 +36,7 @@ public: AggregateFunctionPtr getFunction() const { return function; } String getName() const override; + bool identical(const IDataTypeCustomName & rhs_) const override; static void checkSupportedFunctions(const AggregateFunctionPtr & function); }; diff --git a/src/DataTypes/DataTypeNested.cpp b/src/DataTypes/DataTypeNested.cpp index a4b7442393c..a916b2a5a2b 100644 --- a/src/DataTypes/DataTypeNested.cpp +++ b/src/DataTypes/DataTypeNested.cpp @@ -72,4 +72,19 @@ DataTypePtr createNested(const DataTypes & types, const Names & names) return DataTypeFactory::instance().getCustom(std::move(custom_desc)); } +bool DataTypeNestedCustomName::identical(const IDataTypeCustomName & rhs_) const +{ + if (const auto * rhs = typeid_cast(&rhs_)) + { + if (names != rhs->names) + return false; + if (elems.size() != rhs->elems.size()) + return false; + for (size_t i = 0; i < elems.size(); ++i) + if (!elems[i]->identical(*rhs->elems[i])) + return false; + return true; + } + return false; +} } diff --git a/src/DataTypes/DataTypeNested.h b/src/DataTypes/DataTypeNested.h index 1ad06477a6e..5b4aa290091 100644 --- a/src/DataTypes/DataTypeNested.h +++ b/src/DataTypes/DataTypeNested.h @@ -19,6 +19,7 @@ public: } String getName() const override; + bool identical(const IDataTypeCustomName & rhs_) const override; }; DataTypePtr createNested(const DataTypes & types, const Names & names); diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 2a7e0f246de..ad3c1a78aed 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -252,4 +252,17 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column) return column.type->getDefaultSerialization(); } +bool IDataType::identical(const IDataType & rhs) const +{ + const auto * rhs_custom_name = rhs.getCustomName(); + if (custom_name && rhs_custom_name) + { + return custom_name->identical(*rhs_custom_name); + } + else if (custom_name || rhs_custom_name) + return false; + else + return equals(rhs); +} + } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 7f5cd88fa2b..81524c5a697 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -177,6 +177,12 @@ public: /// Checks that two instances belong to the same type virtual bool equals(const IDataType & rhs) const = 0; + /** Checks that two types is exactly identical. Note that two types may equals but not identical. + * For example, `SimpleAggregateFunction(max, String)` and `SimpleAggregateFunction(anyLast, String)` + * This is used when creating replicated table. Column types in different replicas must be identical + */ + virtual bool identical(const IDataType & rhs) const; + /// Various properties on behaviour of data type. /** The data type is dependent on parameters and types with different parameters are different. diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index dafd341a9f8..5b21b8f120b 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -53,6 +53,18 @@ ColumnDescription::ColumnDescription(String name_, DataTypePtr type_) { } +bool ColumnDescription::identical(const ColumnDescription & other) const +{ + auto ast_to_str = [](const ASTPtr & ast) { return ast ? queryToString(ast) : String{}; }; + + return name == other.name + && type->identical(*other.type) + && default_desc == other.default_desc + && comment == other.comment + && ast_to_str(codec) == ast_to_str(other.codec) + && ast_to_str(ttl) == ast_to_str(other.ttl); +} + bool ColumnDescription::operator==(const ColumnDescription & other) const { auto ast_to_str = [](const ASTPtr & ast) { return ast ? queryToString(ast) : String{}; }; @@ -877,4 +889,13 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N } } +bool ColumnsDescription::identical(const ColumnsDescription & other) const +{ + if (columns.size() != other.columns.size()) + return false; + for (auto it1 = columns.begin(), it2 = other.columns.begin(); it1 != columns.end(); ++it1, ++it2) + if (!it1->identical(*it2)) + return false; + return true; +} } diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 06a2b476144..e982ed93b3b 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -89,6 +89,7 @@ struct ColumnDescription ColumnDescription(const ColumnDescription &) = default; ColumnDescription(String name_, DataTypePtr type_); + bool identical(const ColumnDescription & other) const; bool operator==(const ColumnDescription & other) const; bool operator!=(const ColumnDescription & other) const { return !(*this == other); } @@ -123,6 +124,7 @@ public: /// NOTE Must correspond with Nested::flatten function. void flattenNested(); /// TODO: remove, insert already flattened Nested columns. + bool identical(const ColumnsDescription & other) const; bool operator==(const ColumnsDescription & other) const { return columns == other.columns; } bool operator!=(const ColumnsDescription & other) const { return !(*this == other); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 837c0c6bbbf..2b91499b567 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1230,9 +1230,9 @@ bool StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr const auto & old_columns = metadata_snapshot->getColumns(); /// Replicated tables on different replicas must have exactly same column definitions - /// We cannot just compare column descriptions here because data types like SimpleAggregateFunction + /// We cannot compare column descriptions with `==` here because data types like SimpleAggregateFunction /// may have different aggregate function in 1st argument but still compatible if 2nd argument is same. - if (columns_from_zk.toString() == old_columns.toString()) + if (columns_from_zk.identical(old_columns)) return true; if (!strict_check && metadata_stat.version != 0) diff --git a/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql b/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql index 90da4725ace..84250059c58 100644 --- a/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql +++ b/tests/queries/0_stateless/02910_replicated_with_simple_aggregate_column.sql @@ -14,4 +14,4 @@ CREATE TABLE t_r2 ) ENGINE = ReplicatedAggregatingMergeTree('/tables/{database}/t', 'r2') ORDER BY id -SETTINGS index_granularity = 8192; -- { serverError INCOMPATIBLE_COLUMNS } \ No newline at end of file +SETTINGS index_granularity = 8192; -- { serverError INCOMPATIBLE_COLUMNS }