mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
extract more logic out of QueryNormalizer
This commit is contained in:
parent
04a6c6ac4d
commit
20093fa065
@ -29,7 +29,7 @@
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/Set.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Interpreters/CrossToInnerJoinVisitor.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/QueryNormalizer.h> // for functionIsInOperator
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
|
@ -54,7 +54,7 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
|
||||
|
47
dbms/src/Interpreters/MarkTableIdentifiersVisitor.cpp
Normal file
47
dbms/src/Interpreters/MarkTableIdentifiersVisitor.cpp
Normal file
@ -0,0 +1,47 @@
|
||||
#include <Poco/String.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/MarkTableIdentifiersVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool MarkTableIdentifiersMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
|
||||
{
|
||||
if (child->as<ASTSelectQuery>())
|
||||
return false;
|
||||
if (node->as<ASTTableExpression>())
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
void MarkTableIdentifiersMatcher::visit(ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto * node_func = ast->as<ASTFunction>())
|
||||
visit(*node_func, ast, data);
|
||||
else if (auto * node_table = ast->as<ASTTableExpression>())
|
||||
visit(*node_table, ast, data);
|
||||
}
|
||||
|
||||
void MarkTableIdentifiersMatcher::visit(ASTTableExpression & table, ASTPtr &, Data &)
|
||||
{
|
||||
if (table.database_and_table_name)
|
||||
setIdentifierSpecial(table.database_and_table_name);
|
||||
}
|
||||
|
||||
void MarkTableIdentifiersMatcher::visit(const ASTFunction & func, ASTPtr &, Data & data)
|
||||
{
|
||||
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
|
||||
if (functionIsInOrGlobalInOperator(func.name))
|
||||
{
|
||||
auto & ast = func.arguments->children.at(1);
|
||||
if (auto opt_name = tryGetIdentifierName(ast))
|
||||
if (!data.aliases.count(*opt_name))
|
||||
setIdentifierSpecial(ast);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
33
dbms/src/Interpreters/MarkTableIdentifiersVisitor.h
Normal file
33
dbms/src/Interpreters/MarkTableIdentifiersVisitor.h
Normal file
@ -0,0 +1,33 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTFunction;
|
||||
struct ASTTableExpression;
|
||||
|
||||
class MarkTableIdentifiersMatcher
|
||||
{
|
||||
public:
|
||||
using Visitor = InDepthNodeVisitor<MarkTableIdentifiersMatcher, true>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
const Aliases & aliases;
|
||||
};
|
||||
|
||||
static bool needChildVisit(ASTPtr & node, const ASTPtr & child);
|
||||
static void visit(ASTPtr & ast, Data & data);
|
||||
|
||||
private:
|
||||
static void visit(ASTTableExpression & table, ASTPtr &, Data &);
|
||||
static void visit(const ASTFunction & func, ASTPtr &, Data &);
|
||||
};
|
||||
|
||||
using MarkTableIdentifiersVisitor = MarkTableIdentifiersMatcher::Visitor;
|
||||
|
||||
}
|
@ -21,6 +21,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/QueryAliasesVisitor.h>
|
||||
#include <Interpreters/MarkTableIdentifiersVisitor.h>
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/FindIdentifierBestTableVisitor.h>
|
||||
#include <Interpreters/ExtractFunctionDataVisitor.h>
|
||||
@ -412,6 +413,9 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
|
||||
QueryAliasesVisitor::Data query_aliases_data{aliases};
|
||||
QueryAliasesVisitor(query_aliases_data).visit(ast);
|
||||
|
||||
MarkTableIdentifiersVisitor::Data mark_tables_data{aliases};
|
||||
MarkTableIdentifiersVisitor(mark_tables_data).visit(ast);
|
||||
|
||||
QueryNormalizer::Data normalizer_data(aliases, settings);
|
||||
QueryNormalizer(normalizer_data).visit(ast);
|
||||
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
namespace DB
|
||||
@ -63,34 +62,6 @@ private:
|
||||
};
|
||||
|
||||
|
||||
void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
auto & aliases = data.aliases;
|
||||
String & func_name = node.name;
|
||||
ASTPtr & func_arguments = node.arguments;
|
||||
|
||||
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
|
||||
if (functionIsInOrGlobalInOperator(func_name))
|
||||
{
|
||||
auto & ast = func_arguments->children.at(1);
|
||||
if (auto opt_name = tryGetIdentifierName(ast))
|
||||
if (!aliases.count(*opt_name))
|
||||
setIdentifierSpecial(ast);
|
||||
}
|
||||
|
||||
/// Special cases for count function.
|
||||
String func_name_lowercase = Poco::toLower(func_name);
|
||||
if (startsWith(func_name_lowercase, "count"))
|
||||
{
|
||||
/// Select implementation of countDistinct based on settings.
|
||||
/// Important that it is done as query rewrite. It means rewritten query
|
||||
/// will be sent to remote servers during distributed query execution,
|
||||
/// and on all remote servers, function implementation will be same.
|
||||
if (endsWith(func_name, "Distinct") && func_name_lowercase == "countdistinct")
|
||||
func_name = data.settings.count_distinct_implementation;
|
||||
}
|
||||
}
|
||||
|
||||
void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
|
||||
{
|
||||
auto & current_asts = data.current_asts;
|
||||
@ -144,16 +115,8 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data)
|
||||
}
|
||||
}
|
||||
|
||||
/// mark table identifiers as 'not columns'
|
||||
void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
/// mark table Identifiers as 'not a column'
|
||||
if (node.table_expression)
|
||||
{
|
||||
auto & expr = node.table_expression->as<ASTTableExpression &>();
|
||||
setIdentifierSpecial(expr.database_and_table_name);
|
||||
}
|
||||
|
||||
/// normalize JOIN ON section
|
||||
if (node.table_join)
|
||||
{
|
||||
@ -177,7 +140,6 @@ void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data
|
||||
if (needVisitChild(child))
|
||||
visit(child, data);
|
||||
|
||||
#if 1 /// TODO: legacy?
|
||||
/// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children,
|
||||
/// but also in where_expression and having_expression.
|
||||
if (select.prewhere())
|
||||
@ -186,7 +148,6 @@ void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data
|
||||
visit(select.refWhere(), data);
|
||||
if (select.having())
|
||||
visit(select.refHaving(), data);
|
||||
#endif
|
||||
}
|
||||
|
||||
/// Don't go into subqueries.
|
||||
@ -243,9 +204,7 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data)
|
||||
data.current_alias = my_alias;
|
||||
}
|
||||
|
||||
if (auto * node_func = ast->as<ASTFunction>())
|
||||
visit(*node_func, ast, data);
|
||||
else if (auto * node_id = ast->as<ASTIdentifier>())
|
||||
if (auto * node_id = ast->as<ASTIdentifier>())
|
||||
visit(*node_id, ast, data);
|
||||
else if (auto * node_tables = ast->as<ASTTablesInSelectQueryElement>())
|
||||
visit(*node_tables, ast, data);
|
||||
|
@ -2,25 +2,13 @@
|
||||
|
||||
#include <map>
|
||||
|
||||
#include <Core/Names.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Interpreters/Aliases.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
inline bool functionIsInOperator(const String & name)
|
||||
{
|
||||
return name == "in" || name == "notIn";
|
||||
}
|
||||
|
||||
inline bool functionIsInOrGlobalInOperator(const String & name)
|
||||
{
|
||||
return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn";
|
||||
}
|
||||
|
||||
class ASTSelectQuery;
|
||||
class ASTFunction;
|
||||
class ASTIdentifier;
|
||||
struct ASTTablesInSelectQueryElement;
|
||||
class Context;
|
||||
@ -33,13 +21,11 @@ class QueryNormalizer
|
||||
{
|
||||
const UInt64 max_ast_depth;
|
||||
const UInt64 max_expanded_ast_elements;
|
||||
const String count_distinct_implementation;
|
||||
|
||||
template <typename T>
|
||||
ExtractedSettings(const T & settings)
|
||||
: max_ast_depth(settings.max_ast_depth),
|
||||
max_expanded_ast_elements(settings.max_expanded_ast_elements),
|
||||
count_distinct_implementation(settings.count_distinct_implementation)
|
||||
max_expanded_ast_elements(settings.max_expanded_ast_elements)
|
||||
{}
|
||||
};
|
||||
|
||||
@ -80,7 +66,6 @@ private:
|
||||
static void visit(ASTPtr & query, Data & data);
|
||||
|
||||
static void visit(ASTIdentifier &, ASTPtr &, Data &);
|
||||
static void visit(ASTFunction &, const ASTPtr &, Data &);
|
||||
static void visit(ASTTablesInSelectQueryElement &, const ASTPtr &, Data &);
|
||||
static void visit(ASTSelectQuery &, const ASTPtr &, Data &);
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/ArrayJoinedColumnsVisitor.h>
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/MarkTableIdentifiersVisitor.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
|
||||
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||
@ -72,6 +73,26 @@ namespace
|
||||
|
||||
using LogAST = DebugASTLog<false>; /// set to true to enable logs
|
||||
|
||||
/// Select implementation of countDistinct based on settings.
|
||||
/// Important that it is done as query rewrite. It means rewritten query
|
||||
/// will be sent to remote servers during distributed query execution,
|
||||
/// and on all remote servers, function implementation will be same.
|
||||
struct CustomizeFunctionsData
|
||||
{
|
||||
using TypeToVisit = ASTFunction;
|
||||
|
||||
const String & count_distinct;
|
||||
|
||||
void visit(ASTFunction & func, ASTPtr &)
|
||||
{
|
||||
if (Poco::toLower(func.name) == "countdistinct")
|
||||
func.name = count_distinct;
|
||||
}
|
||||
};
|
||||
|
||||
using CustomizeFunctionsMatcher = OneTypeMatcher<CustomizeFunctionsData>;
|
||||
using CustomizeFunctionsVisitor = InDepthNodeVisitor<CustomizeFunctionsMatcher, true>;
|
||||
|
||||
|
||||
/// Add columns from storage to source_columns list.
|
||||
void collectSourceColumns(const ColumnsDescription & columns, NamesAndTypesList & source_columns, bool add_virtuals)
|
||||
@ -850,6 +871,11 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform();
|
||||
}
|
||||
|
||||
{
|
||||
CustomizeFunctionsVisitor::Data data{settings.count_distinct_implementation};
|
||||
CustomizeFunctionsVisitor(data).visit(query);
|
||||
}
|
||||
|
||||
/// Creates a dictionary `aliases`: alias -> ASTPtr
|
||||
{
|
||||
LogAST log;
|
||||
@ -857,6 +883,12 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
QueryAliasesVisitor(query_aliases_data, log.stream()).visit(query);
|
||||
}
|
||||
|
||||
/// Mark table ASTIdentifiers with not a column marker
|
||||
{
|
||||
MarkTableIdentifiersVisitor::Data data{result.aliases};
|
||||
MarkTableIdentifiersVisitor(data).visit(query);
|
||||
}
|
||||
|
||||
/// Common subexpression elimination. Rewrite rules.
|
||||
{
|
||||
QueryNormalizer::Data normalizer_data(result.aliases, context.getSettingsRef());
|
||||
|
16
dbms/src/Interpreters/misc.h
Normal file
16
dbms/src/Interpreters/misc.h
Normal file
@ -0,0 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
inline bool functionIsInOperator(const std::string & name)
|
||||
{
|
||||
return name == "in" || name == "notIn";
|
||||
}
|
||||
|
||||
inline bool functionIsInOrGlobalInOperator(const std::string & name)
|
||||
{
|
||||
return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn";
|
||||
}
|
||||
|
||||
}
|
@ -4,7 +4,7 @@
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/BloomFilterHash.h>
|
||||
#include <Common/HashTable/ClearableHashMap.h>
|
||||
#include <Storages/MergeTree/RPNBuilder.h>
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/RPNBuilder.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <ext/map.h>
|
||||
|
Loading…
Reference in New Issue
Block a user