mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
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:
commit
764bb1181f
@ -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:
|
||||
|
@ -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};
|
||||
|
125
src/Interpreters/applyTableOverride.cpp
Normal file
125
src/Interpreters/applyTableOverride.cpp
Normal 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
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
16
src/Interpreters/applyTableOverride.h
Normal file
16
src/Interpreters/applyTableOverride.h
Normal 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);
|
||||
|
||||
}
|
88
src/Interpreters/tests/gtest_table_overrides.cpp
Normal file
88
src/Interpreters/tests/gtest_table_overrides.cpp
Normal 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)",
|
||||
}
|
||||
}));
|
@ -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);
|
||||
|
@ -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:
|
||||
|
@ -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)",
|
||||
}
|
||||
}));
|
||||
|
@ -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());
|
||||
|
@ -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")
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user