mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
extract getTableExpressions variants into own file
This commit is contained in:
parent
a180f9adf2
commit
68a33863ab
@ -1,16 +1,11 @@
|
||||
#include <Interpreters/AnalyzedJoin.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/Join.h>
|
||||
#include <Interpreters/MergeJoin.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
@ -225,31 +220,6 @@ bool AnalyzedJoin::sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y)
|
||||
&& x->columns_added_by_join == y->columns_added_by_join;
|
||||
}
|
||||
|
||||
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context)
|
||||
{
|
||||
NamesAndTypesList names_and_type_list;
|
||||
if (table_expression.subquery)
|
||||
{
|
||||
const auto & subquery = table_expression.subquery->children.at(0);
|
||||
names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context).getNamesAndTypesList();
|
||||
}
|
||||
else if (table_expression.table_function)
|
||||
{
|
||||
const auto table_function = table_expression.table_function;
|
||||
auto query_context = const_cast<Context *>(&context.getQueryContext());
|
||||
const auto & function_storage = query_context->executeTableFunction(table_function);
|
||||
names_and_type_list = function_storage->getSampleBlockNonMaterialized().getNamesAndTypesList();
|
||||
}
|
||||
else if (table_expression.database_and_table_name)
|
||||
{
|
||||
DatabaseAndTableWithAlias database_table(table_expression.database_and_table_name);
|
||||
const auto & table = context.getTable(database_table.database, database_table.table);
|
||||
names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList();
|
||||
}
|
||||
|
||||
return names_and_type_list;
|
||||
}
|
||||
|
||||
JoinPtr makeJoin(std::shared_ptr<AnalyzedJoin> table_join, const Block & right_sample_block)
|
||||
{
|
||||
bool is_left_or_inner = isLeft(table_join->kind()) || isInner(table_join->kind());
|
||||
|
@ -123,9 +123,6 @@ public:
|
||||
friend JoinPtr makeJoin(std::shared_ptr<AnalyzedJoin> table_join, const Block & right_sample_block);
|
||||
};
|
||||
|
||||
struct ASTTableExpression;
|
||||
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
|
||||
|
||||
bool isMergeJoin(const JoinPtr &);
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
@ -14,9 +16,6 @@ namespace DB
|
||||
|
||||
NameSet removeDuplicateColumns(NamesAndTypesList & columns);
|
||||
|
||||
struct ASTTableExpression;
|
||||
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
|
||||
{
|
||||
alias = identifier.tryGetAlias();
|
||||
@ -70,44 +69,9 @@ String DatabaseAndTableWithAlias::getQualifiedNamePrefix(bool with_dot) const
|
||||
return (!alias.empty() ? alias : table) + (with_dot ? "." : "");
|
||||
}
|
||||
|
||||
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery & select_query)
|
||||
{
|
||||
if (!select_query.tables())
|
||||
return {};
|
||||
|
||||
std::vector<const ASTTableExpression *> tables_expression;
|
||||
|
||||
for (const auto & child : select_query.tables()->children)
|
||||
{
|
||||
const auto * tables_element = child->as<ASTTablesInSelectQueryElement>();
|
||||
|
||||
if (tables_element->table_expression)
|
||||
tables_expression.emplace_back(tables_element->table_expression->as<ASTTableExpression>());
|
||||
}
|
||||
|
||||
return tables_expression;
|
||||
}
|
||||
|
||||
static const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, size_t table_number)
|
||||
{
|
||||
if (!select.tables())
|
||||
return {};
|
||||
|
||||
const auto & tables_in_select_query = select.tables()->as<ASTTablesInSelectQuery &>();
|
||||
if (tables_in_select_query.children.size() <= table_number)
|
||||
return {};
|
||||
|
||||
const auto & tables_element = tables_in_select_query.children[table_number]->as<ASTTablesInSelectQueryElement &>();
|
||||
|
||||
if (!tables_element.table_expression)
|
||||
return {};
|
||||
|
||||
return tables_element.table_expression->as<ASTTableExpression>();
|
||||
}
|
||||
|
||||
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database)
|
||||
{
|
||||
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(select_query);
|
||||
std::vector<const ASTTableExpression *> tables_expression = getTableExpressions(select_query);
|
||||
|
||||
std::vector<DatabaseAndTableWithAlias> database_and_table_with_aliases;
|
||||
database_and_table_with_aliases.reserve(tables_expression.size());
|
||||
@ -131,43 +95,4 @@ std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuer
|
||||
return DatabaseAndTableWithAlias(database_and_table_name);
|
||||
}
|
||||
|
||||
ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number)
|
||||
{
|
||||
if (const ASTTableExpression * table_expression = getTableExpression(select, table_number))
|
||||
{
|
||||
if (table_expression->database_and_table_name)
|
||||
return table_expression->database_and_table_name;
|
||||
|
||||
if (table_expression->table_function)
|
||||
return table_expression->table_function;
|
||||
|
||||
if (table_expression->subquery)
|
||||
return table_expression->subquery->children[0];
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
std::vector<TableWithColumnNames> getDatabaseAndTablesWithColumnNames(const ASTSelectQuery & select_query, const Context & context)
|
||||
{
|
||||
std::vector<TableWithColumnNames> tables_with_columns;
|
||||
|
||||
if (select_query.tables() && !select_query.tables()->children.empty())
|
||||
{
|
||||
String current_database = context.getCurrentDatabase();
|
||||
|
||||
for (const ASTTableExpression * table_expression : getSelectTablesExpression(select_query))
|
||||
{
|
||||
DatabaseAndTableWithAlias table_name(*table_expression, current_database);
|
||||
|
||||
NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context);
|
||||
removeDuplicateColumns(names_and_types);
|
||||
|
||||
tables_with_columns.emplace_back(std::move(table_name), names_and_types.getNames());
|
||||
}
|
||||
}
|
||||
|
||||
return tables_with_columns;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -41,9 +41,4 @@ using TableWithColumnNames = std::pair<DatabaseAndTableWithAlias, Names>;
|
||||
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database);
|
||||
std::optional<DatabaseAndTableWithAlias> getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number);
|
||||
|
||||
std::vector<TableWithColumnNames> getDatabaseAndTablesWithColumnNames(const ASTSelectQuery & select_query, const Context & context);
|
||||
|
||||
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery & select_query);
|
||||
ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number);
|
||||
|
||||
}
|
||||
|
@ -44,7 +44,7 @@
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
|
||||
#include <Interpreters/CrossToInnerJoinVisitor.h>
|
||||
#include <Interpreters/AnalyzedJoin.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Interpreters/AsteriskSemantic.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
@ -28,8 +29,6 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
}
|
||||
|
||||
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -56,7 +55,7 @@ public:
|
||||
}
|
||||
|
||||
String table_name = DatabaseAndTableWithAlias(*expr, context.getCurrentDatabase()).getQualifiedNamePrefix(false);
|
||||
NamesAndTypesList columns = getNamesAndTypeListFromTableExpression(*expr, context);
|
||||
NamesAndTypesList columns = getColumnsFromTableExpression(*expr, context);
|
||||
tables_order.push_back(table_name);
|
||||
table_columns.emplace(std::move(table_name), std::move(columns));
|
||||
}
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/FindIdentifierBestTableVisitor.h>
|
||||
#include <Interpreters/ExtractFunctionDataVisitor.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
@ -359,7 +360,7 @@ PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsO
|
||||
{
|
||||
SubqueriesProjectionColumns projection_columns;
|
||||
|
||||
for (const auto & table_expression : getSelectTablesExpression(*ast_select))
|
||||
for (const auto & table_expression : getTableExpressions(*ast_select))
|
||||
if (table_expression->subquery)
|
||||
getSubqueryProjectionColumns(table_expression->subquery, projection_columns);
|
||||
|
||||
@ -442,7 +443,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
||||
if (!select_query->tables() || select_query->tables()->children.empty())
|
||||
return {};
|
||||
|
||||
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(*select_query);
|
||||
std::vector<const ASTTableExpression *> tables_expression = getTableExpressions(*select_query);
|
||||
|
||||
if (const auto * qualified_asterisk = asterisk->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Interpreters/GetAggregatesVisitor.h>
|
||||
#include <Interpreters/AnalyzedJoin.h>
|
||||
#include <Interpreters/ExpressionActions.h> /// getSmallestColumn()
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -134,7 +135,7 @@ std::vector<TableWithColumnNames> getTablesWithColumns(const ASTSelectQuery & se
|
||||
{
|
||||
const auto & joined_expression = table_join_node->table_expression->as<ASTTableExpression &>();
|
||||
|
||||
columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
|
||||
columns_from_joined_table = getColumnsFromTableExpression(joined_expression, context);
|
||||
|
||||
joined_table.first = DatabaseAndTableWithAlias(joined_expression, context.getCurrentDatabase());
|
||||
for (const auto & column : columns_from_joined_table)
|
||||
|
109
dbms/src/Interpreters/getTableExpressions.cpp
Normal file
109
dbms/src/Interpreters/getTableExpressions.cpp
Normal file
@ -0,0 +1,109 @@
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::vector<const ASTTableExpression *> getTableExpressions(const ASTSelectQuery & select_query)
|
||||
{
|
||||
if (!select_query.tables())
|
||||
return {};
|
||||
|
||||
std::vector<const ASTTableExpression *> tables_expression;
|
||||
|
||||
for (const auto & child : select_query.tables()->children)
|
||||
{
|
||||
const auto * tables_element = child->as<ASTTablesInSelectQueryElement>();
|
||||
|
||||
if (tables_element->table_expression)
|
||||
tables_expression.emplace_back(tables_element->table_expression->as<ASTTableExpression>());
|
||||
}
|
||||
|
||||
return tables_expression;
|
||||
}
|
||||
|
||||
const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, size_t table_number)
|
||||
{
|
||||
if (!select.tables())
|
||||
return {};
|
||||
|
||||
const auto & tables_in_select_query = select.tables()->as<ASTTablesInSelectQuery &>();
|
||||
if (tables_in_select_query.children.size() <= table_number)
|
||||
return {};
|
||||
|
||||
const auto & tables_element = tables_in_select_query.children[table_number]->as<ASTTablesInSelectQueryElement &>();
|
||||
|
||||
if (!tables_element.table_expression)
|
||||
return {};
|
||||
|
||||
return tables_element.table_expression->as<ASTTableExpression>();
|
||||
}
|
||||
|
||||
ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number)
|
||||
{
|
||||
if (const ASTTableExpression * table_expression = getTableExpression(select, table_number))
|
||||
{
|
||||
if (table_expression->database_and_table_name)
|
||||
return table_expression->database_and_table_name;
|
||||
|
||||
if (table_expression->table_function)
|
||||
return table_expression->table_function;
|
||||
|
||||
if (table_expression->subquery)
|
||||
return table_expression->subquery->children[0];
|
||||
}
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression & table_expression, const Context & context)
|
||||
{
|
||||
NamesAndTypesList names_and_type_list;
|
||||
if (table_expression.subquery)
|
||||
{
|
||||
const auto & subquery = table_expression.subquery->children.at(0);
|
||||
names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context).getNamesAndTypesList();
|
||||
}
|
||||
else if (table_expression.table_function)
|
||||
{
|
||||
const auto table_function = table_expression.table_function;
|
||||
auto query_context = const_cast<Context *>(&context.getQueryContext());
|
||||
const auto & function_storage = query_context->executeTableFunction(table_function);
|
||||
names_and_type_list = function_storage->getSampleBlockNonMaterialized().getNamesAndTypesList();
|
||||
}
|
||||
else if (table_expression.database_and_table_name)
|
||||
{
|
||||
DatabaseAndTableWithAlias database_table(table_expression.database_and_table_name);
|
||||
const auto & table = context.getTable(database_table.database, database_table.table);
|
||||
names_and_type_list = table->getSampleBlockNonMaterialized().getNamesAndTypesList();
|
||||
}
|
||||
|
||||
return names_and_type_list;
|
||||
}
|
||||
|
||||
std::vector<TableWithColumnNames> getDatabaseAndTablesWithColumnNames(const ASTSelectQuery & select_query, const Context & context)
|
||||
{
|
||||
std::vector<TableWithColumnNames> tables_with_columns;
|
||||
|
||||
if (select_query.tables() && !select_query.tables()->children.empty())
|
||||
{
|
||||
String current_database = context.getCurrentDatabase();
|
||||
|
||||
for (const ASTTableExpression * table_expression : getTableExpressions(select_query))
|
||||
{
|
||||
DatabaseAndTableWithAlias table_name(*table_expression, current_database);
|
||||
|
||||
NamesAndTypesList names_and_types = getColumnsFromTableExpression(*table_expression, context);
|
||||
removeDuplicateColumns(names_and_types);
|
||||
|
||||
tables_with_columns.emplace_back(std::move(table_name), names_and_types.getNames());
|
||||
}
|
||||
}
|
||||
|
||||
return tables_with_columns;
|
||||
}
|
||||
|
||||
}
|
20
dbms/src/Interpreters/getTableExpressions.h
Normal file
20
dbms/src/Interpreters/getTableExpressions.h
Normal file
@ -0,0 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ASTTableExpression;
|
||||
class ASTSelectQuery;
|
||||
class Context;
|
||||
|
||||
std::vector<const ASTTableExpression *> getTableExpressions(const ASTSelectQuery & select_query);
|
||||
const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, size_t table_number);
|
||||
ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number);
|
||||
|
||||
NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
|
||||
std::vector<TableWithColumnNames> getDatabaseAndTablesWithColumnNames(const ASTSelectQuery & select_query, const Context & context);
|
||||
|
||||
}
|
@ -37,7 +37,7 @@ limitations under the License. */
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
|
Loading…
Reference in New Issue
Block a user