ISSUES-14114 fix create parse failure when mysql nullable primary key

This commit is contained in:
zhang2014 2020-09-02 13:05:09 +08:00
parent e2fa0eae2f
commit 1f908af189
3 changed files with 100 additions and 42 deletions

View File

@ -18,6 +18,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <Parsers/MySQL/ASTDeclareIndex.h>
#include <Common/quoteString.h>
#include <Common/assert_cast.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Storages/IStorage.h>
@ -124,8 +125,37 @@ static NamesAndTypesList getNames(const ASTFunction & expr, const Context & cont
return expression->getRequiredColumnsWithTypes();
}
static NamesAndTypesList modifyPrimaryKeysToNonNullable(const NamesAndTypesList & primary_keys, NamesAndTypesList & columns)
{
/// https://dev.mysql.com/doc/refman/5.7/en/create-table.html#create-table-indexes-keys
/// PRIMARY KEY:
/// A unique index where all key columns must be defined as NOT NULL.
/// If they are not explicitly declared as NOT NULL, MySQL declares them so implicitly (and silently).
/// A table can have only one PRIMARY KEY. The name of a PRIMARY KEY is always PRIMARY,
/// which thus cannot be used as the name for any other kind of index.
NamesAndTypesList non_nullable_primary_keys;
for (const auto & primary_key : primary_keys)
{
if (!primary_key.type->isNullable())
non_nullable_primary_keys.emplace_back(primary_key);
else
{
non_nullable_primary_keys.emplace_back(
NameAndTypePair(primary_key.name, assert_cast<const DataTypeNullable *>(primary_key.type.get())->getNestedType()));
for (auto & column : columns)
{
if (column.name == primary_key.name)
column.type = assert_cast<const DataTypeNullable *>(column.type.get())->getNestedType();
}
}
}
return non_nullable_primary_keys;
}
static inline std::tuple<NamesAndTypesList, NamesAndTypesList, NamesAndTypesList, NameSet> getKeys(
ASTExpressionList * columns_define, ASTExpressionList * indices_define, const Context & context, const NamesAndTypesList & columns)
ASTExpressionList * columns_define, ASTExpressionList * indices_define, const Context & context, NamesAndTypesList & columns)
{
NameSet increment_columns;
auto keys = makeASTFunction("tuple");
@ -171,8 +201,9 @@ static inline std::tuple<NamesAndTypesList, NamesAndTypesList, NamesAndTypesList
}
}
return std::make_tuple(
getNames(*primary_keys, context, columns), getNames(*unique_keys, context, columns), getNames(*keys, context, columns), increment_columns);
const auto & primary_keys_names_and_types = getNames(*primary_keys, context, columns);
const auto & non_nullable_primary_keys_names_and_types = modifyPrimaryKeysToNonNullable(primary_keys_names_and_types, columns);
return std::make_tuple(non_nullable_primary_keys_names_and_types, getNames(*unique_keys, context, columns), getNames(*keys, context, columns), increment_columns);
}
static String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const String & prefix)
@ -201,14 +232,13 @@ static String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const
static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
{
const auto & numbers_partition = [&](const String & column_name, bool is_nullable, size_t type_max_size)
const auto & numbers_partition = [&](const String & column_name, size_t type_max_size) -> ASTPtr
{
ASTPtr column = std::make_shared<ASTIdentifier>(column_name);
if (type_max_size <= 1000)
return std::make_shared<ASTIdentifier>(column_name);
if (is_nullable)
column = makeASTFunction("assumeNotNull", column);
return makeASTFunction("intDiv", column, std::make_shared<ASTLiteral>(UInt64(type_max_size / 1000)));
return makeASTFunction("intDiv", std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(UInt64(type_max_size / 1000)));
};
ASTPtr best_partition;
@ -219,16 +249,12 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
WhichDataType which(type);
if (which.isNullable())
{
type = (static_cast<const DataTypeNullable &>(*type)).getNestedType();
which = WhichDataType(type);
}
throw Exception("LOGICAL ERROR: MySQL primary key must be not null, it is a bug.", ErrorCodes::LOGICAL_ERROR);
if (which.isDateOrDateTime())
{
/// In any case, date or datetime is always the best partitioning key
ASTPtr res = std::make_shared<ASTIdentifier>(primary_key.name);
return makeASTFunction("toYYYYMM", primary_key.type->isNullable() ? makeASTFunction("assumeNotNull", res) : res);
return makeASTFunction("toYYYYMM", std::make_shared<ASTIdentifier>(primary_key.name));
}
if (type->haveMaximumSizeOfValue() && (!best_size || type->getSizeOfValueInMemory() < best_size))
@ -236,25 +262,22 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
if (which.isInt8() || which.isUInt8())
{
best_size = type->getSizeOfValueInMemory();
best_partition = std::make_shared<ASTIdentifier>(primary_key.name);
if (primary_key.type->isNullable())
best_partition = makeASTFunction("assumeNotNull", best_partition);
best_partition = numbers_partition(primary_key.name, std::numeric_limits<UInt8>::max());
}
else if (which.isInt16() || which.isUInt16())
{
best_size = type->getSizeOfValueInMemory();
best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits<UInt16>::max());
best_partition = numbers_partition(primary_key.name, std::numeric_limits<UInt16>::max());
}
else if (which.isInt32() || which.isUInt32())
{
best_size = type->getSizeOfValueInMemory();
best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits<UInt32>::max());
best_partition = numbers_partition(primary_key.name, std::numeric_limits<UInt32>::max());
}
else if (which.isInt64() || which.isUInt64())
{
best_size = type->getSizeOfValueInMemory();
best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits<UInt64>::max());
best_partition = numbers_partition(primary_key.name, std::numeric_limits<UInt64>::max());
}
}
}
@ -266,12 +289,12 @@ static ASTPtr getOrderByPolicy(
const NamesAndTypesList & primary_keys, const NamesAndTypesList & unique_keys, const NamesAndTypesList & keys, const NameSet & increment_columns)
{
NameSet order_by_columns_set;
std::deque<std::vector<String>> order_by_columns_list;
std::deque<NamesAndTypesList> order_by_columns_list;
const auto & add_order_by_expression = [&](const NamesAndTypesList & names_and_types)
{
std::vector<String> increment_keys;
std::vector<String> non_increment_keys;
NamesAndTypesList increment_keys;
NamesAndTypesList non_increment_keys;
for (const auto & [name, type] : names_and_types)
{
@ -280,13 +303,13 @@ static ASTPtr getOrderByPolicy(
if (increment_columns.count(name))
{
increment_keys.emplace_back(name);
order_by_columns_set.emplace(name);
increment_keys.emplace_back(NameAndTypePair(name, type));
}
else
{
order_by_columns_set.emplace(name);
non_increment_keys.emplace_back(name);
non_increment_keys.emplace_back(NameAndTypePair(name, type));
}
}
@ -305,8 +328,13 @@ static ASTPtr getOrderByPolicy(
for (const auto & order_by_columns : order_by_columns_list)
{
for (const auto & order_by_column : order_by_columns)
order_by_expression->arguments->children.emplace_back(std::make_shared<ASTIdentifier>(order_by_column));
for (const auto & [name, type] : order_by_columns)
{
order_by_expression->arguments->children.emplace_back(std::make_shared<ASTIdentifier>(name));
if (type->isNullable())
order_by_expression->arguments->children.back() = makeASTFunction("assumeNotNull", order_by_expression->arguments->children.back());
}
}
return order_by_expression;

View File

@ -103,21 +103,12 @@ TEST(MySQLCreateRewritten, PartitionPolicy)
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"}
};
const auto & replace_string = [](const String & str, const String & old_str, const String & new_str)
{
String new_string = str;
size_t pos = new_string.find(old_str);
if (pos != std::string::npos)
new_string = new_string.replace(pos, old_str.size(), new_str);
return new_string;
};
for (const auto & [test_type, mapped_type, partition_policy] : test_types)
{
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
"CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " PRIMARY KEY)", context_holder.context)),
"CREATE TABLE test_database.test_table_1 (`key` Nullable(" + mapped_type + "), `_sign` Int8() MATERIALIZED 1, "
"`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + replace_string(partition_policy, "key", "assumeNotNull(key)") + " ORDER BY tuple(key)");
"CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, "
"`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)");
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
"CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY KEY)", context_holder.context)),
@ -126,6 +117,45 @@ TEST(MySQLCreateRewritten, PartitionPolicy)
}
}
TEST(MySQLCreateRewritten, OrderbyPolicy)
{
tryRegisterFunctions();
const auto & context_holder = getContext();
std::vector<std::tuple<String, String, String>> test_types
{
{"TINYINT", "Int8", " PARTITION BY key"}, {"SMALLINT", "Int16", " PARTITION BY intDiv(key, 65)"},
{"MEDIUMINT", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"INT", "Int32", " PARTITION BY intDiv(key, 4294967)"},
{"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"},
{"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""},
{"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"},
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"}
};
for (const auto & [test_type, mapped_type, partition_policy] : test_types)
{
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
"CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " PRIMARY KEY, `key2` " + test_type + " UNIQUE KEY)", context_holder.context)),
"CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` Nullable(" + mapped_type + "), `_sign` Int8() MATERIALIZED 1, "
"`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, assumeNotNull(key2))");
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
"CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY KEY, `key2` " + test_type + " NOT NULL UNIQUE KEY)", context_holder.context)),
"CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, "
"`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)");
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
"CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " KEY UNIQUE KEY)", context_holder.context)),
"CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, "
"`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY tuple(key)");
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
"CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + ", `key2` " + test_type + " UNIQUE KEY, PRIMARY KEY(`key`, `key2`))", context_holder.context)),
"CREATE TABLE test_database.test_table_1 (`key` " + mapped_type + ", `key2` " + mapped_type + ", `_sign` Int8() MATERIALIZED 1, "
"`_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version)" + partition_policy + " ORDER BY (key, key2)");
}
}
TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey)
{
tryRegisterFunctions();

View File

@ -46,10 +46,10 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node,
OptionDescribe("DEFAULT", "default", std::make_unique<ParserExpression>()),
OptionDescribe("ON UPDATE", "on_update", std::make_unique<ParserExpression>()),
OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_unique<ParserAlwaysTrue>()),
OptionDescribe("UNIQUE", "unique_key", std::make_unique<ParserAlwaysTrue>()),
OptionDescribe("UNIQUE KEY", "unique_key", std::make_unique<ParserAlwaysTrue>()),
OptionDescribe("KEY", "primary_key", std::make_unique<ParserAlwaysTrue>()),
OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique<ParserAlwaysTrue>()),
OptionDescribe("UNIQUE", "unique_key", std::make_unique<ParserAlwaysTrue>()),
OptionDescribe("KEY", "primary_key", std::make_unique<ParserAlwaysTrue>()),
OptionDescribe("COMMENT", "comment", std::make_unique<ParserStringLiteral>()),
OptionDescribe("CHARACTER SET", "charset_name", std::make_unique<ParserCharsetName>()),
OptionDescribe("COLLATE", "collate", std::make_unique<ParserCharsetName>()),