Merge pull request #33017 from stigsb/table-overrides-drop-add-column-support

Remove the possibility of adding columns with table overrides
This commit is contained in:
Kseniia Sumarokova 2021-12-22 22:30:21 +03:00 committed by GitHub
commit 764bb1181f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 292 additions and 215 deletions

View File

@ -151,14 +151,32 @@ Table overrides can be used to customize the ClickHouse DDL queries, allowing yo
application. This is especially useful for controlling partitioning, which is important for the overall performance of
MaterializedMySQL.
These are the schema conversion manipulations you can do with table overrides for MaterializedMySQL:
* Modify column type. Must be compatible with the original type, or replication will fail. For example,
you can modify a UInt32 column to UInt64, but you can not modify a String column to Array(String).
* Modify [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl).
* Modify [column compression codec](../../sql-reference/statements/create/table/#codecs).
* Add [ALIAS columns](../../sql-reference/statements/create/table/#alias).
* Add [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes)
* Add [projections](../table-engines/mergetree-family/mergetree/#projections). Note that projection optimizations are
disabled when using `SELECT ... FINAL` (which MaterializedMySQL does by default), so their utility is limited here.
`INDEX ... TYPE hypothesis` as [described in the v21.12 blog post]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/)
may be more useful in this case.
* Modify [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/)
* Modify [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses)
* Modify [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses)
* Add [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses)
* Add [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses)
```sql
CREATE DATABASE db_name ENGINE = MaterializedMySQL(...)
[SETTINGS ...]
[TABLE OVERRIDE table_name (
[COLUMNS (
[name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], ...]
[INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, ...]
[PROJECTION projection_name_1 (SELECT <COLUMN LIST EXPR> [GROUP BY] [ORDER BY]), ...]
[col_name [datatype] [ALIAS expr] [CODEC(...)] [TTL expr], ...]
[INDEX index_name expr TYPE indextype[(...)] GRANULARITY val, ...]
[PROJECTION projection_name (SELECT <COLUMN LIST EXPR> [GROUP BY] [ORDER BY]), ...]
)]
[ORDER BY expr]
[PRIMARY KEY expr]
@ -182,26 +200,26 @@ TABLE OVERRIDE table1 (
),
TABLE OVERRIDE table2 (
COLUMNS (
ip_hash UInt32 MATERIALIZED xxHash32(client_ip),
client_ip String TTL created + INTERVAL 72 HOUR
)
SAMPLE BY ip_hash
)
```
The `COLUMNS` list is sparse; it contains only modified or extra (MATERIALIZED or ALIAS) columns. Modified columns with
a different type must be assignable from the original type. There is currently no validation of this or similar issues
when the `CREATE DATABASE` query executes, so extra care needs to be taken.
The `COLUMNS` list is sparse; existing columns are modified as specified, extra ALIAS columns are added. It is not
possible to add ordinary or MATERIALIZED columns. Modified columns with a different type must be assignable from the
original type. There is currently no validation of this or similar issues when the `CREATE DATABASE` query executes, so
extra care needs to be taken.
You may specify overrides for tables that do not exist yet.
!!! note "Warning"
It is easy to break replication with TABLE OVERRIDEs if not used with care. For example:
!!! warning "Warning"
It is easy to break replication with table overrides if not used with care. For example:
* If a column is added with a table override, but then later added to the source MySQL table, the converted ALTER TABLE
query in ClickHouse will fail because the column already exists.
* If an ALIAS column is added with a table override, and a column with the same name is later added to the source
MySQL table, the converted ALTER TABLE query in ClickHouse will fail and replication stops.
* It is currently possible to add overrides that reference nullable columns where not-nullable are required, such as in
`ORDER BY` or `PARTITION BY`.
`ORDER BY` or `PARTITION BY`. This will cause CREATE TABLE queries that will fail, also causing replication to stop.
## Examples of Use {#examples-of-use}
@ -218,11 +236,9 @@ mysql> SELECT * FROM test;
```
```text
+---+------+------+
| a | b | c |
+---+------+------+
| 2 | 222 | Wow! |
+---+------+------+
┌─a─┬───b─┬─c────┐
│ 2 │ 222 │ Wow! │
└───┴─────┴──────┘
```
Database in ClickHouse, exchanging data with the MySQL server:

View File

@ -25,6 +25,7 @@
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/applyTableOverride.h>
#include <Storages/IStorage.h>
namespace DB
@ -520,10 +521,10 @@ ASTs InterpreterCreateImpl::getRewrittenQueries(
rewritten_query->set(rewritten_query->storage, storage);
rewritten_query->set(rewritten_query->columns_list, columns);
if (auto table_override = tryGetTableOverride(mapped_to_database, create_query.table))
if (auto override_ast = tryGetTableOverride(mapped_to_database, create_query.table))
{
auto * override_ast = table_override->as<ASTTableOverride>();
override_ast->applyToCreateTableQuery(rewritten_query.get());
const auto & override = override_ast->as<const ASTTableOverride &>();
applyTableOverrideToCreateQuery(override, rewritten_query.get());
}
return ASTs{rewritten_query};

View File

@ -0,0 +1,125 @@
#include <Interpreters/applyTableOverride.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTTableOverrides.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ASTProjectionDeclaration.h>
namespace DB
{
void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreateQuery * create_query)
{
if (auto * columns = override.columns)
{
if (!create_query->columns_list)
create_query->set(create_query->columns_list, std::make_shared<ASTColumns>());
if (columns->columns)
{
for (const auto & override_column_ast : columns->columns->children)
{
auto * override_column = override_column_ast->as<ASTColumnDeclaration>();
if (!override_column)
continue;
if (!create_query->columns_list->columns)
create_query->columns_list->set(create_query->columns_list->columns, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->columns->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTColumnDeclaration>()->name == override_column->name;
});
/// For columns, only allow adding ALIAS (non-physical) for now.
/// TODO: This logic should instead be handled by validation that is
/// executed from InterpreterCreateQuery / InterpreterAlterQuery.
if (exists == dest_children.end())
{
if (override_column->default_specifier == "ALIAS")
dest_children.emplace_back(override_column_ast);
}
else
dest_children[exists - dest_children.begin()] = override_column_ast;
}
}
if (columns->indices)
{
for (const auto & override_index_ast : columns->indices->children)
{
auto * override_index = override_index_ast->as<ASTIndexDeclaration>();
if (!override_index)
continue;
if (!create_query->columns_list->indices)
create_query->columns_list->set(create_query->columns_list->indices, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->indices->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTIndexDeclaration>()->name == override_index->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_index_ast);
else
dest_children[exists - dest_children.begin()] = override_index_ast;
}
}
if (columns->constraints)
{
for (const auto & override_constraint_ast : columns->constraints->children)
{
auto * override_constraint = override_constraint_ast->as<ASTConstraintDeclaration>();
if (!override_constraint)
continue;
if (!create_query->columns_list->constraints)
create_query->columns_list->set(create_query->columns_list->constraints, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->constraints->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTConstraintDeclaration>()->name == override_constraint->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_constraint_ast);
else
dest_children[exists - dest_children.begin()] = override_constraint_ast;
}
}
if (columns->projections)
{
for (const auto & override_projection_ast : columns->projections->children)
{
auto * override_projection = override_projection_ast->as<ASTProjectionDeclaration>();
if (!override_projection)
continue;
if (!create_query->columns_list->projections)
create_query->columns_list->set(create_query->columns_list->projections, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->projections->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTProjectionDeclaration>()->name == override_projection->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_projection_ast);
else
dest_children[exists - dest_children.begin()] = override_projection_ast;
}
}
}
if (auto * storage = override.storage)
{
if (!create_query->storage)
create_query->set(create_query->storage, std::make_shared<ASTStorage>());
if (storage->partition_by)
create_query->storage->set(create_query->storage->partition_by, storage->partition_by->clone());
if (storage->primary_key)
create_query->storage->set(create_query->storage->primary_key, storage->primary_key->clone());
if (storage->order_by)
create_query->storage->set(create_query->storage->order_by, storage->order_by->clone());
if (storage->sample_by)
create_query->storage->set(create_query->storage->sample_by, storage->sample_by->clone());
if (storage->ttl_table)
create_query->storage->set(create_query->storage->ttl_table, storage->ttl_table->clone());
// No support for overriding ENGINE and SETTINGS
}
}
}

View File

@ -0,0 +1,16 @@
#pragma once
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTTableOverrides.h>
namespace DB
{
class ASTTableOverride;
class ASTCreateQuery;
class ASTIndentifier;
void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreateQuery * create_query);
}

View File

@ -0,0 +1,88 @@
#include <Interpreters/applyTableOverride.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <iostream>
#include <gtest/gtest.h>
namespace
{
using namespace DB;
using namespace std::literals;
}
struct TableOverrideTestCase
{
String create_database_query;
String create_table_query;
String expected_create_table_query;
};
std::ostream & operator<<(std::ostream & ostr, const TableOverrideTestCase & test_case)
{
return ostr << "database: " << test_case.create_database_query << ", table: " << test_case.create_table_query
<< ", expected: " << test_case.expected_create_table_query;
}
class TableOverrideTest : public ::testing::TestWithParam<TableOverrideTestCase>
{};
TEST_P(TableOverrideTest, applyOverrides)
{
const auto & [database_query, table_query, expected_query] = GetParam();
ParserCreateQuery parser;
ASTPtr database_ast;
ASSERT_NO_THROW(database_ast = parseQuery(parser, database_query, 0, 0));
auto * database = database_ast->as<ASTCreateQuery>();
ASSERT_NE(nullptr, database);
ASTPtr table_ast;
ASSERT_NO_THROW(table_ast = parseQuery(parser, table_query, 0, 0));
auto * table = table_ast->as<ASTCreateQuery>();
ASSERT_NE(nullptr, table);
auto table_name = table->table->as<ASTIdentifier>()->name();
if (database->table_overrides)
{
auto override_ast = database->table_overrides->tryGetTableOverride(table_name);
ASSERT_NE(nullptr, override_ast);
auto * override_table_ast = override_ast->as<ASTTableOverride>();
ASSERT_NE(nullptr, override_table_ast);
applyTableOverrideToCreateQuery(*override_table_ast, table);
}
EXPECT_EQ(expected_query, serializeAST(*table));
}
INSTANTIATE_TEST_SUITE_P(ApplyTableOverrides, TableOverrideTest,
::testing::ValuesIn(std::initializer_list<TableOverrideTestCase>{
{
"CREATE DATABASE db",
"CREATE TABLE db.t (id Int64) ENGINE=Log",
"CREATE TABLE db.t (`id` Int64) ENGINE = Log"
},
{
"CREATE DATABASE db TABLE OVERRIDE t (PARTITION BY tuple())",
"CREATE TABLE db.t (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.t (`id` Int64) ENGINE = MergeTree PARTITION BY tuple()"
},
{
"CREATE DATABASE db TABLE OVERRIDE t (COLUMNS (id UInt64 CODEC(Delta), shard UInt8 ALIAS modulo(id, 16)) PARTITION BY shard)",
"CREATE TABLE db.t (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.t (`id` UInt64 CODEC(Delta), `shard` UInt8 ALIAS id % 16) ENGINE = MergeTree PARTITION BY shard"
},
{
"CREATE DATABASE db TABLE OVERRIDE a (PARTITION BY modulo(id, 3)), TABLE OVERRIDE b (PARTITION BY modulo(id, 5))",
"CREATE TABLE db.a (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.a (`id` Int64) ENGINE = MergeTree PARTITION BY id % 3"
},
{
"CREATE DATABASE db TABLE OVERRIDE a (PARTITION BY modulo(id, 3)), TABLE OVERRIDE b (PARTITION BY modulo(id, 5))",
"CREATE TABLE db.b (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.b (`id` Int64) ENGINE = MergeTree PARTITION BY id % 5"
},
{
"CREATE DATABASE db TABLE OVERRIDE `tbl` (PARTITION BY toYYYYMM(created))",
"CREATE TABLE db.tbl (id Int64, created DateTime) ENGINE=Foo",
"CREATE TABLE db.tbl (`id` Int64, `created` DateTime) ENGINE = Foo PARTITION BY toYYYYMM(created)",
}
}));

View File

@ -71,111 +71,6 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState
settings.ostr << nl_or_nothing << ')';
}
void ASTTableOverride::applyToCreateTableQuery(ASTCreateQuery * create_query) const
{
if (columns)
{
if (!create_query->columns_list)
create_query->set(create_query->columns_list, std::make_shared<ASTColumns>());
if (columns->columns)
{
for (const auto & override_column_ast : columns->columns->children)
{
auto * override_column = override_column_ast->as<ASTColumnDeclaration>();
if (!override_column)
continue;
if (!create_query->columns_list->columns)
create_query->columns_list->set(create_query->columns_list->columns, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->columns->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTColumnDeclaration>()->name == override_column->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_column_ast);
else
dest_children[exists - dest_children.begin()] = override_column_ast;
}
}
if (columns->indices)
{
for (const auto & override_index_ast : columns->indices->children)
{
auto * override_index = override_index_ast->as<ASTIndexDeclaration>();
if (!override_index)
continue;
if (!create_query->columns_list->indices)
create_query->columns_list->set(create_query->columns_list->indices, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->indices->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTIndexDeclaration>()->name == override_index->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_index_ast);
else
dest_children[exists - dest_children.begin()] = override_index_ast;
}
}
if (columns->constraints)
{
for (const auto & override_constraint_ast : columns->constraints->children)
{
auto * override_constraint = override_constraint_ast->as<ASTConstraintDeclaration>();
if (!override_constraint)
continue;
if (!create_query->columns_list->constraints)
create_query->columns_list->set(create_query->columns_list->constraints, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->constraints->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTConstraintDeclaration>()->name == override_constraint->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_constraint_ast);
else
dest_children[exists - dest_children.begin()] = override_constraint_ast;
}
}
if (columns->projections)
{
for (const auto & override_projection_ast : columns->projections->children)
{
auto * override_projection = override_projection_ast->as<ASTProjectionDeclaration>();
if (!override_projection)
continue;
if (!create_query->columns_list->projections)
create_query->columns_list->set(create_query->columns_list->projections, std::make_shared<ASTExpressionList>());
auto & dest_children = create_query->columns_list->projections->children;
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
{
return node->as<ASTProjectionDeclaration>()->name == override_projection->name;
});
if (exists == dest_children.end())
dest_children.emplace_back(override_projection_ast);
else
dest_children[exists - dest_children.begin()] = override_projection_ast;
}
}
}
if (storage)
{
if (!create_query->storage)
create_query->set(create_query->storage, std::make_shared<ASTStorage>());
if (storage->partition_by)
create_query->storage->set(create_query->storage->partition_by, storage->partition_by->clone());
if (storage->primary_key)
create_query->storage->set(create_query->storage->primary_key, storage->primary_key->clone());
if (storage->order_by)
create_query->storage->set(create_query->storage->order_by, storage->order_by->clone());
if (storage->sample_by)
create_query->storage->set(create_query->storage->sample_by, storage->sample_by->clone());
if (storage->ttl_table)
create_query->storage->set(create_query->storage->ttl_table, storage->ttl_table->clone());
// not supporting overriding ENGINE
}
}
ASTPtr ASTTableOverrideList::clone() const
{
auto res = std::make_shared<ASTTableOverrideList>(*this);

View File

@ -26,7 +26,6 @@ public:
String getID(char) const override { return "TableOverride " + table_name; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void applyToCreateTableQuery(ASTCreateQuery * create_query) const;
};
/// List of table overrides, for example:

View File

@ -1,4 +1,5 @@
#include <IO/WriteBufferFromOStream.h>
#include <Interpreters/applyTableOverride.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -57,46 +58,6 @@ TEST_P(ParserTest, parseQuery)
}
}
struct TableOverrideTestCase
{
DB::String create_database_query;
DB::String create_table_query;
DB::String expected_create_table_query;
};
std::ostream & operator<<(std::ostream & ostr, const TableOverrideTestCase & test_case)
{
return ostr << "database: " << test_case.create_database_query << ", table: " << test_case.create_table_query
<< ", expected: " << test_case.expected_create_table_query;
}
class TableOverrideTest : public ::testing::TestWithParam<TableOverrideTestCase>
{};
TEST_P(TableOverrideTest, applyOverrides)
{
const auto & [database_query, table_query, expected_query] = GetParam();
ParserCreateQuery parser;
ASTPtr database_ast;
ASSERT_NO_THROW(database_ast = parseQuery(parser, database_query, 0, 0));
auto * database = database_ast->as<ASTCreateQuery>();
ASSERT_NE(nullptr, database);
ASTPtr table_ast;
ASSERT_NO_THROW(table_ast = parseQuery(parser, table_query, 0, 0));
auto * table = table_ast->as<ASTCreateQuery>();
ASSERT_NE(nullptr, table);
auto table_name = table->table->as<ASTIdentifier>()->name();
if (database->table_overrides)
{
auto override_ast = database->table_overrides->tryGetTableOverride(table_name);
ASSERT_NE(nullptr, override_ast);
auto * override_table_ast = override_ast->as<ASTTableOverride>();
ASSERT_NE(nullptr, override_table_ast);
override_table_ast->applyToCreateTableQuery(table);
}
EXPECT_EQ(expected_query, serializeAST(*table));
}
INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery, ParserTest,
::testing::Combine(
::testing::Values(std::make_shared<ParserOptimizeQuery>()),
@ -265,37 +226,3 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest,
"CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)\nCOMMENT 'db comment'"
}
})));
INSTANTIATE_TEST_SUITE_P(ApplyTableOverrides, TableOverrideTest,
::testing::ValuesIn(std::initializer_list<TableOverrideTestCase>{
{
"CREATE DATABASE db",
"CREATE TABLE db.t (id Int64) ENGINE=Log",
"CREATE TABLE db.t (`id` Int64) ENGINE = Log"
},
{
"CREATE DATABASE db TABLE OVERRIDE t (PARTITION BY tuple())",
"CREATE TABLE db.t (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.t (`id` Int64) ENGINE = MergeTree PARTITION BY tuple()"
},
{
"CREATE DATABASE db TABLE OVERRIDE t (COLUMNS (id UInt64, shard UInt8 MATERIALIZED modulo(id, 16)) PARTITION BY shard)",
"CREATE TABLE db.t (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.t (`id` UInt64, `shard` UInt8 MATERIALIZED id % 16) ENGINE = MergeTree PARTITION BY shard"
},
{
"CREATE DATABASE db TABLE OVERRIDE a (PARTITION BY modulo(id, 3)), TABLE OVERRIDE b (PARTITION BY modulo(id, 5))",
"CREATE TABLE db.a (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.a (`id` Int64) ENGINE = MergeTree PARTITION BY id % 3"
},
{
"CREATE DATABASE db TABLE OVERRIDE a (PARTITION BY modulo(id, 3)), TABLE OVERRIDE b (PARTITION BY modulo(id, 5))",
"CREATE TABLE db.b (id Int64) ENGINE=MergeTree",
"CREATE TABLE db.b (`id` Int64) ENGINE = MergeTree PARTITION BY id % 5"
},
{
"CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl` (PARTITION BY toYYYYMM(created))",
"CREATE TABLE db.tbl (id Int64, created DateTime) ENGINE=Foo",
"CREATE TABLE db.tbl (`id` Int64, `created` DateTime) ENGINE = Foo PARTITION BY toYYYYMM(created)",
}
}));

