mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
ISSUES-4006 resolve review comment & add test case
This commit is contained in:
parent
2311cda334
commit
93d73c78c9
@ -201,11 +201,11 @@ static String getUniqueColumnName(NamesAndTypesList columns_name_and_type, const
|
||||
|
||||
static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
|
||||
{
|
||||
const auto & numbers_partition = [&](const String & column_name, const DataTypePtr & type, size_t type_max_size)
|
||||
const auto & numbers_partition = [&](const String & column_name, bool is_nullable, size_t type_max_size)
|
||||
{
|
||||
ASTPtr column = std::make_shared<ASTIdentifier>(column_name);
|
||||
|
||||
if (type->isNullable())
|
||||
if (is_nullable)
|
||||
column = makeASTFunction("assumeNotNull", column);
|
||||
|
||||
return makeASTFunction("intDiv", column, std::make_shared<ASTLiteral>(UInt64(type_max_size / 1000)));
|
||||
@ -237,21 +237,24 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
|
||||
{
|
||||
best_size = type->getSizeOfValueInMemory();
|
||||
best_partition = std::make_shared<ASTIdentifier>(primary_key.name);
|
||||
|
||||
if (primary_key.type->isNullable())
|
||||
best_partition = makeASTFunction("assumeNotNull", best_partition);
|
||||
}
|
||||
else if (which.isInt16() || which.isUInt16())
|
||||
{
|
||||
best_size = type->getSizeOfValueInMemory();
|
||||
best_partition = numbers_partition(primary_key.name, type, std::numeric_limits<UInt16>::max());
|
||||
best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits<UInt16>::max());
|
||||
}
|
||||
else if (which.isInt32() || which.isUInt32())
|
||||
{
|
||||
best_size = type->getSizeOfValueInMemory();
|
||||
best_partition = numbers_partition(primary_key.name, type, std::numeric_limits<UInt32>::max());
|
||||
best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits<UInt32>::max());
|
||||
}
|
||||
else if (which.isInt64() || which.isUInt64())
|
||||
{
|
||||
best_size = type->getSizeOfValueInMemory();
|
||||
best_partition = numbers_partition(primary_key.name, type, std::numeric_limits<UInt64>::max());
|
||||
best_partition = numbers_partition(primary_key.name, primary_key.type->isNullable(), std::numeric_limits<UInt64>::max());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -19,40 +19,136 @@ using namespace DB;
|
||||
static inline ASTPtr tryRewrittenCreateQuery(const String & query, const Context & context)
|
||||
{
|
||||
ParserExternalDDLQuery external_ddl_parser;
|
||||
ASTPtr ast = parseQuery(external_ddl_parser, query, 0, 0);
|
||||
ASTPtr ast = parseQuery(external_ddl_parser, "EXTERNAL DDL FROM MySQL(test_database, test_database) " + query, 0, 0);
|
||||
|
||||
return MySQLInterpreter::InterpreterCreateImpl::getRewrittenQueries(
|
||||
*ast->as<ASTExternalDDLQuery>()->external_ddl->as<MySQLParser::ASTCreateQuery>(),
|
||||
context, "test_database", "test_database")[0];
|
||||
}
|
||||
|
||||
TEST(MySQLCreateRewritten, ColumnsDataType)
|
||||
{
|
||||
tryRegisterFunctions();
|
||||
const auto & context_holder = getContext();
|
||||
|
||||
std::vector<std::pair<String, String>> test_types
|
||||
{
|
||||
{"TINYINT", "Int8"}, {"SMALLINT", "Int16"}, {"MEDIUMINT", "Int32"}, {"INT", "Int32"},
|
||||
{"INTEGER", "Int32"}, {"BIGINT", "Int64"}, {"FLOAT", "Float32"}, {"DOUBLE", "Float64"},
|
||||
{"VARCHAR(10)", "String"}, {"CHAR(10)", "String"}, {"Date", "Date"}, {"DateTime", "DateTime"},
|
||||
{"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Int8"}
|
||||
};
|
||||
|
||||
for (const auto & [test_type, mapped_type] : test_types)
|
||||
{
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + ")", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(" + mapped_type + ")"
|
||||
", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " NOT NULL)", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `test` " + mapped_type +
|
||||
", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' NOT NULL)", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `test` " + mapped_type +
|
||||
", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
if (Poco::toUpper(test_type).find("INT") != std::string::npos)
|
||||
{
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " UNSIGNED)", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")"
|
||||
", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' UNSIGNED)", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `test` Nullable(U" + mapped_type + ")"
|
||||
", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " NOT NULL UNSIGNED)", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `test` U" + mapped_type +
|
||||
", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " COMMENT 'test_comment' UNSIGNED NOT NULL)", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `test` U" + mapped_type +
|
||||
", `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST(MySQLCreateRewritten, PartitionPolicy)
|
||||
{
|
||||
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"}
|
||||
};
|
||||
|
||||
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)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"CREATE TABLE `test_database`.`test_table_1` (`key` " + test_type + " NOT NULL PRIMARY 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)");
|
||||
}
|
||||
}
|
||||
|
||||
TEST(MySQLCreateRewritten, RewrittenQueryWithPrimaryKey)
|
||||
{
|
||||
tryRegisterFunctions();
|
||||
const auto & context_holder = getContext();
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY "
|
||||
"KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) "
|
||||
"CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL PRIMARY KEY) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version) "
|
||||
"PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL, "
|
||||
" PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8, `_version` UInt64) ENGINE = ReplacingMergeTree(_version) "
|
||||
"CREATE TABLE `test_database`.`test_table_1` (`key` int NOT NULL, PRIMARY KEY (`key`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = ReplacingMergeTree(_version) "
|
||||
"PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` int NOT NULL, "
|
||||
" key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = "
|
||||
"CREATE TABLE `test_database`.`test_table_1` (`key_1` int NOT NULL, key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key_1` Int32, `key_2` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key_1, 4294967) ORDER BY (key_1, key_2)");
|
||||
|
||||
EXPECT_EQ(queryToString(tryRewrittenCreateQuery(
|
||||
"EXTERNAL DDL FROM MySQL(test_database, test_database) CREATE TABLE `test_database`.`test_table_1` (`key_1` BIGINT NOT NULL, "
|
||||
" key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32, `_sign` Int8, `_version` UInt64) ENGINE = "
|
||||
"CREATE TABLE `test_database`.`test_table_1` (`key_1` BIGINT NOT NULL, key_2 INT NOT NULL, PRIMARY KEY (`key_1`, `key_2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8", context_holder.context)),
|
||||
"CREATE TABLE test_database.test_table_1 (`key_1` Int64, `key_2` Int32, `_sign` Int8() MATERIALIZED 1, `_version` UInt64() MATERIALIZED 1) ENGINE = "
|
||||
"ReplacingMergeTree(_version) PARTITION BY intDiv(key_2, 4294967) ORDER BY (key_1, key_2)");
|
||||
}
|
||||
|
||||
|
@ -43,7 +43,7 @@ Pipes StorageMaterializeMySQL::read(
|
||||
NameSet column_names_set = NameSet(column_names.begin(), column_names.end());
|
||||
const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr();
|
||||
|
||||
Block nested_header = nested_metadata->getSampleBlockNonMaterialized();
|
||||
Block nested_header = nested_metadata->getSampleBlock();
|
||||
ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2);
|
||||
ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user