mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
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:
parent
3152cd25dc
commit
44a3dc1cd8
@ -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;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,7 @@ public:
|
||||
}
|
||||
|
||||
String getName() const override;
|
||||
bool identical(const IDataTypeCustomName & rhs_) const override;
|
||||
};
|
||||
|
||||
DataTypePtr createNested(const DataTypes & types, const Names & names);
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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); }
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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 }
|
||||
|
Loading…
Reference in New Issue
Block a user