mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Analyzers: Memoizing table functions [#CLICKHOUSE-31]
This commit is contained in:
parent
d5ca92093a
commit
5548f1fdd9
@ -20,7 +20,7 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context)
|
void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
||||||
if (!select)
|
if (!select)
|
||||||
@ -35,13 +35,13 @@ void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context)
|
|||||||
collect_aliases.process(ast);
|
collect_aliases.process(ast);
|
||||||
|
|
||||||
CollectTables collect_tables;
|
CollectTables collect_tables;
|
||||||
collect_tables.process(ast, context, collect_aliases);
|
collect_tables.process(ast, context, collect_aliases, table_functions);
|
||||||
|
|
||||||
AnalyzeColumns analyze_columns;
|
AnalyzeColumns analyze_columns;
|
||||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||||
|
|
||||||
TypeAndConstantInference inference;
|
TypeAndConstantInference inference;
|
||||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas);
|
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, table_functions);
|
||||||
|
|
||||||
for (const ASTPtr & child : select->select_expression_list->children)
|
for (const ASTPtr & child : select->select_expression_list->children)
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,7 @@ namespace DB
|
|||||||
|
|
||||||
class WriteBuffer;
|
class WriteBuffer;
|
||||||
class Context;
|
class Context;
|
||||||
|
struct ExecuteTableFunctions;
|
||||||
|
|
||||||
|
|
||||||
/** For SELECT query, determine names and types of columns of result,
|
/** For SELECT query, determine names and types of columns of result,
|
||||||
@ -19,7 +20,7 @@ class Context;
|
|||||||
*/
|
*/
|
||||||
struct AnalyzeResultOfQuery
|
struct AnalyzeResultOfQuery
|
||||||
{
|
{
|
||||||
void process(ASTPtr & ast, const Context & context);
|
void process(ASTPtr & ast, const Context & context, ExecuteTableFunctions & table_functions);
|
||||||
|
|
||||||
/// Block will have non-nullptr columns for constant expressions.
|
/// Block will have non-nullptr columns for constant expressions.
|
||||||
Block result;
|
Block result;
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <Analyzers/CollectTables.h>
|
#include <Analyzers/CollectTables.h>
|
||||||
#include <Analyzers/CollectAliases.h>
|
#include <Analyzers/CollectAliases.h>
|
||||||
|
#include <Analyzers/ExecuteTableFunctions.h>
|
||||||
#include <Analyzers/AnalyzeResultOfQuery.h>
|
#include <Analyzers/AnalyzeResultOfQuery.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
@ -49,20 +50,20 @@ static CollectTables::TableInfo processOrdinaryTable(const ASTPtr & ast_database
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, const Context & context)
|
static CollectTables::TableInfo processTableFunction(const ASTPtr & ast_table_function, const ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
|
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
|
||||||
|
|
||||||
|
IAST::Hash ast_hash = ast_table_function->getTreeHash();
|
||||||
|
|
||||||
|
auto it = table_functions.tables.find(ast_hash);
|
||||||
|
if (table_functions.tables.end() == it)
|
||||||
|
throw Exception("Table function " + function.name + " was not executed in advance.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
CollectTables::TableInfo res;
|
CollectTables::TableInfo res;
|
||||||
res.node = ast_table_function;
|
res.node = ast_table_function;
|
||||||
res.alias = function.tryGetAlias();
|
res.alias = function.tryGetAlias();
|
||||||
|
res.storage = it->second;
|
||||||
/// Obtain table function
|
|
||||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(function.name, context);
|
|
||||||
/// Execute it and store result
|
|
||||||
/// TODO Avoid double execution of table functions during type inference in subqueries.
|
|
||||||
/// TODO Avoid double execution of same table functions.
|
|
||||||
res.storage = table_function_ptr->execute(ast_table_function, context);
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -78,10 +79,10 @@ static CollectTables::TableInfo processNoTables(const Context & context)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Context & context)
|
static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Context & context, ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
AnalyzeResultOfQuery analyzer;
|
AnalyzeResultOfQuery analyzer;
|
||||||
analyzer.process(typeid_cast<ASTSubquery &>(*ast_subquery).children.at(0), context);
|
analyzer.process(typeid_cast<ASTSubquery &>(*ast_subquery).children.at(0), context, table_functions);
|
||||||
|
|
||||||
CollectTables::TableInfo res;
|
CollectTables::TableInfo res;
|
||||||
res.node = ast_subquery;
|
res.node = ast_subquery;
|
||||||
@ -91,7 +92,7 @@ static CollectTables::TableInfo processSubquery(ASTPtr & ast_subquery, const Con
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & aliases)
|
void CollectTables::process(ASTPtr & ast, const Context & context, const CollectAliases & aliases, ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
||||||
if (!select)
|
if (!select)
|
||||||
@ -120,11 +121,11 @@ void CollectTables::process(ASTPtr & ast, const Context & context, const Collect
|
|||||||
}
|
}
|
||||||
else if (table_expression.table_function)
|
else if (table_expression.table_function)
|
||||||
{
|
{
|
||||||
tables.emplace_back(processTableFunction(table_expression.table_function, context));
|
tables.emplace_back(processTableFunction(table_expression.table_function, table_functions));
|
||||||
}
|
}
|
||||||
else if (table_expression.subquery)
|
else if (table_expression.subquery)
|
||||||
{
|
{
|
||||||
tables.emplace_back(processSubquery(table_expression.subquery, context));
|
tables.emplace_back(processSubquery(table_expression.subquery, context, table_functions));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
@ -10,6 +10,7 @@ namespace DB
|
|||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
struct CollectAliases;
|
struct CollectAliases;
|
||||||
|
struct ExecuteTableFunctions;
|
||||||
class WriteBuffer;
|
class WriteBuffer;
|
||||||
|
|
||||||
|
|
||||||
@ -18,13 +19,11 @@ class WriteBuffer;
|
|||||||
*
|
*
|
||||||
* For ordinary tables, determine database and table name, obtain and keep StoragePtr.
|
* For ordinary tables, determine database and table name, obtain and keep StoragePtr.
|
||||||
* For subqueries, determine result structure. This requires analysis of subquery, such as type inference.
|
* For subqueries, determine result structure. This requires analysis of subquery, such as type inference.
|
||||||
* For table functions, execute them to obtain resulting StoragePtr.
|
* For table functions, grab them from prepared ExecuteTableFunctions object.
|
||||||
*
|
|
||||||
* NOTE: We assume, that execution of table functions is cheap, as we do it during analysis.
|
|
||||||
*/
|
*/
|
||||||
struct CollectTables
|
struct CollectTables
|
||||||
{
|
{
|
||||||
void process(ASTPtr & ast, const Context & context, const CollectAliases & aliases);
|
void process(ASTPtr & ast, const Context & context, const CollectAliases & aliases, ExecuteTableFunctions & table_functions);
|
||||||
|
|
||||||
enum class Kind
|
enum class Kind
|
||||||
{
|
{
|
||||||
|
79
dbms/src/Analyzers/ExecuteTableFunctions.cpp
Normal file
79
dbms/src/Analyzers/ExecuteTableFunctions.cpp
Normal file
@ -0,0 +1,79 @@
|
|||||||
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <TableFunctions/ITableFunction.h>
|
||||||
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
|
||||||
|
#include <Analyzers/ExecuteTableFunctions.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Allows to execute exactly same table functions only once.
|
||||||
|
using ASTTreeToTable = std::map<IAST::Hash, StoragePtr>;
|
||||||
|
|
||||||
|
|
||||||
|
static void processTableFunction(const ASTPtr & ast_table_function, const Context & context, ExecuteTableFunctions::Tables & result_map)
|
||||||
|
{
|
||||||
|
const ASTFunction & function = typeid_cast<const ASTFunction &>(*ast_table_function);
|
||||||
|
|
||||||
|
/// If already executed.
|
||||||
|
IAST::Hash ast_hash = ast_table_function->getTreeHash();
|
||||||
|
if (result_map.count(ast_hash))
|
||||||
|
return;
|
||||||
|
|
||||||
|
/// Obtain table function
|
||||||
|
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(function.name, context);
|
||||||
|
/// Execute it and store result
|
||||||
|
StoragePtr table = table_function_ptr->execute(ast_table_function, context);
|
||||||
|
result_map[ast_hash] = table;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void ExecuteTableFunctions::process(ASTPtr & ast, const Context & context)
|
||||||
|
{
|
||||||
|
const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get());
|
||||||
|
if (!select)
|
||||||
|
throw Exception("ExecuteTableFunctions::process was called for not a SELECT query", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||||
|
|
||||||
|
if (!select->tables)
|
||||||
|
return;
|
||||||
|
|
||||||
|
for (auto & child : select->tables->children)
|
||||||
|
{
|
||||||
|
ASTTablesInSelectQueryElement & element = static_cast<ASTTablesInSelectQueryElement &>(*child);
|
||||||
|
|
||||||
|
if (!element.table_expression) /// This is ARRAY JOIN
|
||||||
|
continue;
|
||||||
|
|
||||||
|
ASTTableExpression & table_expression = static_cast<ASTTableExpression &>(*element.table_expression);
|
||||||
|
|
||||||
|
if (!table_expression.table_function)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
processTableFunction(table_expression.table_function, context, tables);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void ExecuteTableFunctions::dump(WriteBuffer & out) const
|
||||||
|
{
|
||||||
|
for (const auto & table : tables)
|
||||||
|
{
|
||||||
|
writeString(table.second->getName(), out);
|
||||||
|
writeCString("\n\n", out);
|
||||||
|
writeString(table.second->getColumnsList().toString(), out);
|
||||||
|
writeCString("\n", out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
35
dbms/src/Analyzers/ExecuteTableFunctions.h
Normal file
35
dbms/src/Analyzers/ExecuteTableFunctions.h
Normal file
@ -0,0 +1,35 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
#include <Storages/IStorage.h>
|
||||||
|
#include <map>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class Context;
|
||||||
|
class WriteBuffer;
|
||||||
|
|
||||||
|
|
||||||
|
/** For every table function, found in first level of the query
|
||||||
|
* (don't go into subqueries)
|
||||||
|
* execute it and save corresponding StoragePtr.
|
||||||
|
*
|
||||||
|
* Execution of table functions must be done in a stage of query analysis,
|
||||||
|
* because otherwise we don't know table structure. So, it is assumed as cheap operation.
|
||||||
|
*
|
||||||
|
* Multiple occurences of table functions with same arguments will be executed only once.
|
||||||
|
*/
|
||||||
|
struct ExecuteTableFunctions
|
||||||
|
{
|
||||||
|
void process(ASTPtr & ast, const Context & context);
|
||||||
|
|
||||||
|
using Tables = std::map<IAST::Hash, StoragePtr>;
|
||||||
|
Tables tables;
|
||||||
|
|
||||||
|
/// Debug output
|
||||||
|
void dump(WriteBuffer & out) const;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -59,7 +59,8 @@ void processImpl(
|
|||||||
ASTPtr & ast, const Context & context,
|
ASTPtr & ast, const Context & context,
|
||||||
CollectAliases & aliases, const AnalyzeColumns & columns,
|
CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||||
TypeAndConstantInference::Info & info,
|
TypeAndConstantInference::Info & info,
|
||||||
const AnalyzeLambdas & lambdas);
|
const AnalyzeLambdas & lambdas,
|
||||||
|
ExecuteTableFunctions & table_functions);
|
||||||
|
|
||||||
|
|
||||||
void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info)
|
void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info)
|
||||||
@ -77,7 +78,7 @@ void processLiteral(const String & column_name, const ASTPtr & ast, TypeAndConst
|
|||||||
|
|
||||||
void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info,
|
void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndConstantInference::Info & info,
|
||||||
const Context & context, CollectAliases & aliases, const AnalyzeColumns & columns,
|
const Context & context, CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||||
const AnalyzeLambdas & lambdas)
|
const AnalyzeLambdas & lambdas, ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
/// Column from table
|
/// Column from table
|
||||||
auto it = columns.columns.find(column_name);
|
auto it = columns.columns.find(column_name);
|
||||||
@ -112,7 +113,7 @@ void processIdentifier(const String & column_name, const ASTPtr & ast, TypeAndCo
|
|||||||
if (it->second.kind != CollectAliases::Kind::Expression)
|
if (it->second.kind != CollectAliases::Kind::Expression)
|
||||||
throw Exception("Logical error: unexpected kind of alias", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: unexpected kind of alias", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
processImpl(it->second.node, context, aliases, columns, info, lambdas);
|
processImpl(it->second.node, context, aliases, columns, info, lambdas, table_functions);
|
||||||
info[column_name] = info[it->second.node->getColumnName()];
|
info[column_name] = info[it->second.node->getColumnName()];
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -258,12 +259,12 @@ void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantIn
|
|||||||
|
|
||||||
|
|
||||||
void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info,
|
void processScalarSubquery(const String & column_name, ASTPtr & ast, TypeAndConstantInference::Info & info,
|
||||||
const Context & context)
|
const Context & context, ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
ASTSubquery * subquery = static_cast<ASTSubquery *>(ast.get());
|
ASTSubquery * subquery = static_cast<ASTSubquery *>(ast.get());
|
||||||
|
|
||||||
AnalyzeResultOfQuery analyzer;
|
AnalyzeResultOfQuery analyzer;
|
||||||
analyzer.process(subquery->children.at(0), context);
|
analyzer.process(subquery->children.at(0), context, table_functions);
|
||||||
|
|
||||||
if (!analyzer.result)
|
if (!analyzer.result)
|
||||||
throw Exception("Logical error: no columns returned from scalar subquery", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: no columns returned from scalar subquery", ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -319,7 +320,8 @@ void processHigherOrderFunction(const String & column_name,
|
|||||||
ASTPtr & ast, const Context & context,
|
ASTPtr & ast, const Context & context,
|
||||||
CollectAliases & aliases, const AnalyzeColumns & columns,
|
CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||||
TypeAndConstantInference::Info & info,
|
TypeAndConstantInference::Info & info,
|
||||||
const AnalyzeLambdas & lambdas)
|
const AnalyzeLambdas & lambdas,
|
||||||
|
ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
ASTFunction * function = static_cast<ASTFunction *>(ast.get());
|
ASTFunction * function = static_cast<ASTFunction *>(ast.get());
|
||||||
|
|
||||||
@ -383,7 +385,7 @@ void processHigherOrderFunction(const String & column_name,
|
|||||||
|
|
||||||
/// Now dive into.
|
/// Now dive into.
|
||||||
|
|
||||||
processImpl(lambda->arguments->children[1], context, aliases, columns, info, lambdas);
|
processImpl(lambda->arguments->children[1], context, aliases, columns, info, lambdas, table_functions);
|
||||||
|
|
||||||
/// Update Expression type (expression signature).
|
/// Update Expression type (expression signature).
|
||||||
|
|
||||||
@ -398,7 +400,8 @@ void processImpl(
|
|||||||
ASTPtr & ast, const Context & context,
|
ASTPtr & ast, const Context & context,
|
||||||
CollectAliases & aliases, const AnalyzeColumns & columns,
|
CollectAliases & aliases, const AnalyzeColumns & columns,
|
||||||
TypeAndConstantInference::Info & info,
|
TypeAndConstantInference::Info & info,
|
||||||
const AnalyzeLambdas & lambdas)
|
const AnalyzeLambdas & lambdas,
|
||||||
|
ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
const ASTFunction * function = typeid_cast<const ASTFunction *>(ast.get());
|
const ASTFunction * function = typeid_cast<const ASTFunction *>(ast.get());
|
||||||
|
|
||||||
@ -428,7 +431,7 @@ void processImpl(
|
|||||||
if (function && function->name == "lambda")
|
if (function && function->name == "lambda")
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
processImpl(child, context, aliases, columns, info, lambdas);
|
processImpl(child, context, aliases, columns, info, lambdas, table_functions);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -453,25 +456,28 @@ void processImpl(
|
|||||||
{
|
{
|
||||||
/// If this is higher-order function, determine types of lambda arguments and infer types of subexpressions inside lambdas.
|
/// If this is higher-order function, determine types of lambda arguments and infer types of subexpressions inside lambdas.
|
||||||
if (lambdas.higher_order_functions.end() != std::find(lambdas.higher_order_functions.begin(), lambdas.higher_order_functions.end(), ast))
|
if (lambdas.higher_order_functions.end() != std::find(lambdas.higher_order_functions.begin(), lambdas.higher_order_functions.end(), ast))
|
||||||
processHigherOrderFunction(column_name, ast, context, aliases, columns, info, lambdas);
|
processHigherOrderFunction(column_name, ast, context, aliases, columns, info, lambdas, table_functions);
|
||||||
|
|
||||||
processFunction(column_name, ast, info, context);
|
processFunction(column_name, ast, info, context);
|
||||||
}
|
}
|
||||||
else if (literal)
|
else if (literal)
|
||||||
processLiteral(column_name, ast, info);
|
processLiteral(column_name, ast, info);
|
||||||
else if (identifier)
|
else if (identifier)
|
||||||
processIdentifier(column_name, ast, info, context, aliases, columns, lambdas);
|
processIdentifier(column_name, ast, info, context, aliases, columns, lambdas, table_functions);
|
||||||
else if (subquery)
|
else if (subquery)
|
||||||
processScalarSubquery(column_name, ast, info, context);
|
processScalarSubquery(column_name, ast, info, context, table_functions);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void TypeAndConstantInference::process(ASTPtr & ast, const Context & context,
|
void TypeAndConstantInference::process(ASTPtr & ast, const Context & context,
|
||||||
CollectAliases & aliases, const AnalyzeColumns & columns, const AnalyzeLambdas & lambdas)
|
CollectAliases & aliases,
|
||||||
|
const AnalyzeColumns & columns,
|
||||||
|
const AnalyzeLambdas & lambdas,
|
||||||
|
ExecuteTableFunctions & table_functions)
|
||||||
{
|
{
|
||||||
processImpl(ast, context, aliases, columns, info, lambdas);
|
processImpl(ast, context, aliases, columns, info, lambdas, table_functions);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -14,6 +14,7 @@ class WriteBuffer;
|
|||||||
struct CollectAliases;
|
struct CollectAliases;
|
||||||
struct AnalyzeColumns;
|
struct AnalyzeColumns;
|
||||||
struct AnalyzeLambdas;
|
struct AnalyzeLambdas;
|
||||||
|
struct ExecuteTableFunctions;
|
||||||
class IFunction;
|
class IFunction;
|
||||||
class IAggregateFunction;
|
class IAggregateFunction;
|
||||||
|
|
||||||
@ -33,7 +34,8 @@ struct TypeAndConstantInference
|
|||||||
void process(ASTPtr & ast, const Context & context,
|
void process(ASTPtr & ast, const Context & context,
|
||||||
CollectAliases & aliases,
|
CollectAliases & aliases,
|
||||||
const AnalyzeColumns & columns,
|
const AnalyzeColumns & columns,
|
||||||
const AnalyzeLambdas & analyze_lambdas);
|
const AnalyzeLambdas & analyze_lambdas,
|
||||||
|
ExecuteTableFunctions & table_functions);
|
||||||
|
|
||||||
struct ExpressionInfo
|
struct ExpressionInfo
|
||||||
{
|
{
|
||||||
|
@ -8,7 +8,7 @@ add_executable(analyze_columns analyze_columns.cpp)
|
|||||||
target_link_libraries(analyze_columns dbms clickhouse_storages_system)
|
target_link_libraries(analyze_columns dbms clickhouse_storages_system)
|
||||||
|
|
||||||
add_executable(type_and_constant_inference type_and_constant_inference.cpp)
|
add_executable(type_and_constant_inference type_and_constant_inference.cpp)
|
||||||
target_link_libraries(type_and_constant_inference clickhouse_storages_system clickhouse_functions dbms)
|
target_link_libraries(type_and_constant_inference clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions dbms)
|
||||||
|
|
||||||
add_executable(analyze_result_of_query analyze_result_of_query.cpp)
|
add_executable(analyze_result_of_query analyze_result_of_query.cpp)
|
||||||
target_link_libraries(analyze_result_of_query dbms clickhouse_storages_system)
|
target_link_libraries(analyze_result_of_query dbms clickhouse_storages_system)
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <Analyzers/CollectTables.h>
|
#include <Analyzers/CollectTables.h>
|
||||||
#include <Analyzers/AnalyzeColumns.h>
|
#include <Analyzers/AnalyzeColumns.h>
|
||||||
#include <Analyzers/AnalyzeLambdas.h>
|
#include <Analyzers/AnalyzeLambdas.h>
|
||||||
|
#include <Analyzers/ExecuteTableFunctions.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/ParserSelectQuery.h>
|
#include <Parsers/ParserSelectQuery.h>
|
||||||
#include <Parsers/formatAST.h>
|
#include <Parsers/formatAST.h>
|
||||||
@ -45,8 +46,11 @@ try
|
|||||||
CollectAliases collect_aliases;
|
CollectAliases collect_aliases;
|
||||||
collect_aliases.process(ast);
|
collect_aliases.process(ast);
|
||||||
|
|
||||||
|
ExecuteTableFunctions execute_table_functions;
|
||||||
|
execute_table_functions.process(ast, context);
|
||||||
|
|
||||||
CollectTables collect_tables;
|
CollectTables collect_tables;
|
||||||
collect_tables.process(ast, context, collect_aliases);
|
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||||
|
|
||||||
AnalyzeColumns analyze_columns;
|
AnalyzeColumns analyze_columns;
|
||||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Analyzers/AnalyzeResultOfQuery.h>
|
#include <Analyzers/AnalyzeResultOfQuery.h>
|
||||||
|
#include <Analyzers/ExecuteTableFunctions.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/ParserSelectQuery.h>
|
#include <Parsers/ParserSelectQuery.h>
|
||||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||||
@ -35,8 +36,11 @@ try
|
|||||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||||
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers", false));
|
||||||
|
|
||||||
|
ExecuteTableFunctions execute_table_functions;
|
||||||
|
execute_table_functions.process(ast, context);
|
||||||
|
|
||||||
AnalyzeResultOfQuery analyzer;
|
AnalyzeResultOfQuery analyzer;
|
||||||
analyzer.process(ast, context);
|
analyzer.process(ast, context, execute_table_functions);
|
||||||
|
|
||||||
analyzer.dump(out);
|
analyzer.dump(out);
|
||||||
|
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Analyzers/CollectAliases.h>
|
#include <Analyzers/CollectAliases.h>
|
||||||
|
#include <Analyzers/ExecuteTableFunctions.h>
|
||||||
#include <Analyzers/CollectTables.h>
|
#include <Analyzers/CollectTables.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/ParserSelectQuery.h>
|
#include <Parsers/ParserSelectQuery.h>
|
||||||
@ -39,8 +40,11 @@ try
|
|||||||
CollectAliases collect_aliases;
|
CollectAliases collect_aliases;
|
||||||
collect_aliases.process(ast);
|
collect_aliases.process(ast);
|
||||||
|
|
||||||
|
ExecuteTableFunctions execute_table_functions;
|
||||||
|
execute_table_functions.process(ast, context);
|
||||||
|
|
||||||
CollectTables collect_tables;
|
CollectTables collect_tables;
|
||||||
collect_tables.process(ast, context, collect_aliases);
|
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||||
collect_tables.dump(out);
|
collect_tables.dump(out);
|
||||||
|
|
||||||
return 0;
|
return 0;
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <Analyzers/CollectTables.h>
|
#include <Analyzers/CollectTables.h>
|
||||||
#include <Analyzers/AnalyzeColumns.h>
|
#include <Analyzers/AnalyzeColumns.h>
|
||||||
#include <Analyzers/AnalyzeLambdas.h>
|
#include <Analyzers/AnalyzeLambdas.h>
|
||||||
|
#include <Analyzers/ExecuteTableFunctions.h>
|
||||||
#include <Analyzers/TypeAndConstantInference.h>
|
#include <Analyzers/TypeAndConstantInference.h>
|
||||||
#include <Analyzers/TranslatePositionalArguments.h>
|
#include <Analyzers/TranslatePositionalArguments.h>
|
||||||
#include <Analyzers/OptimizeGroupOrderLimitBy.h>
|
#include <Analyzers/OptimizeGroupOrderLimitBy.h>
|
||||||
@ -48,14 +49,17 @@ try
|
|||||||
CollectAliases collect_aliases;
|
CollectAliases collect_aliases;
|
||||||
collect_aliases.process(ast);
|
collect_aliases.process(ast);
|
||||||
|
|
||||||
|
ExecuteTableFunctions execute_table_functions;
|
||||||
|
execute_table_functions.process(ast, context);
|
||||||
|
|
||||||
CollectTables collect_tables;
|
CollectTables collect_tables;
|
||||||
collect_tables.process(ast, context, collect_aliases);
|
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||||
|
|
||||||
AnalyzeColumns analyze_columns;
|
AnalyzeColumns analyze_columns;
|
||||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||||
|
|
||||||
TypeAndConstantInference inference;
|
TypeAndConstantInference inference;
|
||||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas);
|
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions);
|
||||||
|
|
||||||
TranslatePositionalArguments translation;
|
TranslatePositionalArguments translation;
|
||||||
translation.process(ast);
|
translation.process(ast);
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <Analyzers/CollectTables.h>
|
#include <Analyzers/CollectTables.h>
|
||||||
#include <Analyzers/AnalyzeColumns.h>
|
#include <Analyzers/AnalyzeColumns.h>
|
||||||
#include <Analyzers/AnalyzeLambdas.h>
|
#include <Analyzers/AnalyzeLambdas.h>
|
||||||
|
#include <Analyzers/ExecuteTableFunctions.h>
|
||||||
#include <Analyzers/TypeAndConstantInference.h>
|
#include <Analyzers/TypeAndConstantInference.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/ParserSelectQuery.h>
|
#include <Parsers/ParserSelectQuery.h>
|
||||||
@ -15,6 +16,7 @@
|
|||||||
#include <Storages/System/StorageSystemNumbers.h>
|
#include <Storages/System/StorageSystemNumbers.h>
|
||||||
#include <Databases/DatabaseMemory.h>
|
#include <Databases/DatabaseMemory.h>
|
||||||
#include <Functions/registerFunctions.h>
|
#include <Functions/registerFunctions.h>
|
||||||
|
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||||
|
|
||||||
|
|
||||||
/// Parses query from stdin and print data types of expressions; and for constant expressions, print its values.
|
/// Parses query from stdin and print data types of expressions; and for constant expressions, print its values.
|
||||||
@ -25,6 +27,7 @@ try
|
|||||||
using namespace DB;
|
using namespace DB;
|
||||||
|
|
||||||
registerFunctions();
|
registerFunctions();
|
||||||
|
registerAggregateFunctions();
|
||||||
|
|
||||||
ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||||
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
|
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
|
||||||
@ -49,14 +52,17 @@ try
|
|||||||
CollectAliases collect_aliases;
|
CollectAliases collect_aliases;
|
||||||
collect_aliases.process(ast);
|
collect_aliases.process(ast);
|
||||||
|
|
||||||
|
ExecuteTableFunctions execute_table_functions;
|
||||||
|
execute_table_functions.process(ast, context);
|
||||||
|
|
||||||
CollectTables collect_tables;
|
CollectTables collect_tables;
|
||||||
collect_tables.process(ast, context, collect_aliases);
|
collect_tables.process(ast, context, collect_aliases, execute_table_functions);
|
||||||
|
|
||||||
AnalyzeColumns analyze_columns;
|
AnalyzeColumns analyze_columns;
|
||||||
analyze_columns.process(ast, collect_aliases, collect_tables);
|
analyze_columns.process(ast, collect_aliases, collect_tables);
|
||||||
|
|
||||||
TypeAndConstantInference inference;
|
TypeAndConstantInference inference;
|
||||||
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas);
|
inference.process(ast, context, collect_aliases, analyze_columns, analyze_lambdas, execute_table_functions);
|
||||||
|
|
||||||
inference.dump(out);
|
inference.dump(out);
|
||||||
out.next();
|
out.next();
|
||||||
|
Loading…
Reference in New Issue
Block a user