support for CREATE OR REPLACE VIEW

This commit is contained in:
bgranvea 2019-03-11 17:50:31 +01:00
parent 0f33986c1f
commit 2029f71869
10 changed files with 42 additions and 8 deletions

View File

@ -35,6 +35,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query)
create.as_table.clear();
create.if_not_exists = false;
create.is_populate = false;
create.replace_view = 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)

View File

@ -45,6 +45,7 @@
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Compression/CompressionFactory.h>
#include <Interpreters/InterpreterDropQuery.h>
namespace DB
@ -623,6 +624,17 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
{
if (create.if_not_exists)
return {};
else if (create.replace_view)
{
/// when executing CREATE OR REPLACE VIEW, drop current existing view
auto drop_ast = std::make_shared<ASTDropQuery>();
drop_ast->database = database_name;
drop_ast->table = table_name;
drop_ast->no_ddl_lock = true;
InterpreterDropQuery interpreter(drop_ast, context);
interpreter.execute();
}
else
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}

View File

@ -39,7 +39,7 @@ BlockIO InterpreterDropQuery::execute()
return executeDDLQueryOnCluster(query_ptr, context, {drop.database});
if (!drop.table.empty())
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary);
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
else if (!drop.database.empty())
return executeToDatabase(drop.database, drop.kind, drop.if_exists);
else
@ -47,7 +47,7 @@ BlockIO InterpreterDropQuery::execute()
}
BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary)
BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock)
{
if (if_temporary || database_name_.empty())
{
@ -59,7 +59,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t
String database_name = database_name_.empty() ? context.getCurrentDatabase() : database_name_;
auto ddl_guard = context.getDDLGuard(database_name, table_name);
auto ddl_guard = (!no_ddl_lock ? context.getDDLGuard(database_name, table_name) : nullptr);
DatabaseAndTable database_and_table = tryGetDatabaseAndTable(database_name, table_name, if_exists);
@ -166,7 +166,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
{
String current_table_name = iterator->table()->getTableName();
executeToTable(database_name, current_table_name, kind, false, false);
executeToTable(database_name, current_table_name, kind, false, false, false);
}
auto context_lock = context.getLock();

View File

@ -32,7 +32,7 @@ private:
BlockIO executeToDatabase(String & database_name, ASTDropQuery::Kind kind, bool if_exists);
BlockIO executeToTable(String & database_name, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary);
BlockIO executeToTable(String & database_name, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock);
DatabasePtr tryGetDatabase(String & database_name, bool exists);

View File

@ -180,6 +180,7 @@ public:
bool is_view{false};
bool is_materialized_view{false};
bool is_populate{false};
bool replace_view{false}; /// CREATE OR REPLACE VIEW
ASTColumns * columns_list = nullptr;
String to_database; /// For CREATE MATERIALIZED VIEW mv TO table.
String to_table;
@ -244,6 +245,7 @@ protected:
<< (settings.hilite ? hilite_keyword : "")
<< (attach ? "ATTACH " : "CREATE ")
<< (temporary ? "TEMPORARY " : "")
<< (replace_view ? "OR REPLACE " : "")
<< what << " "
<< (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "")

View File

@ -22,6 +22,9 @@ public:
Kind kind;
bool if_exists{false};
/// Useful if we already have a DDL lock
bool no_ddl_lock{false};
/** Get the text that identifies this element. */
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -298,6 +298,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_view("VIEW");
ParserKeyword s_materialized("MATERIALIZED");
ParserKeyword s_populate("POPULATE");
ParserKeyword s_or_replace("OR REPLACE");
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
@ -322,6 +323,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool is_materialized_view = false;
bool is_populate = false;
bool is_temporary = false;
bool replace_view = false;
if (!s_create.ignore(pos, expected))
{
@ -432,7 +434,12 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
else
{
/// VIEW or MATERIALIZED VIEW
if (s_materialized.ignore(pos, expected))
if (s_or_replace.ignore(pos, expected))
{
replace_view = true;
}
if (!replace_view && s_materialized.ignore(pos, expected))
{
is_materialized_view = true;
}
@ -442,7 +449,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!s_view.ignore(pos, expected))
return false;
if (s_if_not_exists.ignore(pos, expected))
if (!replace_view && s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
if (!name_p.parse(pos, table, expected))
@ -512,6 +519,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->is_materialized_view = is_materialized_view;
query->is_populate = is_populate;
query->temporary = is_temporary;
query->replace_view = replace_view;
getIdentifierName(database, query->database);
getIdentifierName(table, query->table);

View File

@ -285,7 +285,7 @@ protected:
* CREATE|ATTACH DATABASE db [ENGINE = engine]
*
* Or:
* CREATE|ATTACH [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
* CREATE [OR REPLACE]|ATTACH [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
*/
class ParserCreateQuery : public IParserBase
{

View File

@ -0,0 +1,2 @@
CREATE VIEW test.t ( number UInt64) AS SELECT number FROM system.numbers
CREATE VIEW test.t ( next_number UInt64) AS SELECT number + 1 FROM system.numbers

View File

@ -0,0 +1,6 @@
DROP TABLE IF EXISTS test.t;
CREATE OR REPLACE VIEW test.t (number UInt64) AS SELECT number FROM system.numbers;
SHOW CREATE TABLE test.t;
CREATE OR REPLACE VIEW test.t (next_number UInt64) AS SELECT number+1 FROM system.numbers;
SHOW CREATE TABLE test.t;