ClickHouse/dbms/src/Storages/MergeTree/RPNBuilder.h

149 lines
4.6 KiB
C++
Raw Normal View History

2019-03-07 09:15:58 +00:00
#pragma once
#include <Common/typeid_cast.h>
#include <Core/Block.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <Storages/SelectQueryInfo.h>
namespace DB
{
/** Calculate expressions, that depend only on constants.
* For index to work when something like "WHERE Date = toDate(now())" is written.
*/
static Block getBlockWithConstants(
2019-03-07 11:31:23 +00:00
const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, const Context & context)
2019-03-07 09:15:58 +00:00
{
Block result
2019-03-07 11:31:23 +00:00
{
{ DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared<DataTypeUInt8>(), "_dummy" }
};
2019-03-07 09:15:58 +00:00
const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActions();
expr_for_constant_folding->execute(result);
return result;
}
2019-03-07 13:54:56 +00:00
/// Builds reverse polish notation
2019-03-07 09:15:58 +00:00
template <typename RPNElement>
class RPNBuilder
{
public:
using RPN = std::vector<RPNElement>;
using AtomFromASTFunc = std::function<
bool(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)>;
RPNBuilder(
const SelectQueryInfo & query_info,
const Context & context_,
const AtomFromASTFunc & atomFromAST_)
: context(context_), atomFromAST(atomFromAST_)
{
/** Evaluation of expressions that depend only on constants.
* For the index to be used, if it is written, for example `WHERE Date = toDate(now())`.
*/
block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
/// Trasform WHERE section to Reverse Polish notation
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query_info.query);
if (select.where_expression)
{
traverseAST(select.where_expression);
if (select.prewhere_expression)
{
traverseAST(select.prewhere_expression);
rpn.emplace_back(RPNElement::FUNCTION_AND);
}
}
else if (select.prewhere_expression)
{
traverseAST(select.prewhere_expression);
}
else
{
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
}
}
2019-03-07 13:54:56 +00:00
void extractRPN(RPN & out) { out = std::move(rpn); }
2019-03-07 09:15:58 +00:00
private:
void traverseAST(const ASTPtr & node)
{
RPNElement element;
if (ASTFunction * func = typeid_cast<ASTFunction *>(&*node))
{
if (operatorFromAST(func, element))
{
auto & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
for (size_t i = 0, size = args.size(); i < size; ++i)
{
traverseAST(args[i]);
/** The first part of the condition is for the correct support of `and` and `or` functions of arbitrary arity
* - in this case `n - 1` elements are added (where `n` is the number of arguments).
*/
if (i != 0 || element.function == RPNElement::FUNCTION_NOT)
rpn.emplace_back(std::move(element));
}
return;
}
}
if (!atomFromAST(node, context, block_with_constants, element))
{
element.function = RPNElement::FUNCTION_UNKNOWN;
}
rpn.emplace_back(std::move(element));
}
bool operatorFromAST(const ASTFunction * func, RPNElement & out)
{
/// Functions AND, OR, NOT.
/** Also a special function `indexHint` - works as if instead of calling a function there are just parentheses
* (or, the same thing - calling the function `and` from one argument).
*/
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
if (func->name == "not")
{
if (args.size() != 1)
return false;
out.function = RPNElement::FUNCTION_NOT;
}
else
{
if (func->name == "and" || func->name == "indexHint")
out.function = RPNElement::FUNCTION_AND;
else if (func->name == "or")
out.function = RPNElement::FUNCTION_OR;
else
return false;
}
return true;
}
const Context & context;
const AtomFromASTFunc & atomFromAST;
Block block_with_constants;
RPN rpn;
};
};