more refactoring: funcs to extract database_and_table from AST

This commit is contained in:
chertus 2018-10-29 22:04:28 +03:00
parent ccfb51b8dd
commit d2518fdb3f
35 changed files with 146 additions and 159 deletions

View File

@ -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>

View File

@ -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())

View File

@ -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);
}

View File

@ -10,7 +10,6 @@
#include <DataStreams/SquashingBlockOutputStream.h>
#include <DataStreams/copyData.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>

View File

@ -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)))

View File

@ -5,7 +5,6 @@
#include <Interpreters/InterpreterShowProcesslistQuery.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTIdentifier.h>
namespace DB

View File

@ -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>

View File

@ -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;

View File

@ -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

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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);
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
namespace DB

View File

@ -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>

View File

@ -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);

View File

@ -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;

View File

@ -1,4 +1,3 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/CommonParsers.h>

View File

@ -1,5 +1,4 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTInsertQuery.h>

View File

@ -4,7 +4,6 @@
#include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Common/typeid_cast.h>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -10,6 +10,7 @@
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/Set.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTLiteral.h>
namespace DB

View File

@ -10,7 +10,6 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Storages/SelectQueryInfo.h>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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);

View File

@ -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>

View File

@ -3,7 +3,6 @@
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <IO/ReadWriteBufferFromHTTP.h>

View File

@ -11,7 +11,6 @@
#include <Storages/VirtualColumnUtils.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Databases/IDatabase.h>

View File

@ -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>