mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
implement CREATE OR REPLACE TABLE
This commit is contained in:
parent
2c2e1c915c
commit
42911e2438
@ -103,12 +103,15 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
create->attach = true;
|
||||
|
||||
/// We remove everything that is not needed for ATTACH from the query.
|
||||
assert(!create->temporary);
|
||||
create->database.clear();
|
||||
create->as_database.clear();
|
||||
create->as_table.clear();
|
||||
create->if_not_exists = false;
|
||||
create->is_populate = false;
|
||||
create->replace_view = false;
|
||||
create->replace_table = false;
|
||||
create->create_or_replace = false;
|
||||
|
||||
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
|
||||
if (!create->is_view && !create->is_materialized_view && !create->is_live_view)
|
||||
|
@ -733,6 +733,36 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
|
||||
/// Ignore UUID if it's ON CLUSTER query
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
if (create.replace_table)
|
||||
{
|
||||
if (database->getUUID() == UUIDHelpers::Nil)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"{} query is supported only for Atomic databases",
|
||||
create.create_or_replace ? "CREATE OR REPLACE TABLE" : "REPLACE TABLE");
|
||||
|
||||
UUID uuid_of_table_to_replace;
|
||||
if (create.create_or_replace)
|
||||
{
|
||||
uuid_of_table_to_replace = context.tryResolveStorageID(StorageID(create.database, create.table)).uuid;
|
||||
if (uuid_of_table_to_replace == UUIDHelpers::Nil)
|
||||
{
|
||||
/// Convert to usual CREATE
|
||||
create.replace_table = false;
|
||||
assert(!database->isTableExist(create.table, context));
|
||||
}
|
||||
else
|
||||
create.table = "_tmp_replace_" + toString(uuid_of_table_to_replace);
|
||||
}
|
||||
else
|
||||
{
|
||||
uuid_of_table_to_replace = context.resolveStorageID(StorageID(create.database, create.table)).uuid;
|
||||
if (uuid_of_table_to_replace == UUIDHelpers::Nil)
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
|
||||
backQuoteIfNeed(create.database), backQuoteIfNeed(create.table));
|
||||
create.table = "_tmp_replace_" + toString(uuid_of_table_to_replace);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -756,9 +786,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
auto query = database->getCreateTableQuery(create.table, context);
|
||||
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
|
||||
if (create.is_dictionary)
|
||||
throw Exception(
|
||||
"Cannot ATTACH TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(create.table) + ", it is a Dictionary",
|
||||
ErrorCodes::INCORRECT_QUERY);
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
||||
"Cannot ATTACH TABLE {}.{}, it is a Dictionary",
|
||||
backQuoteIfNeed(database_name), backQuoteIfNeed(create.table));
|
||||
create.attach = true;
|
||||
create.attach_short_syntax = true;
|
||||
create.if_not_exists = if_not_exists;
|
||||
@ -804,6 +834,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
|
||||
TableProperties properties = setProperties(create);
|
||||
|
||||
if (create.replace_table)
|
||||
return doCreateOrReplaceTable(create, properties);
|
||||
|
||||
/// Actually creates table
|
||||
bool created = doCreateTable(create, properties);
|
||||
if (!created) /// Table already exists
|
||||
@ -820,20 +853,19 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
String data_path;
|
||||
DatabasePtr database;
|
||||
|
||||
const String table_name = create.table;
|
||||
bool need_add_to_database = !create.temporary;
|
||||
if (need_add_to_database)
|
||||
{
|
||||
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
|
||||
* If table doesn't exist, one thread is creating table, while others wait in DDLGuard.
|
||||
*/
|
||||
guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name);
|
||||
guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table);
|
||||
|
||||
database = DatabaseCatalog::instance().getDatabase(create.database);
|
||||
assertOrSetUUID(create, database);
|
||||
|
||||
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
|
||||
if (database->isTableExist(table_name, context))
|
||||
if (database->isTableExist(create.table, context))
|
||||
{
|
||||
/// TODO Check structure of table
|
||||
if (create.if_not_exists)
|
||||
@ -843,27 +875,27 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
/// when executing CREATE OR REPLACE VIEW, drop current existing view
|
||||
auto drop_ast = std::make_shared<ASTDropQuery>();
|
||||
drop_ast->database = create.database;
|
||||
drop_ast->table = table_name;
|
||||
drop_ast->table = create.table;
|
||||
drop_ast->no_ddl_lock = true;
|
||||
|
||||
InterpreterDropQuery interpreter(drop_ast, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
else
|
||||
throw Exception("Table " + create.database + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists.", backQuoteIfNeed(create.database), backQuoteIfNeed(create.table));
|
||||
}
|
||||
|
||||
data_path = database->getTableDataPath(create);
|
||||
if (!create.attach && !data_path.empty() && fs::exists(fs::path{context.getPath()} / data_path))
|
||||
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Directory for table data {} already exists", String(data_path));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (create.if_not_exists && context.tryResolveStorageID({"", table_name}, Context::ResolveExternal))
|
||||
if (create.if_not_exists && context.tryResolveStorageID({"", create.table}, Context::ResolveExternal))
|
||||
return false;
|
||||
|
||||
auto temporary_table = TemporaryTableHolder(context, properties.columns, properties.constraints, query_ptr);
|
||||
context.getSessionContext().addExternalTable(table_name, std::move(temporary_table));
|
||||
context.getSessionContext().addExternalTable(create.table, std::move(temporary_table));
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -903,7 +935,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
"ATTACH ... FROM ... query is not supported for {} table engine, "
|
||||
"because such tables do not store any data on disk. Use CREATE instead.", res->getName());
|
||||
|
||||
database->createTable(context, table_name, res, query_ptr);
|
||||
database->createTable(context, create.table, res, query_ptr);
|
||||
|
||||
/// Move table data to the proper place. Wo do not move data earlier to avoid situations
|
||||
/// when data directory moved, but table has not been created due to some error.
|
||||
@ -927,6 +959,50 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
||||
const InterpreterCreateQuery::TableProperties & properties)
|
||||
{
|
||||
auto ast_drop = std::make_shared<ASTDropQuery>();
|
||||
String table_to_replace_name = create.table;
|
||||
bool created = false;
|
||||
bool replaced = false;
|
||||
|
||||
try
|
||||
{
|
||||
[[maybe_unused]] bool done = doCreateTable(create, properties);
|
||||
assert(done);
|
||||
ast_drop->table = create.table;
|
||||
ast_drop->database = create.database;
|
||||
ast_drop->kind = ASTDropQuery::Drop;
|
||||
created = true;
|
||||
if (!create.replace_table)
|
||||
return fillTableIfNeeded(create);
|
||||
|
||||
auto ast_rename = std::make_shared<ASTRenameQuery>();
|
||||
ASTRenameQuery::Element elem
|
||||
{
|
||||
ASTRenameQuery::Table{create.database, create.table},
|
||||
ASTRenameQuery::Table{create.database, table_to_replace_name}
|
||||
};
|
||||
ast_rename->elements.push_back(std::move(elem));
|
||||
ast_rename->exchange = true;
|
||||
InterpreterRenameQuery(ast_rename, context).execute();
|
||||
replaced = true;
|
||||
|
||||
InterpreterDropQuery(ast_drop, context).execute();
|
||||
|
||||
create.table = table_to_replace_name;
|
||||
return fillTableIfNeeded(create);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (created && create.replace_table && !replaced)
|
||||
InterpreterDropQuery(ast_drop, context).execute();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create)
|
||||
{
|
||||
/// If the query is a CREATE SELECT, insert the data into the table.
|
||||
@ -1079,22 +1155,22 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const
|
||||
}
|
||||
else if (create.is_view || create.is_materialized_view || create.is_live_view)
|
||||
{
|
||||
if (create.temporary)
|
||||
required_access.emplace_back(AccessType::CREATE_TEMPORARY_TABLE);
|
||||
assert(!create.temporary);
|
||||
if (create.replace_view)
|
||||
required_access.emplace_back(AccessType::DROP_VIEW | AccessType::CREATE_VIEW, create.database, create.table);
|
||||
else
|
||||
{
|
||||
if (create.replace_view)
|
||||
required_access.emplace_back(AccessType::DROP_VIEW | AccessType::CREATE_VIEW, create.database, create.table);
|
||||
else
|
||||
required_access.emplace_back(AccessType::CREATE_VIEW, create.database, create.table);
|
||||
}
|
||||
required_access.emplace_back(AccessType::CREATE_VIEW, create.database, create.table);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (create.temporary)
|
||||
required_access.emplace_back(AccessType::CREATE_TEMPORARY_TABLE);
|
||||
else
|
||||
{
|
||||
if (create.replace_table)
|
||||
required_access.emplace_back(AccessType::DROP_TABLE, create.database, create.table);
|
||||
required_access.emplace_back(AccessType::CREATE_TABLE, create.database, create.table);
|
||||
}
|
||||
}
|
||||
|
||||
if (create.to_table_id)
|
||||
|
@ -81,6 +81,7 @@ private:
|
||||
|
||||
/// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false.
|
||||
bool doCreateTable(ASTCreateQuery & create, const TableProperties & properties);
|
||||
BlockIO doCreateOrReplaceTable(ASTCreateQuery & create, const InterpreterCreateQuery::TableProperties & properties);
|
||||
/// Inserts data in created table if it's CREATE ... SELECT
|
||||
BlockIO fillTableIfNeeded(const ASTCreateQuery & create);
|
||||
|
||||
|
@ -230,19 +230,28 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
|
||||
if (!is_dictionary)
|
||||
{
|
||||
std::string what = "TABLE";
|
||||
String action = "CREATE";
|
||||
if (attach)
|
||||
action = "ATTACH";
|
||||
else if (replace_view)
|
||||
action = "CREATE OR REPLACE";
|
||||
else if (replace_table && create_or_replace)
|
||||
action = "CREATE OR REPLACE";
|
||||
else if (replace_table)
|
||||
action = "REPLACE";
|
||||
|
||||
String what = "TABLE";
|
||||
if (is_view)
|
||||
what = "VIEW";
|
||||
if (is_materialized_view)
|
||||
else if (is_materialized_view)
|
||||
what = "MATERIALIZED VIEW";
|
||||
if (is_live_view)
|
||||
else if (is_live_view)
|
||||
what = "LIVE VIEW";
|
||||
|
||||
settings.ostr
|
||||
<< (settings.hilite ? hilite_keyword : "")
|
||||
<< (attach ? "ATTACH " : "CREATE ")
|
||||
<< action << " "
|
||||
<< (temporary ? "TEMPORARY " : "")
|
||||
<< (replace_view ? "OR REPLACE " : "")
|
||||
<< what << " "
|
||||
<< (if_not_exists ? "IF NOT EXISTS " : "")
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
|
@ -64,7 +64,7 @@ public:
|
||||
bool replace_view{false}; /// CREATE OR REPLACE VIEW
|
||||
ASTColumns * columns_list = nullptr;
|
||||
ASTExpressionList * tables = nullptr;
|
||||
//FIXME
|
||||
|
||||
StorageID to_table_id = StorageID::createEmpty(); /// For CREATE MATERIALIZED VIEW mv TO table.
|
||||
ASTStorage * storage = nullptr;
|
||||
String as_database;
|
||||
@ -81,6 +81,9 @@ public:
|
||||
|
||||
std::optional<String> attach_from_path = std::nullopt;
|
||||
|
||||
bool replace_table{false};
|
||||
bool create_or_replace{false};
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; }
|
||||
|
||||
|
@ -355,6 +355,8 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
{
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_attach("ATTACH");
|
||||
ParserKeyword s_replace("REPLACE");
|
||||
ParserKeyword s_or_replace("OR REPLACE");
|
||||
ParserKeyword s_temporary("TEMPORARY");
|
||||
ParserKeyword s_table("TABLE");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
@ -383,25 +385,32 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
|
||||
String cluster_str;
|
||||
bool attach = false;
|
||||
bool replace = false;
|
||||
bool or_replace = false;
|
||||
bool if_not_exists = false;
|
||||
bool is_temporary = false;
|
||||
|
||||
if (!s_create.ignore(pos, expected))
|
||||
if (s_create.ignore(pos, expected))
|
||||
{
|
||||
if (s_attach.ignore(pos, expected))
|
||||
attach = true;
|
||||
else
|
||||
return false;
|
||||
if (s_or_replace.ignore(pos, expected))
|
||||
replace = or_replace = true;
|
||||
}
|
||||
else if (s_attach.ignore(pos, expected))
|
||||
attach = true;
|
||||
else if (s_replace.ignore(pos, expected))
|
||||
replace = true;
|
||||
else
|
||||
return false;
|
||||
|
||||
if (s_temporary.ignore(pos, expected))
|
||||
|
||||
if (!replace && !or_replace && s_temporary.ignore(pos, expected))
|
||||
{
|
||||
is_temporary = true;
|
||||
}
|
||||
if (!s_table.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
if (!replace && !or_replace && s_if_not_exists.ignore(pos, expected))
|
||||
if_not_exists = true;
|
||||
|
||||
if (!table_name_p.parse(pos, table, expected))
|
||||
@ -505,6 +514,8 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
query->as_table_function = as_table_function;
|
||||
|
||||
query->attach = attach;
|
||||
query->replace_table = replace;
|
||||
query->create_or_replace = or_replace;
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->temporary = is_temporary;
|
||||
|
||||
|
@ -0,0 +1,8 @@
|
||||
t1
|
||||
CREATE TABLE test_01185.t1\n(\n `n` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192
|
||||
t1
|
||||
CREATE TABLE test_01185.t1\n(\n `n` UInt64,\n `s` Nullable(String)\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192
|
||||
2 \N
|
||||
t1
|
||||
CREATE TABLE test_01185.t1\n(\n `n` UInt64\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192
|
||||
3
|
23
tests/queries/0_stateless/01185_create_or_replace_table.sql
Normal file
23
tests/queries/0_stateless/01185_create_or_replace_table.sql
Normal file
@ -0,0 +1,23 @@
|
||||
drop database if exists test_01185;
|
||||
create database test_01185 engine=Atomic;
|
||||
|
||||
replace table test_01185.t1 (n UInt64, s String) engine=MergeTree order by n; -- { serverError 60 }
|
||||
show tables from test_01185;
|
||||
create or replace table test_01185.t1 (n UInt64, s String) engine=MergeTree order by n;
|
||||
show tables from test_01185;
|
||||
show create table test_01185.t1;
|
||||
|
||||
insert into test_01185.t1 values (1, 'test');
|
||||
create or replace table test_01185.t1 (n UInt64, s Nullable(String)) engine=MergeTree order by n;
|
||||
insert into test_01185.t1 values (2, null);
|
||||
show tables from test_01185;
|
||||
show create table test_01185.t1;
|
||||
select * from test_01185.t1;
|
||||
|
||||
replace table test_01185.t1 (n UInt64) engine=MergeTree order by n;
|
||||
insert into test_01185.t1 values (3);
|
||||
show tables from test_01185;
|
||||
show create table test_01185.t1;
|
||||
select * from test_01185.t1;
|
||||
|
||||
drop database test_01185;
|
Loading…
Reference in New Issue
Block a user