View File

@ -24,6 +24,7 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/applyTableOverride.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterDropQuery.h>
@ -395,7 +396,7 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(
{
auto create_table_query = std::make_shared<ASTCreateQuery>();
if (table_override)
table_override->applyToCreateTableQuery(create_table_query.get());
applyTableOverrideToCreateQuery(*table_override, create_table_query.get());
auto table_id = getStorageID();
create_table_query->setTable(getNestedTableName());

View File

@ -1066,10 +1066,19 @@ def table_overrides(clickhouse_node, mysql_node, service_name):
mysql_node.query("COMMIT")
clickhouse_node.query(f"""
CREATE DATABASE table_overrides ENGINE=MaterializeMySQL('{service_name}:3306', 'table_overrides', 'root', 'clickhouse')
TABLE OVERRIDE t1 (COLUMNS (sensor_id UInt64))
TABLE OVERRIDE t1 (COLUMNS (sensor_id UInt64, temp_f Nullable(Float32) ALIAS if(isNull(temperature), NULL, (temperature * 9 / 5) + 32)))
""")
check_query(
clickhouse_node,
"SELECT type FROM system.columns WHERE database = 'table_overrides' AND table = 't1' AND name = 'sensor_id'",
"UInt64\n")
check_query(
clickhouse_node,
"SELECT type, default_kind FROM system.columns WHERE database = 'table_overrides' AND table = 't1' AND name = 'temp_f'",
"Nullable(Float32)\tALIAS\n")
check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1000\n")
check_query(clickhouse_node, "SELECT type FROM system.columns WHERE database = 'table_overrides' AND table = 't1' AND name = 'sensor_id'", "UInt64\n")
mysql_node.query("INSERT INTO table_overrides.t1 VALUES(1001, '2021-10-01 00:00:00', 42.0)")
check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1001\n")
clickhouse_node.query("DROP DATABASE IF EXISTS table_overrides")
mysql_node.query("DROP DATABASE IF EXISTS table_overrides")