support operations with views in antlr parser

This commit is contained in:
Anton Popov 2021-01-14 14:54:03 +03:00
parent 11f459ad03
commit 9c9e2a5936
13 changed files with 2577 additions and 2033 deletions

View File

@ -10,7 +10,6 @@
#include <Access/AccessFlags.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -40,6 +39,7 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
{
ASTQueryWithTableAndOutput * exists_query;
bool result = false;
if ((exists_query = query_ptr->as<ASTExistsTableQuery>()))
{
if (exists_query->temporary)
@ -57,8 +57,8 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
{
String database = context.resolveDatabase(exists_query->database);
context.checkAccess(AccessType::SHOW_TABLES, database, exists_query->table);
auto tbl = DatabaseCatalog::instance().tryGetTable({database, exists_query->table}, context);
result = tbl != nullptr && tbl->isView();
auto table = DatabaseCatalog::instance().tryGetTable({database, exists_query->table}, context);
result = table && table->isView();
}
else if ((exists_query = query_ptr->as<ASTExistsDatabaseQuery>()))
{

View File

@ -41,6 +41,16 @@ DropQuery::createDropTable(bool detach, bool if_exists, bool temporary, PtrTo<Ta
return query;
}
// static
PtrTo<DropQuery>
DropQuery::createDropView(bool detach, bool if_exists, PtrTo<TableIdentifier> identifier, PtrTo<ClusterClause> cluster)
{
auto query = PtrTo<DropQuery>(new DropQuery(cluster, QueryType::VIEW, {identifier}));
query->detach = detach;
query->if_exists = if_exists;
return query;
}
DropQuery::DropQuery(PtrTo<ClusterClause> cluster, QueryType type, PtrList exprs) : DDLQuery(cluster, exprs), query_type(type)
{
}
@ -73,6 +83,14 @@ ASTPtr DropQuery::convertToOld() const
query->database = database->getName();
break;
}
case QueryType::VIEW:
{
query->is_view = true;
query->table = get<TableIdentifier>(NAME)->getName();
if (auto database = get<TableIdentifier>(NAME)->getDatabase())
query->database = database->getName();
break;
}
}
convertToOldPartially(query);
@ -100,6 +118,8 @@ antlrcpp::Any ParseTreeVisitor::visitDropTableStmt(ClickHouseParser::DropTableSt
return DropQuery::createDropTable(!!ctx->DETACH(), !!ctx->EXISTS(), !!ctx->TEMPORARY(), visit(ctx->tableIdentifier()), cluster);
if (ctx->DICTIONARY())
return DropQuery::createDropDictionary(!!ctx->DETACH(), !!ctx->EXISTS(), visit(ctx->tableIdentifier()), cluster);
if (ctx->VIEW())
return DropQuery::createDropView(!!ctx->DETACH(), !!ctx->EXISTS(), visit(ctx->tableIdentifier()), cluster);
__builtin_unreachable();
}

View File

@ -15,6 +15,8 @@ class DropQuery : public DDLQuery
createDropTable(bool detach, bool if_exists, bool temporary, PtrTo<TableIdentifier> identifier, PtrTo<ClusterClause> cluster);
static PtrTo<DropQuery>
createDropDictionary(bool detach, bool if_exists, PtrTo<TableIdentifier> identifier, PtrTo<ClusterClause> cluster);
static PtrTo<DropQuery>
createDropView(bool detach, bool if_exists, PtrTo<TableIdentifier> identifier, PtrTo<ClusterClause> cluster);
ASTPtr convertToOld() const override;
@ -29,6 +31,7 @@ class DropQuery : public DDLQuery
DATABASE,
DICTIONARY,
TABLE,
VIEW,
};
const QueryType query_type;

View File

