mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
more refactoring: funcs to extract database_and_table from AST
This commit is contained in:
parent
ccfb51b8dd
commit
d2518fdb3f
@ -1,6 +1,5 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
@ -172,19 +172,9 @@ ExpressionAnalyzer::ExpressionAnalyzer(
|
||||
|
||||
if (!storage && select_query)
|
||||
{
|
||||
auto select_database = select_query->database();
|
||||
auto select_table = select_query->table();
|
||||
|
||||
if (select_table
|
||||
&& !typeid_cast<const ASTSelectWithUnionQuery *>(select_table.get())
|
||||
&& !typeid_cast<const ASTFunction *>(select_table.get()))
|
||||
{
|
||||
String database = select_database
|
||||
? typeid_cast<const ASTIdentifier &>(*select_database).name
|
||||
: "";
|
||||
const String & table = typeid_cast<const ASTIdentifier &>(*select_table).name;
|
||||
storage = context.tryGetTable(database, table);
|
||||
}
|
||||
DatabaseAndTableWithAlias db_and_table;
|
||||
if (getDatabaseAndTable(*select_query, 0, db_and_table))
|
||||
storage = context.tryGetTable(db_and_table.database, db_and_table.table);
|
||||
}
|
||||
|
||||
if (storage && source_columns.empty())
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateQualified.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
@ -81,22 +82,13 @@ void forEachTable(IAST * node, F && f)
|
||||
|
||||
StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & context)
|
||||
{
|
||||
String database;
|
||||
String table;
|
||||
|
||||
const ASTIdentifier * id = static_cast<const ASTIdentifier *>(database_and_table.get());
|
||||
if (!id)
|
||||
throw Exception("Logical error: identifier expected", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (id->children.empty())
|
||||
table = id->name;
|
||||
else if (id->children.size() == 2)
|
||||
{
|
||||
database = static_cast<const ASTIdentifier *>(id->children[0].get())->name;
|
||||
table = static_cast<const ASTIdentifier *>(id->children[1].get())->name;
|
||||
}
|
||||
else
|
||||
throw Exception("Logical error: unexpected number of components in table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
std::pair<String, String> db_and_table = getDatabaseAndTableNameFromIdentifier(*id);
|
||||
|
||||
return context.tryGetTable(database, table);
|
||||
return context.tryGetTable(db_and_table.first, db_and_table.second);
|
||||
}
|
||||
|
||||
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <DataStreams/SquashingBlockOutputStream.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterSetQuery.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/evaluateQualified.h>
|
||||
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
@ -146,7 +147,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
max_streams = settings.max_threads;
|
||||
|
||||
const auto & table_expression = query.table();
|
||||
ASTPtr table_expression = getTableFunctionOrSubquery(query, 0);
|
||||
|
||||
if (input)
|
||||
{
|
||||
@ -205,7 +206,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
if (query_analyzer->isRewriteSubqueriesPredicate())
|
||||
{
|
||||
/// remake interpreter_subquery when PredicateOptimizer is rewrite subqueries and main table is subquery
|
||||
if (typeid_cast<ASTSelectWithUnionQuery *>(table_expression.get()))
|
||||
if (table_expression && typeid_cast<ASTSelectWithUnionQuery *>(table_expression.get()))
|
||||
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||
table_expression, getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1,
|
||||
only_analyze);
|
||||
@ -236,29 +237,21 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, String & table_name)
|
||||
{
|
||||
auto query_database = query.database();
|
||||
auto query_table = query.table();
|
||||
DatabaseAndTableWithAlias db_and_table;
|
||||
if (getDatabaseAndTable(query, 0, db_and_table))
|
||||
{
|
||||
table_name = db_and_table.table;
|
||||
database_name = db_and_table.database;
|
||||
|
||||
/** If the table is not specified - use the table `system.one`.
|
||||
* If the database is not specified - use the current database.
|
||||
*/
|
||||
if (query_database)
|
||||
database_name = typeid_cast<ASTIdentifier &>(*query_database).name;
|
||||
if (query_table)
|
||||
table_name = typeid_cast<ASTIdentifier &>(*query_table).name;
|
||||
|
||||
if (!query_table)
|
||||
/// If the database is not specified - use the current database.
|
||||
if (database_name.empty() && !context.tryGetTable("", table_name))
|
||||
database_name = context.getCurrentDatabase();
|
||||
}
|
||||
else /// If the table is not specified - use the table `system.one`.
|
||||
{
|
||||
database_name = "system";
|
||||
table_name = "one";
|
||||
}
|
||||
else if (!query_database)
|
||||
{
|
||||
if (context.tryGetTable("", table_name))
|
||||
database_name = "";
|
||||
else
|
||||
database_name = context.getCurrentDatabase();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -884,8 +877,12 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
/// If we need less number of columns that subquery have - update the interpreter.
|
||||
if (required_columns.size() < source_header.columns())
|
||||
{
|
||||
ASTPtr subquery = getTableFunctionOrSubquery(query, 0);
|
||||
if (!subquery)
|
||||
throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||
query.table(), getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, only_analyze);
|
||||
subquery, getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, only_analyze);
|
||||
|
||||
if (query_analyzer->hasAggregation())
|
||||
interpreter_subquery->ignoreWithTotals();
|
||||
@ -1362,11 +1359,9 @@ bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query)
|
||||
* In other cases, totals will be computed on the initiating server of the query, and it is not necessary to read the data to the end.
|
||||
*/
|
||||
|
||||
auto query_table = query.table();
|
||||
if (query_table)
|
||||
if (auto query_table = getTableFunctionOrSubquery(query, 0))
|
||||
{
|
||||
auto ast_union = typeid_cast<const ASTSelectWithUnionQuery *>(query_table.get());
|
||||
if (ast_union)
|
||||
if (auto ast_union = typeid_cast<const ASTSelectWithUnionQuery *>(query_table.get()))
|
||||
{
|
||||
for (const auto & elem : ast_union->list_of_selects->children)
|
||||
if (hasWithTotalsInAnySubqueryInFromClause(typeid_cast<const ASTSelectQuery &>(*elem)))
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Interpreters/InterpreterShowProcesslistQuery.h>
|
||||
|
||||
#include <Parsers/ASTQueryWithOutput.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Parsers/ASTShowTablesQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/InterpreterShowTablesQuery.h>
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateQualified.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTKillQueryQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
@ -51,28 +51,15 @@ static bool isUnlimitedQuery(const IAST * ast)
|
||||
if (!ast_selects->list_of_selects || ast_selects->list_of_selects->children.empty())
|
||||
return false;
|
||||
|
||||
auto ast_select = typeid_cast<ASTSelectQuery *>(ast_selects->list_of_selects->children[0].get());
|
||||
|
||||
auto ast_select = typeid_cast<const ASTSelectQuery *>(ast_selects->list_of_selects->children[0].get());
|
||||
if (!ast_select)
|
||||
return false;
|
||||
|
||||
auto ast_database = ast_select->database();
|
||||
if (!ast_database)
|
||||
DatabaseAndTableWithAlias database_and_table;
|
||||
if (!getDatabaseAndTable(*ast_select, 0, database_and_table))
|
||||
return false;
|
||||
|
||||
auto ast_table = ast_select->table();
|
||||
if (!ast_table)
|
||||
return false;
|
||||
|
||||
auto ast_database_id = typeid_cast<const ASTIdentifier *>(ast_database.get());
|
||||
if (!ast_database_id)
|
||||
return false;
|
||||
|
||||
auto ast_table_id = typeid_cast<const ASTIdentifier *>(ast_table.get());
|
||||
if (!ast_table_id)
|
||||
return false;
|
||||
|
||||
return ast_database_id->name == "system" && ast_table_id->name == "processes";
|
||||
return database_and_table.database == "system" && database_and_table.table == "processes";
|
||||
}
|
||||
|
||||
return false;
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Poco/String.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
//#include <iostream>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -65,18 +65,23 @@ void TranslateQualifiedNamesVisitor::visit(ASTQualifiedAsterisk *, ASTPtr & ast,
|
||||
if (num_components > 2)
|
||||
throw Exception("Qualified asterisk cannot have more than two qualifiers", ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
|
||||
|
||||
std::pair<String, String> db_and_table = getDatabaseAndTableNameFromIdentifier(*ident);
|
||||
|
||||
for (const auto & table_names : tables)
|
||||
{
|
||||
/// database.table.*, table.* or alias.*
|
||||
if ((num_components == 2
|
||||
&& !table_names.database.empty()
|
||||
&& static_cast<const ASTIdentifier &>(*ident->children[0]).name == table_names.database
|
||||
&& static_cast<const ASTIdentifier &>(*ident->children[1]).name == table_names.table)
|
||||
|| (num_components == 0
|
||||
&& ((!table_names.table.empty() && ident->name == table_names.table)
|
||||
|| (!table_names.alias.empty() && ident->name == table_names.alias))))
|
||||
if (num_components == 2)
|
||||
{
|
||||
return;
|
||||
if (!table_names.database.empty() &&
|
||||
db_and_table.first == table_names.database &&
|
||||
db_and_table.second == table_names.table)
|
||||
return;
|
||||
}
|
||||
else if (num_components == 0)
|
||||
{
|
||||
if ((!table_names.table.empty() && db_and_table.second == table_names.table) ||
|
||||
(!table_names.alias.empty() && db_and_table.second == table_names.alias))
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -196,4 +197,81 @@ std::vector<DatabaseAndTableWithAlias> getDatabaseAndTableWithAliases(const ASTS
|
||||
return database_and_table_with_aliases;
|
||||
}
|
||||
|
||||
|
||||
static const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, size_t table_number)
|
||||
{
|
||||
if (!select.tables)
|
||||
return {};
|
||||
|
||||
ASTTablesInSelectQuery & tables_in_select_query = static_cast<ASTTablesInSelectQuery &>(*select.tables);
|
||||
if (tables_in_select_query.children.size() <= table_number)
|
||||
return {};
|
||||
|
||||
ASTTablesInSelectQueryElement & tables_element =
|
||||
static_cast<ASTTablesInSelectQueryElement &>(*tables_in_select_query.children[table_number]);
|
||||
if (!tables_element.table_expression)
|
||||
return {};
|
||||
|
||||
return static_cast<const ASTTableExpression *>(tables_element.table_expression.get());
|
||||
}
|
||||
|
||||
bool getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number, DatabaseAndTableWithAlias & db_and_table)
|
||||
{
|
||||
const ASTTableExpression * table_expression = getTableExpression(select, table_number);
|
||||
if (!table_expression)
|
||||
return false;
|
||||
|
||||
ASTPtr database_and_table_name = table_expression->database_and_table_name;
|
||||
if (!database_and_table_name)
|
||||
return false;
|
||||
|
||||
if (database_and_table_name->children.empty())
|
||||
{
|
||||
const ASTIdentifier * db_name = typeid_cast<const ASTIdentifier *>(database_and_table_name.get());
|
||||
if (!db_name)
|
||||
throw Exception("Logical error: Unexpected database node type. Identifier expected.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
db_and_table = DatabaseAndTableWithAlias{{}, db_name->name, {}};
|
||||
return true;
|
||||
}
|
||||
else if (database_and_table_name->children.size() == 2)
|
||||
{
|
||||
const ASTIdentifier * db = typeid_cast<const ASTIdentifier *>(database_and_table_name->children[0].get());
|
||||
const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(database_and_table_name->children[1].get());
|
||||
|
||||
if (db && table)
|
||||
{
|
||||
db_and_table = DatabaseAndTableWithAlias{db->name, table->name, {}};
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
ASTPtr getTableFunctionOrSubquery(const ASTSelectQuery & select, size_t table_number)
|
||||
{
|
||||
const ASTTableExpression * table_expression = getTableExpression(select, table_number);
|
||||
if (table_expression)
|
||||
{
|
||||
#if 1 /// TODO: It hides some logical error in InterpreterSelectQuery & distributed tables
|
||||
if (table_expression->database_and_table_name)
|
||||
{
|
||||
if (table_expression->database_and_table_name->children.empty())
|
||||
return table_expression->database_and_table_name;
|
||||
|
||||
if (table_expression->database_and_table_name->children.size() == 2)
|
||||
return table_expression->database_and_table_name->children[1];
|
||||
}
|
||||
#endif
|
||||
if (table_expression->table_function)
|
||||
return table_expression->table_function;
|
||||
|
||||
if (table_expression->subquery)
|
||||
return static_cast<const ASTSubquery *>(table_expression->subquery.get())->children[0];
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -40,4 +40,7 @@ std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentif
|
||||
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery * select_query);
|
||||
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTableWithAliases(const ASTSelectQuery * select_query, const String & current_database);
|
||||
|
||||
bool getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number, DatabaseAndTableWithAlias &);
|
||||
ASTPtr getTableFunctionOrSubquery(const ASTSelectQuery & select, size_t table_number);
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -2,8 +2,6 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
@ -242,46 +242,6 @@ static const ASTTablesInSelectQueryElement * getFirstTableJoin(const ASTSelectQu
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ASTSelectQuery::database() const
|
||||
{
|
||||
const ASTTableExpression * table_expression = getFirstTableExpression(*this);
|
||||
if (!table_expression || !table_expression->database_and_table_name || table_expression->database_and_table_name->children.empty())
|
||||
return {};
|
||||
|
||||
if (table_expression->database_and_table_name->children.size() != 2)
|
||||
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return table_expression->database_and_table_name->children[0];
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ASTSelectQuery::table() const
|
||||
{
|
||||
const ASTTableExpression * table_expression = getFirstTableExpression(*this);
|
||||
if (!table_expression)
|
||||
return {};
|
||||
|
||||
if (table_expression->database_and_table_name)
|
||||
{
|
||||
if (table_expression->database_and_table_name->children.empty())
|
||||
return table_expression->database_and_table_name;
|
||||
|
||||
if (table_expression->database_and_table_name->children.size() != 2)
|
||||
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return table_expression->database_and_table_name->children[1];
|
||||
}
|
||||
|
||||
if (table_expression->table_function)
|
||||
return table_expression->table_function;
|
||||
|
||||
if (table_expression->subquery)
|
||||
return static_cast<const ASTSubquery *>(table_expression->subquery.get())->children.at(0);
|
||||
|
||||
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ASTSelectQuery::sample_size() const
|
||||
{
|
||||
const ASTTableExpression * table_expression = getFirstTableExpression(*this);
|
||||
|
@ -39,8 +39,6 @@ public:
|
||||
ASTPtr settings;
|
||||
|
||||
/// Compatibility with old parser of tables list. TODO remove
|
||||
ASTPtr database() const;
|
||||
ASTPtr table() const;
|
||||
ASTPtr sample_size() const;
|
||||
ASTPtr sample_offset() const;
|
||||
ASTPtr array_join_expression_list() const;
|
||||
|
@ -1,4 +1,3 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
|
||||
#include <Parsers/CommonParsers.h>
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#include <Parsers/ASTOptimizeQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <memory>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
|
@ -1,8 +1,6 @@
|
||||
#include <Parsers/ParserSystemQuery.h>
|
||||
#include <Parsers/ASTSystemQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ParserSelectQuery.h>
|
||||
#include <Parsers/ParserSampleRatio.h>
|
||||
|
@ -19,7 +19,6 @@
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
|
||||
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeThreadBlockInputStream.h>
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSampleRatio.h>
|
||||
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ParserAlterQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTAssignment.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
@ -16,7 +16,6 @@
|
||||
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <Parsers/ParserAlterQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
@ -2,11 +2,11 @@
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/evaluateQualified.h>
|
||||
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
@ -27,23 +27,26 @@ namespace ErrorCodes
|
||||
|
||||
static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name)
|
||||
{
|
||||
auto query_table = query.table();
|
||||
DatabaseAndTableWithAlias db_and_table;
|
||||
bool is_table = getDatabaseAndTable(query, 0, db_and_table);
|
||||
ASTPtr subquery = getTableFunctionOrSubquery(query, 0);
|
||||
|
||||
if (!query_table)
|
||||
if (!is_table && !subquery)
|
||||
return;
|
||||
|
||||
if (auto ast_id = typeid_cast<const ASTIdentifier *>(query_table.get()))
|
||||
if (is_table)
|
||||
{
|
||||
auto query_database = query.database();
|
||||
select_table_name = db_and_table.table;
|
||||
|
||||
if (!query_database)
|
||||
if (db_and_table.database.empty())
|
||||
{
|
||||
db_and_table.database = select_database_name;
|
||||
query.setDatabaseIfNeeded(select_database_name);
|
||||
|
||||
select_table_name = ast_id->name;
|
||||
select_database_name = query_database ? typeid_cast<const ASTIdentifier &>(*query_database).name : select_database_name;
|
||||
|
||||
}
|
||||
else
|
||||
select_database_name = db_and_table.database;
|
||||
}
|
||||
else if (auto ast_select = typeid_cast<ASTSelectWithUnionQuery *>(query_table.get()))
|
||||
else if (auto ast_select = typeid_cast<ASTSelectWithUnionQuery *>(subquery.get()))
|
||||
{
|
||||
if (ast_select->list_of_selects->children.size() != 1)
|
||||
throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
|
||||
@ -64,12 +67,11 @@ static void checkAllowedQueries(const ASTSelectQuery & query)
|
||||
if (query.prewhere_expression || query.final() || query.sample_size())
|
||||
throw Exception("MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL.", DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
|
||||
|
||||
auto query_table = query.table();
|
||||
|
||||
if (!query_table)
|
||||
ASTPtr subquery = getTableFunctionOrSubquery(query, 0);
|
||||
if (!subquery)
|
||||
return;
|
||||
|
||||
if (auto ast_select = typeid_cast<const ASTSelectWithUnionQuery *>(query_table.get()))
|
||||
if (auto ast_select = typeid_cast<const ASTSelectWithUnionQuery *>(subquery.get()))
|
||||
{
|
||||
if (ast_select->list_of_selects->children.size() != 1)
|
||||
throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
|
||||
|
@ -15,7 +15,6 @@
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
@ -3,7 +3,6 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
|
||||
#include <Storages/StorageMerge.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
Loading…
Reference in New Issue
Block a user