mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #18095 from spongedu/support_show_create_view
Support show create view syntax
This commit is contained in:
commit
a8f9eade19
@ -114,7 +114,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
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)
|
||||
if (!create->isView())
|
||||
create->select = nullptr;
|
||||
|
||||
create->format = nullptr;
|
||||
|
@ -641,7 +641,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
||||
if (create.as_table_function)
|
||||
return;
|
||||
|
||||
if (create.storage || create.is_view || create.is_materialized_view || create.is_live_view || create.is_dictionary)
|
||||
if (create.storage || create.is_dictionary || create.isView())
|
||||
{
|
||||
if (create.temporary && create.storage && create.storage->engine && create.storage->engine->name != "Memory")
|
||||
throw Exception(
|
||||
@ -672,7 +672,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
||||
|
||||
const String qualified_name = backQuoteIfNeed(as_database_name) + "." + backQuoteIfNeed(as_table_name);
|
||||
|
||||
if (as_create.is_view)
|
||||
if (as_create.is_ordinary_view)
|
||||
throw Exception(
|
||||
"Cannot CREATE a table AS " + qualified_name + ", it is a View",
|
||||
ErrorCodes::INCORRECT_QUERY);
|
||||
@ -826,7 +826,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
if (create.to_table_id && create.to_table_id.database_name.empty())
|
||||
create.to_table_id.database_name = current_database;
|
||||
|
||||
if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view))
|
||||
if (create.select && create.isView())
|
||||
{
|
||||
AddDefaultDatabaseVisitor visitor(current_database);
|
||||
visitor.visit(*create.select);
|
||||
@ -1009,7 +1009,7 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create)
|
||||
{
|
||||
/// If the query is a CREATE SELECT, insert the data into the table.
|
||||
if (create.select && !create.attach
|
||||
&& !create.is_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate))
|
||||
&& !create.is_ordinary_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate))
|
||||
{
|
||||
auto insert = std::make_shared<ASTInsertQuery>();
|
||||
insert->table_id = {create.database, create.table, create.uuid};
|
||||
@ -1155,7 +1155,7 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const
|
||||
{
|
||||
required_access.emplace_back(AccessType::CREATE_DICTIONARY, create.database, create.table);
|
||||
}
|
||||
else if (create.is_view || create.is_materialized_view || create.is_live_view)
|
||||
else if (create.isView())
|
||||
{
|
||||
assert(!create.temporary);
|
||||
if (create.replace_view)
|
||||
|
@ -164,6 +164,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTShowCreateViewQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTShowCreateDatabaseQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
|
@ -20,6 +20,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int THERE_IS_NO_QUERY;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
BlockIO InterpreterShowCreateQuery::execute()
|
||||
@ -43,12 +44,19 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
{
|
||||
ASTPtr create_query;
|
||||
ASTQueryWithTableAndOutput * show_query;
|
||||
if ((show_query = query_ptr->as<ASTShowCreateTableQuery>()))
|
||||
if ((show_query = query_ptr->as<ASTShowCreateTableQuery>()) ||
|
||||
(show_query = query_ptr->as<ASTShowCreateViewQuery>()))
|
||||
{
|
||||
auto resolve_table_type = show_query->temporary ? Context::ResolveExternal : Context::ResolveOrdinary;
|
||||
auto table_id = context.resolveStorageID(*show_query, resolve_table_type);
|
||||
context.checkAccess(AccessType::SHOW_COLUMNS, table_id);
|
||||
create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, context);
|
||||
if (query_ptr->as<ASTShowCreateViewQuery>())
|
||||
{
|
||||
auto & ast_create_query = create_query->as<ASTCreateQuery &>();
|
||||
if (!ast_create_query.isView())
|
||||
throw Exception(backQuote(ast_create_query.database) + "." + backQuote(ast_create_query.table) + " is not a VIEW", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
else if ((show_query = query_ptr->as<ASTShowCreateDatabaseQuery>()))
|
||||
{
|
||||
|
@ -241,7 +241,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
action = "REPLACE";
|
||||
|
||||
String what = "TABLE";
|
||||
if (is_view)
|
||||
if (is_ordinary_view)
|
||||
what = "VIEW";
|
||||
else if (is_materialized_view)
|
||||
what = "MATERIALIZED VIEW";
|
||||
|
@ -57,7 +57,7 @@ class ASTCreateQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnC
|
||||
public:
|
||||
bool attach{false}; /// Query ATTACH TABLE, not CREATE TABLE.
|
||||
bool if_not_exists{false};
|
||||
bool is_view{false};
|
||||
bool is_ordinary_view{false};
|
||||
bool is_materialized_view{false};
|
||||
bool is_live_view{false};
|
||||
bool is_populate{false};
|
||||
@ -94,6 +94,8 @@ public:
|
||||
return removeOnCluster<ASTCreateQuery>(clone(), new_database);
|
||||
}
|
||||
|
||||
bool isView() const { return is_ordinary_view || is_materialized_view || is_live_view; }
|
||||
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
@ -35,7 +35,7 @@ ASTPtr CreateViewQuery::convertToOld() const
|
||||
query->attach = attach;
|
||||
query->replace_view = replace;
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->is_view = true;
|
||||
query->is_ordinary_view = true;
|
||||
query->cluster = cluster_name;
|
||||
|
||||
if (has(SCHEMA)) query->set(query->columns_list, get(SCHEMA)->convertToOld());
|
||||
|
@ -756,7 +756,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
String cluster_str;
|
||||
bool attach = false;
|
||||
bool if_not_exists = false;
|
||||
bool is_view = false;
|
||||
bool is_ordinary_view = false;
|
||||
bool is_materialized_view = false;
|
||||
bool is_populate = false;
|
||||
bool replace_view = false;
|
||||
@ -780,7 +780,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
is_materialized_view = true;
|
||||
}
|
||||
else
|
||||
is_view = true;
|
||||
is_ordinary_view = true;
|
||||
|
||||
if (!s_view.ignore(pos, expected))
|
||||
return false;
|
||||
@ -837,7 +837,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
|
||||
query->attach = attach;
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->is_view = is_view;
|
||||
query->is_ordinary_view = is_ordinary_view;
|
||||
query->is_materialized_view = is_materialized_view;
|
||||
query->is_populate = is_populate;
|
||||
query->replace_view = replace_view;
|
||||
|
@ -21,6 +21,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_database("DATABASE");
|
||||
ParserKeyword s_table("TABLE");
|
||||
ParserKeyword s_view("VIEW");
|
||||
ParserKeyword s_dictionary("DICTIONARY");
|
||||
ParserToken s_dot(TokenType::Dot);
|
||||
ParserIdentifier name_p;
|
||||
@ -30,6 +31,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
std::shared_ptr<ASTQueryWithTableAndOutput> query;
|
||||
|
||||
bool parse_only_database_name = false;
|
||||
bool parse_show_create_view = false;
|
||||
|
||||
bool temporary = false;
|
||||
if (s_exists.ignore(pos, expected))
|
||||
@ -64,6 +66,11 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
}
|
||||
else if (s_dictionary.checkWithoutMoving(pos, expected))
|
||||
query = std::make_shared<ASTShowCreateDictionaryQuery>();
|
||||
else if (s_view.ignore(pos, expected))
|
||||
{
|
||||
query = std::make_shared<ASTShowCreateViewQuery>();
|
||||
parse_show_create_view = true;
|
||||
}
|
||||
else
|
||||
query = std::make_shared<ASTShowCreateTableQuery>();
|
||||
}
|
||||
@ -79,15 +86,16 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
}
|
||||
else
|
||||
{
|
||||
if (temporary || s_temporary.ignore(pos, expected))
|
||||
query->temporary = true;
|
||||
|
||||
if (!s_table.ignore(pos, expected))
|
||||
s_dictionary.ignore(pos, expected);
|
||||
if (!parse_show_create_view)
|
||||
{
|
||||
if (temporary || s_temporary.ignore(pos, expected))
|
||||
query->temporary = true;
|
||||
|
||||
if (!s_table.ignore(pos, expected))
|
||||
s_dictionary.ignore(pos, expected);
|
||||
}
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
database = table;
|
||||
|
@ -7,7 +7,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Query (EXISTS | SHOW CREATE) [TABLE|DICTIONARY] [db.]name [FORMAT format]
|
||||
/** Query (EXISTS | SHOW CREATE) [DATABASE|TABLE|DICTIONARY] [db.]name [FORMAT format]
|
||||
*/
|
||||
class ParserTablePropertiesQuery : public IParserBase
|
||||
{
|
||||
|
@ -37,6 +37,14 @@ struct ASTShowCreateTableQueryIDAndQueryNames
|
||||
static constexpr auto QueryTemporary = "SHOW CREATE TEMPORARY TABLE";
|
||||
};
|
||||
|
||||
struct ASTShowCreateViewQueryIDAndQueryNames
|
||||
{
|
||||
static constexpr auto ID = "ShowCreateViewQuery";
|
||||
static constexpr auto Query = "SHOW CREATE VIEW";
|
||||
/// No temporary view are supported, just for parsing
|
||||
static constexpr auto QueryTemporary = "";
|
||||
};
|
||||
|
||||
struct ASTShowCreateDatabaseQueryIDAndQueryNames
|
||||
{
|
||||
static constexpr auto ID = "ShowCreateDatabaseQuery";
|
||||
@ -63,6 +71,7 @@ using ASTExistsDatabaseQuery = ASTQueryWithTableAndOutputImpl<ASTExistsDatabaseQ
|
||||
using ASTExistsTableQuery = ASTQueryWithTableAndOutputImpl<ASTExistsTableQueryIDAndQueryNames>;
|
||||
using ASTExistsDictionaryQuery = ASTQueryWithTableAndOutputImpl<ASTExistsDictionaryQueryIDAndQueryNames>;
|
||||
using ASTShowCreateTableQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateTableQueryIDAndQueryNames>;
|
||||
using ASTShowCreateViewQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateViewQueryIDAndQueryNames>;
|
||||
using ASTShowCreateDictionaryQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateDictionaryQueryIDAndQueryNames>;
|
||||
|
||||
class ASTShowCreateDatabaseQuery : public ASTQueryWithTableAndOutputImpl<ASTShowCreateDatabaseQueryIDAndQueryNames>
|
||||
|
@ -53,7 +53,7 @@ StoragePtr StorageFactory::get(
|
||||
|
||||
bool has_engine_args = false;
|
||||
|
||||
if (query.is_view)
|
||||
if (query.is_ordinary_view)
|
||||
{
|
||||
if (query.storage)
|
||||
throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY);
|
||||
|
@ -59,7 +59,7 @@ SHOW CREATE distributed;
|
||||
SHOW CREATE distributed_tf;
|
||||
SHOW CREATE url;
|
||||
SHOW CREATE rich_syntax;
|
||||
SHOW CREATE view;
|
||||
SHOW CREATE VIEW view;
|
||||
SHOW CREATE dict;
|
||||
|
||||
INSERT INTO buffer VALUES (1);
|
||||
|
@ -0,0 +1,7 @@
|
||||
CREATE VIEW test_1602.v\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||
CREATE MATERIALIZED VIEW test_1602.vv\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n)\nENGINE = MergeTree\nPARTITION BY toYYYYMM(EventDate)\nORDER BY (CounterID, EventDate, intHash32(UserID))\nSETTINGS index_granularity = 8192 AS\nSELECT *\nFROM test_1602.tbl
|
||||
CREATE LIVE VIEW test_1602.vvv\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||
CREATE VIEW test_1602.VIEW\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||
CREATE VIEW test_1602.DATABASE\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||
CREATE VIEW test_1602.DICTIONARY\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||
CREATE VIEW test_1602.TABLE\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
51
tests/queries/0_stateless/01602_show_create_view.sql
Normal file
51
tests/queries/0_stateless/01602_show_create_view.sql
Normal file
@ -0,0 +1,51 @@
|
||||
DROP DATABASE IF EXISTS test_1602;
|
||||
|
||||
CREATE DATABASE test_1602;
|
||||
|
||||
CREATE TABLE test_1602.tbl (`EventDate` DateTime, `CounterID` UInt32, `UserID` UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SETTINGS index_granularity = 8192;
|
||||
|
||||
CREATE VIEW test_1602.v AS SELECT * FROM test_1602.tbl;
|
||||
|
||||
CREATE VIEW test_1602.DATABASE AS SELECT * FROM test_1602.tbl;
|
||||
|
||||
CREATE VIEW test_1602.DICTIONARY AS SELECT * FROM test_1602.tbl;
|
||||
|
||||
CREATE VIEW test_1602.TABLE AS SELECT * FROM test_1602.tbl;
|
||||
|
||||
CREATE MATERIALIZED VIEW test_1602.vv (`EventDate` DateTime, `CounterID` UInt32, `UserID` UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SETTINGS index_granularity = 8192 AS SELECT * FROM test_1602.tbl;
|
||||
|
||||
CREATE VIEW test_1602.VIEW AS SELECT * FROM test_1602.tbl;
|
||||
|
||||
SET allow_experimental_live_view=1;
|
||||
|
||||
CREATE LIVE VIEW test_1602.vvv AS SELECT * FROM test_1602.tbl;
|
||||
|
||||
SHOW CREATE VIEW test_1602.v;
|
||||
|
||||
SHOW CREATE VIEW test_1602.vv;
|
||||
|
||||
SHOW CREATE VIEW test_1602.vvv;
|
||||
|
||||
SHOW CREATE VIEW test_1602.not_exist_view; -- { serverError 390 }
|
||||
|
||||
SHOW CREATE VIEW test_1602.tbl; -- { serverError 36 }
|
||||
|
||||
SHOW CREATE TEMPORARY VIEW; -- { serverError 60 }
|
||||
|
||||
SHOW CREATE VIEW; -- { clientError 62 }
|
||||
|
||||
SHOW CREATE DATABASE; -- { clientError 62 }
|
||||
|
||||
SHOW CREATE DICTIONARY; -- { clientError 62 }
|
||||
|
||||
SHOW CREATE TABLE; -- { clientError 62 }
|
||||
|
||||
SHOW CREATE test_1602.VIEW;
|
||||
|
||||
SHOW CREATE test_1602.DATABASE;
|
||||
|
||||
SHOW CREATE test_1602.DICTIONARY;
|
||||
|
||||
SHOW CREATE test_1602.TABLE;
|
||||
|
||||
DROP DATABASE IF EXISTS test_1602;
|
Loading…
Reference in New Issue
Block a user