@ -10,31 +10,51 @@
namespace DB::AST
{
ExistsQuery::ExistsQuery(QueryType type, bool temporary_, PtrTo<TableIdentifier> identifier)
: Query{identifier}, query_type(type), temporary(temporary_)
ExistsQuery::ExistsQuery(QueryType type, bool temporary_, PtrList exprs)
: Query(exprs), query_type(type), temporary(temporary_)
{
}
// static
PtrTo<ExistsQuery> ExistsQuery::createTable(QueryType type, bool temporary, PtrTo<TableIdentifier> identifier)
{
return PtrTo<ExistsQuery>(new ExistsQuery(type, temporary, {identifier}));
}
// static
PtrTo<ExistsQuery> ExistsQuery::createDatabase(PtrTo<DatabaseIdentifier> identifier)
{
return PtrTo<ExistsQuery>(new ExistsQuery(QueryType::DATABASE, false, {identifier}));
}
ASTPtr ExistsQuery::convertToOld() const
{
std::shared_ptr<ASTQueryWithTableAndOutput> query;
switch(query_type)
{
case QueryType::DATABASE:
query = std::make_shared<ASTExistsDatabaseQuery>();
tryGetIdentifierNameInto(get<DatabaseIdentifier>(IDENTIFIER)->convertToOld(), query->database);
return query;
case QueryType::DICTIONARY:
query = std::make_shared<ASTExistsDictionaryQuery>();
break;
case QueryType::TABLE:
query = std::make_shared<ASTExistsTableQuery>();
query->temporary = temporary;
break;
case QueryType::VIEW:
query = std::make_shared<ASTExistsViewQuery>();
break;
}
// FIXME: this won't work if table doesn't exist
auto table_id = getTableIdentifier(get(TABLE)->convertToOld());
auto table_id = getTableIdentifier(get<TableIdentifier>(IDENTIFIER)->convertToOld());
query->database = table_id.database_name;
query->table = table_id.table_name;
query->uuid = table_id.uuid;
query->temporary = temporary;
return query;
}
@ -46,10 +66,22 @@ namespace DB
using namespace AST;
antlrcpp::Any ParseTreeVisitor::visitExistsStmt(ClickHouseParser::ExistsStmtContext *ctx)
antlrcpp::Any ParseTreeVisitor::visitExistsTableStmt(ClickHouseParser::ExistsTableStmtContext *ctx)
{
auto type = ctx->TABLE() ? ExistsQuery::QueryType::TABLE : ExistsQuery::QueryType::DICTIONARY;
return std::make_shared<ExistsQuery>(type, !!ctx->TEMPORARY(), visit(ctx->tableIdentifier()));
ExistsQuery::QueryType type;
if (ctx->DICTIONARY())
type = ExistsQuery::QueryType::DICTIONARY;
else if (ctx->VIEW())
type = ExistsQuery::QueryType::VIEW;
else // Query 'EXISTS <table_name>' is interptered as 'EXISTS TABLE <table_name>'
type = ExistsQuery::QueryType::TABLE;
return ExistsQuery::createTable(type, !!ctx->TEMPORARY(), visit(ctx->tableIdentifier()));
}
antlrcpp::Any ParseTreeVisitor::visitExistsDatabaseStmt(ClickHouseParser::ExistsDatabaseStmtContext *ctx)
{
return ExistsQuery::createDatabase(visit(ctx->databaseIdentifier()));
}
}

View File

@ -13,16 +13,21 @@ class ExistsQuery : public Query
{
DICTIONARY,
TABLE,
VIEW,
DATABASE,
};
ExistsQuery(QueryType type, bool temporary, PtrTo<TableIdentifier> identifier);
static PtrTo<ExistsQuery> createTable(QueryType type, bool temporary, PtrTo<TableIdentifier> identifier);
static PtrTo<ExistsQuery> createDatabase(PtrTo<DatabaseIdentifier> identifier);
ExistsQuery(QueryType type, bool temporary, PtrList exprs);
ASTPtr convertToOld() const override;
private:
enum ChildIndex : UInt8
{
TABLE = 0, // TableIdentifier
IDENTIFIER = 0, // DatabaseIdentifier or TableIdentifier
};
const QueryType query_type;

File diff suppressed because it is too large Load Diff

View File

@ -173,12 +173,15 @@ describeStmt: (DESCRIBE | DESC) TABLE? tableExpr;
dropStmt
: (DETACH | DROP) DATABASE (IF EXISTS)? databaseIdentifier clusterClause? # DropDatabaseStmt
| (DETACH | DROP) (DICTIONARY | TEMPORARY? TABLE) (IF EXISTS)? tableIdentifier clusterClause? (NO DELAY)? # DropTableStmt
| (DETACH | DROP) (DICTIONARY | TEMPORARY? TABLE | VIEW) (IF EXISTS)? tableIdentifier clusterClause? (NO DELAY)? # DropTableStmt
;
// EXISTS statement
existsStmt: EXISTS (DICTIONARY | TEMPORARY? TABLE)? tableIdentifier;
existsStmt
: EXISTS DATABASE databaseIdentifier # ExistsDatabaseStmt
| EXISTS (DICTIONARY | TEMPORARY? TABLE | VIEW)? tableIdentifier # ExistsTableStmt
;
// EXPLAIN statement

File diff suppressed because it is too large Load Diff

View File

