add new method to check if two types are exactly identical

Signed-off-by: Duc Canh Le <duccanh.le@ahrefs.com>
This commit is contained in:
Duc Canh Le 2023-11-22 15:02:34 +00:00
parent 3152cd25dc
commit 44a3dc1cd8
11 changed files with 85 additions and 3 deletions

View File

@ -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<const IDataTypeCustomName>;
@ -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<decltype(this)>(&rhs_))
return name == rhs->getName();
return false;
}
};
}

View File

@ -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<decltype(this)>(&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;
}
}

View File

@ -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);
};

View File

@ -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<decltype(this)>(&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;
}
}

View File

@ -19,6 +19,7 @@ public:
}
String getName() const override;
bool identical(const IDataTypeCustomName & rhs_) const override;
};
DataTypePtr createNested(const DataTypes & types, const Names & names);

View File

@ -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);
}
}

View File

@ -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.

View File

@ -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;
}
}

View File

@ -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); }

View File

@ -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)

View File

@ -14,4 +14,4 @@ CREATE TABLE t_r2
)
ENGINE = ReplicatedAggregatingMergeTree('/tables/{database}/t', 'r2')
ORDER BY id
SETTINGS index_granularity = 8192; -- { serverError INCOMPATIBLE_COLUMNS }
SETTINGS index_granularity = 8192; -- { serverError INCOMPATIBLE_COLUMNS }