@ -164,7 +164,9 @@ public:
virtual antlrcpp::Any visitDropTableStmt(ClickHouseParser::DropTableStmtContext *context) = 0;
virtual antlrcpp::Any visitExistsStmt(ClickHouseParser::ExistsStmtContext *context) = 0;
virtual antlrcpp::Any visitExistsDatabaseStmt(ClickHouseParser::ExistsDatabaseStmtContext *context) = 0;
virtual antlrcpp::Any visitExistsTableStmt(ClickHouseParser::ExistsTableStmtContext *context) = 0;
virtual antlrcpp::Any visitExplainStmt(ClickHouseParser::ExplainStmtContext *context) = 0;

View File

@ -136,7 +136,8 @@ public:
antlrcpp::Any visitTtlExpr(ClickHouseParser::TtlExprContext * ctx) override;
// ExistsQuery
antlrcpp::Any visitExistsStmt(ClickHouseParser::ExistsStmtContext * ctx) override;
antlrcpp::Any visitExistsTableStmt(ClickHouseParser::ExistsTableStmtContext * ctx) override;
antlrcpp::Any visitExistsDatabaseStmt(ClickHouseParser::ExistsDatabaseStmtContext * ctx) override;
// ExplainQuery
antlrcpp::Any visitExplainStmt(ClickHouseParser::ExplainStmtContext * ctx) override;

View File

@ -76,7 +76,6 @@ struct ASTDescribeQueryExistsQueryIDAndQueryNames
static constexpr auto QueryTemporary = "DESCRIBE TEMPORARY TABLE";
};
using ASTExistsDatabaseQuery = ASTQueryWithTableAndOutputImpl<ASTExistsDatabaseQueryIDAndQueryNames>;
using ASTExistsTableQuery = ASTQueryWithTableAndOutputImpl<ASTExistsTableQueryIDAndQueryNames>;
using ASTExistsViewQuery = ASTQueryWithTableAndOutputImpl<ASTExistsViewQueryIDAndQueryNames>;
using ASTExistsDictionaryQuery = ASTQueryWithTableAndOutputImpl<ASTExistsDictionaryQueryIDAndQueryNames>;
@ -84,6 +83,16 @@ using ASTShowCreateTableQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateTabl
using ASTShowCreateViewQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateViewQueryIDAndQueryNames>;
using ASTShowCreateDictionaryQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateDictionaryQueryIDAndQueryNames>;
class ASTExistsDatabaseQuery : public ASTQueryWithTableAndOutputImpl<ASTExistsDatabaseQueryIDAndQueryNames>
{
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << ASTExistsDatabaseQueryIDAndQueryNames::Query
<< " " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database);
}
};
class ASTShowCreateDatabaseQuery : public ASTQueryWithTableAndOutputImpl<ASTShowCreateDatabaseQueryIDAndQueryNames>
{
protected:

View File

@ -28,7 +28,7 @@ EXISTS t_01048; -- Does not work for temporary tables. Maybe have to fix.
EXISTS TABLE t_01048;
EXISTS DICTIONARY t_01048;
CREATE DICTIONARY db_01048.t_01048 (k UInt64, v String) PRIMARY KEY k LAYOUT(FLAT()) SOURCE(HTTP(URL 'http://example.test/' FORMAT TSV)) LIFETIME(1000);
CREATE DICTIONARY db_01048.t_01048 (k UInt64, v String) PRIMARY KEY k LAYOUT(FLAT()) SOURCE(HTTP(URL 'http://example.test/' FORMAT 'TSV')) LIFETIME(1000);
EXISTS db_01048.t_01048;
EXISTS TABLE db_01048.t_01048; -- Dictionaries are tables as well. But not all tables are dictionaries.
EXISTS DICTIONARY db_01048.t_01048;

View File

@ -162,7 +162,6 @@
"01039_test_setting_parse",
"01042_system_reload_dictionary_reloads_completely",
"01045_dictionaries_restrictions",
"01048_exists_query",
"01055_compact_parts_1",
"01056_create_table_as",
"01066_bit_count",
@ -197,7 +196,6 @@
"01190_full_attach_syntax",
"01191_rename_dictionary",
"01192_rename_database_zookeeper",
"01210_drop_view",
"01213_alter_rename_column",
"01232_untuple",
"01244_optimize_distributed_group_by_sharding_key",
@ -209,7 +207,6 @@
"01269_create_with_null",
"01271_show_privileges",
"01272_offset_without_limit",
"01275_parallel_mv",
"01277_alter_rename_column_constraint_zookeeper",
"01278_min_insert_block_size_rows_for_materialized_views",
"01280_min_map_max_map",