mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
try to get rid of IAST::range #4058
This commit is contained in:
parent
addc9a43b0
commit
5154814bf9
@ -131,7 +131,7 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool
|
|||||||
|
|
||||||
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set);
|
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set);
|
||||||
set->createFromAST(set_element_types, elements_ast, context);
|
set->createFromAST(set_element_types, elements_ast, context);
|
||||||
prepared_sets[right_arg->range] = std::move(set);
|
prepared_sets[right_arg->getTreeHash()] = std::move(set);
|
||||||
}
|
}
|
||||||
|
|
||||||
static String getUniqueName(const Block & block, const String & prefix)
|
static String getUniqueName(const Block & block, const String & prefix)
|
||||||
@ -380,12 +380,12 @@ void ActionsVisitor::visit(const ASTPtr & ast)
|
|||||||
/// Select the name in the next cycle.
|
/// Select the name in the next cycle.
|
||||||
argument_names.emplace_back();
|
argument_names.emplace_back();
|
||||||
}
|
}
|
||||||
else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1)
|
else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_sets.count(child->getTreeHash()))
|
||||||
{
|
{
|
||||||
ColumnWithTypeAndName column;
|
ColumnWithTypeAndName column;
|
||||||
column.type = std::make_shared<DataTypeSet>();
|
column.type = std::make_shared<DataTypeSet>();
|
||||||
|
|
||||||
const SetPtr & set = prepared_sets[child->range];
|
const SetPtr & set = prepared_sets[child->getTreeHash()];
|
||||||
|
|
||||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||||
@ -530,9 +530,10 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc
|
|||||||
*/
|
*/
|
||||||
const IAST & args = *node->arguments;
|
const IAST & args = *node->arguments;
|
||||||
const ASTPtr & arg = args.children.at(1);
|
const ASTPtr & arg = args.children.at(1);
|
||||||
|
IAST::Hash tree_hash = arg->getTreeHash();
|
||||||
|
|
||||||
/// Already converted.
|
/// Already converted.
|
||||||
if (prepared_sets.count(arg->range))
|
if (prepared_sets.count(tree_hash))
|
||||||
return;
|
return;
|
||||||
|
|
||||||
/// If the subquery or table name for SELECT.
|
/// If the subquery or table name for SELECT.
|
||||||
@ -552,7 +553,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc
|
|||||||
|
|
||||||
if (storage_set)
|
if (storage_set)
|
||||||
{
|
{
|
||||||
prepared_sets[arg->range] = storage_set->getSet();
|
prepared_sets[tree_hash] = storage_set->getSet();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -566,7 +567,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc
|
|||||||
/// If you already created a Set with the same subquery / table.
|
/// If you already created a Set with the same subquery / table.
|
||||||
if (subquery_for_set.set)
|
if (subquery_for_set.set)
|
||||||
{
|
{
|
||||||
prepared_sets[arg->range] = subquery_for_set.set;
|
prepared_sets[tree_hash] = subquery_for_set.set;
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -612,7 +613,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc
|
|||||||
}
|
}
|
||||||
|
|
||||||
subquery_for_set.set = set;
|
subquery_for_set.set = set;
|
||||||
prepared_sets[arg->range] = set;
|
prepared_sets[tree_hash] = set;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Parsers/StringRange.h>
|
#include <Parsers/IAST.h>
|
||||||
|
#include <Interpreters/PreparedSets.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
|
|
||||||
|
|
||||||
@ -10,13 +11,6 @@ namespace DB
|
|||||||
class Context;
|
class Context;
|
||||||
class ASTFunction;
|
class ASTFunction;
|
||||||
|
|
||||||
|
|
||||||
class Set;
|
|
||||||
using SetPtr = std::shared_ptr<Set>;
|
|
||||||
/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range.
|
|
||||||
/// It should be taken into account when we want to change AST part which contains sets.
|
|
||||||
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointersHash, StringRangePointersEqualTo>;
|
|
||||||
|
|
||||||
class Join;
|
class Join;
|
||||||
using JoinPtr = std::shared_ptr<Join>;
|
using JoinPtr = std::shared_ptr<Join>;
|
||||||
|
|
||||||
|
@ -279,7 +279,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
prepared_sets[subquery_or_table_name->range] = std::move(set);
|
prepared_sets[subquery_or_table_name->getTreeHash()] = std::move(set);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -308,7 +308,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
|
|||||||
{
|
{
|
||||||
const ASTPtr & arg = args.children.at(1);
|
const ASTPtr & arg = args.children.at(1);
|
||||||
|
|
||||||
if (!prepared_sets.count(arg->range)) /// Not already prepared.
|
if (!prepared_sets.count(arg->getTreeHash())) /// Not already prepared.
|
||||||
{
|
{
|
||||||
if (typeid_cast<ASTSubquery *>(arg.get()) || isIdentifier(arg))
|
if (typeid_cast<ASTSubquery *>(arg.get()) || isIdentifier(arg))
|
||||||
{
|
{
|
||||||
|
@ -339,11 +339,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
|
|||||||
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
|
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
|
||||||
column_declaration->name = column.name;
|
column_declaration->name = column.name;
|
||||||
|
|
||||||
StringPtr type_name = std::make_shared<String>(column.type->getName());
|
|
||||||
auto pos = type_name->data();
|
|
||||||
const auto end = pos + type_name->size();
|
|
||||||
|
|
||||||
ParserIdentifierWithOptionalParameters storage_p;
|
ParserIdentifierWithOptionalParameters storage_p;
|
||||||
|
String type_name = column.type->getName();
|
||||||
|
auto pos = type_name.data();
|
||||||
|
const auto end = pos + type_name.size();
|
||||||
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0);
|
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0);
|
||||||
columns_list->children.emplace_back(column_declaration);
|
columns_list->children.emplace_back(column_declaration);
|
||||||
}
|
}
|
||||||
@ -362,11 +361,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
|||||||
|
|
||||||
column_declaration->name = column.name;
|
column_declaration->name = column.name;
|
||||||
|
|
||||||
StringPtr type_name = std::make_shared<String>(column.type->getName());
|
|
||||||
auto type_name_pos = type_name->data();
|
|
||||||
const auto type_name_end = type_name_pos + type_name->size();
|
|
||||||
|
|
||||||
ParserIdentifierWithOptionalParameters storage_p;
|
ParserIdentifierWithOptionalParameters storage_p;
|
||||||
|
String type_name = column.type->getName();
|
||||||
|
auto type_name_pos = type_name.data();
|
||||||
|
const auto type_name_end = type_name_pos + type_name.size();
|
||||||
column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0);
|
column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0);
|
||||||
|
|
||||||
const auto defaults_it = columns.defaults.find(column.name);
|
const auto defaults_it = columns.defaults.find(column.name);
|
||||||
|
@ -228,17 +228,10 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
|||||||
|
|
||||||
/// Construct a list of literals `x1, ..., xN` from the string `expr = x1 OR ... OR expr = xN`
|
/// Construct a list of literals `x1, ..., xN` from the string `expr = x1 OR ... OR expr = xN`
|
||||||
ASTPtr value_list = std::make_shared<ASTExpressionList>();
|
ASTPtr value_list = std::make_shared<ASTExpressionList>();
|
||||||
const char * min_range_first = nullptr;
|
|
||||||
const char * max_range_second = nullptr;
|
|
||||||
for (const auto function : equality_functions)
|
for (const auto function : equality_functions)
|
||||||
{
|
{
|
||||||
const auto & operands = getFunctionOperands(function);
|
const auto & operands = getFunctionOperands(function);
|
||||||
value_list->children.push_back(operands[1]);
|
value_list->children.push_back(operands[1]);
|
||||||
/// Get range min/max from all literals x1...xN, which will be used as tuple_functions' range
|
|
||||||
if (min_range_first == nullptr || min_range_first > operands[1]->range.first)
|
|
||||||
min_range_first = operands[1]->range.first;
|
|
||||||
if (max_range_second == nullptr || max_range_second < operands[1]->range.second)
|
|
||||||
max_range_second = operands[1]->range.second;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Sort the literals so that they are specified in the same order in the IN expression.
|
/// Sort the literals so that they are specified in the same order in the IN expression.
|
||||||
@ -260,7 +253,6 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
|||||||
|
|
||||||
auto tuple_function = std::make_shared<ASTFunction>();
|
auto tuple_function = std::make_shared<ASTFunction>();
|
||||||
tuple_function->name = "tuple";
|
tuple_function->name = "tuple";
|
||||||
tuple_function->range = StringRange(min_range_first, max_range_second);
|
|
||||||
tuple_function->arguments = value_list;
|
tuple_function->arguments = value_list;
|
||||||
tuple_function->children.push_back(tuple_function->arguments);
|
tuple_function->children.push_back(tuple_function->arguments);
|
||||||
|
|
||||||
|
20
dbms/src/Interpreters/PreparedSets.h
Normal file
20
dbms/src/Interpreters/PreparedSets.h
Normal file
@ -0,0 +1,20 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
#include <memory>
|
||||||
|
#include <unordered_map>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct ASTHalfHash
|
||||||
|
{
|
||||||
|
UInt64 operator()(const IAST::Hash & ast_hash) const { return ast_hash.first; }
|
||||||
|
};
|
||||||
|
|
||||||
|
class Set;
|
||||||
|
using SetPtr = std::shared_ptr<Set>;
|
||||||
|
|
||||||
|
using PreparedSets = std::unordered_map<IAST::Hash, SetPtr, ASTHalfHash>;
|
||||||
|
|
||||||
|
}
|
@ -3,6 +3,7 @@
|
|||||||
#include <Core/Names.h>
|
#include <Core/Names.h>
|
||||||
#include <Parsers/IAST.h>
|
#include <Parsers/IAST.h>
|
||||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||||
|
#include <map>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -150,7 +150,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
|
|
||||||
ParserQuery parser(end, settings.enable_debug_queries);
|
ParserQuery parser(end, settings.enable_debug_queries);
|
||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
size_t query_size;
|
const char * query_end;
|
||||||
|
|
||||||
/// Don't limit the size of internal queries.
|
/// Don't limit the size of internal queries.
|
||||||
size_t max_query_size = 0;
|
size_t max_query_size = 0;
|
||||||
@ -162,10 +162,11 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
/// TODO Parser should fail early when max_query_size limit is reached.
|
/// TODO Parser should fail early when max_query_size limit is reached.
|
||||||
ast = parseQuery(parser, begin, end, "", max_query_size);
|
ast = parseQuery(parser, begin, end, "", max_query_size);
|
||||||
|
|
||||||
/// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion.
|
const auto * insert_query = dynamic_cast<const ASTInsertQuery *>(ast.get());
|
||||||
if (!(begin <= ast->range.first && ast->range.second <= end))
|
if (insert_query && insert_query->data)
|
||||||
throw Exception("Unexpected behavior: AST chars range is not inside source range", ErrorCodes::LOGICAL_ERROR);
|
query_end = insert_query->data;
|
||||||
query_size = ast->range.second - begin;
|
else
|
||||||
|
query_end = end;
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
@ -180,7 +181,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
|
|
||||||
String query(begin, query_size);
|
/// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion.
|
||||||
|
String query(begin, query_end);
|
||||||
BlockIO res;
|
BlockIO res;
|
||||||
|
|
||||||
try
|
try
|
||||||
|
@ -24,12 +24,22 @@ std::string getClusterName(const IAST & node)
|
|||||||
if (const ASTLiteral * ast_lit = typeid_cast<const ASTLiteral *>(&node))
|
if (const ASTLiteral * ast_lit = typeid_cast<const ASTLiteral *>(&node))
|
||||||
return ast_lit->value.safeGet<String>();
|
return ast_lit->value.safeGet<String>();
|
||||||
|
|
||||||
|
/// A hack to support hyphens in cluster names.
|
||||||
if (const ASTFunction * ast_func = typeid_cast<const ASTFunction *>(&node))
|
if (const ASTFunction * ast_func = typeid_cast<const ASTFunction *>(&node))
|
||||||
{
|
{
|
||||||
if (!ast_func->range.first || !ast_func->range.second)
|
if (ast_func->name != "minus" || !ast_func->arguments || ast_func->arguments->children.size() < 2)
|
||||||
throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
return String(ast_func->range.first, ast_func->range.second);
|
String name;
|
||||||
|
for (const auto & arg : ast_func->arguments->children)
|
||||||
|
{
|
||||||
|
if (name.empty())
|
||||||
|
name += getClusterName(*arg);
|
||||||
|
else
|
||||||
|
name += "-" + getClusterName(*arg);
|
||||||
|
}
|
||||||
|
|
||||||
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
@ -19,7 +19,6 @@ public:
|
|||||||
: name(name_)
|
: name(name_)
|
||||||
, special(false)
|
, special(false)
|
||||||
{
|
{
|
||||||
range = StringRange(name.data(), name.data() + name.size());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Get the text that identifies this element. */
|
/** Get the text that identifies this element. */
|
||||||
|
@ -672,8 +672,6 @@ bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
|
|
||||||
bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
auto begin = pos;
|
|
||||||
|
|
||||||
if (!ParserKeyword("EXTRACT").ignore(pos, expected))
|
if (!ParserKeyword("EXTRACT").ignore(pos, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -734,14 +732,10 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
|||||||
|
|
||||||
auto function = std::make_shared<ASTFunction>();
|
auto function = std::make_shared<ASTFunction>();
|
||||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||||
function->range.first = begin->begin;
|
|
||||||
function->range.second = pos->begin;
|
|
||||||
function->name = function_name; //"toYear";
|
function->name = function_name; //"toYear";
|
||||||
function->arguments = exp_list;
|
function->arguments = exp_list;
|
||||||
function->children.push_back(exp_list);
|
function->children.push_back(exp_list);
|
||||||
exp_list->children.push_back(expr);
|
exp_list->children.push_back(expr);
|
||||||
exp_list->range.first = begin->begin;
|
|
||||||
exp_list->range.second = pos->begin;
|
|
||||||
node = function;
|
node = function;
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
|
@ -138,7 +138,6 @@ static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expect
|
|||||||
bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
bool first = true;
|
bool first = true;
|
||||||
Pos begin = pos;
|
|
||||||
|
|
||||||
while (1)
|
while (1)
|
||||||
{
|
{
|
||||||
@ -174,16 +173,12 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
|
|||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// the first argument of the function is the previous element, the second is the next one
|
/// the first argument of the function is the previous element, the second is the next one
|
||||||
function->range.first = begin->begin;
|
|
||||||
function->range.second = pos->begin;
|
|
||||||
function->name = it[1];
|
function->name = it[1];
|
||||||
function->arguments = exp_list;
|
function->arguments = exp_list;
|
||||||
function->children.push_back(exp_list);
|
function->children.push_back(exp_list);
|
||||||
|
|
||||||
exp_list->children.push_back(node);
|
exp_list->children.push_back(node);
|
||||||
exp_list->children.push_back(elem);
|
exp_list->children.push_back(elem);
|
||||||
exp_list->range.first = begin->begin;
|
|
||||||
exp_list->range.second = pos->begin;
|
|
||||||
|
|
||||||
/** special exception for the access operator to the element of the array `x[y]`, which
|
/** special exception for the access operator to the element of the array `x[y]`, which
|
||||||
* contains the infix part '[' and the suffix ''] '(specified as' [')
|
* contains the infix part '[' and the suffix ''] '(specified as' [')
|
||||||
@ -243,8 +238,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
|||||||
ASTPtr left;
|
ASTPtr left;
|
||||||
ASTPtr right;
|
ASTPtr right;
|
||||||
|
|
||||||
Pos begin = pos;
|
|
||||||
|
|
||||||
if (!elem_parser.parse(pos, subject, expected))
|
if (!elem_parser.parse(pos, subject, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -279,14 +272,10 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
|||||||
args_le->children.emplace_back(subject);
|
args_le->children.emplace_back(subject);
|
||||||
args_le->children.emplace_back(right);
|
args_le->children.emplace_back(right);
|
||||||
|
|
||||||
f_ge->range.first = begin->begin;
|
|
||||||
f_ge->range.second = pos->begin;
|
|
||||||
f_ge->name = "greaterOrEquals";
|
f_ge->name = "greaterOrEquals";
|
||||||
f_ge->arguments = args_ge;
|
f_ge->arguments = args_ge;
|
||||||
f_ge->children.emplace_back(f_ge->arguments);
|
f_ge->children.emplace_back(f_ge->arguments);
|
||||||
|
|
||||||
f_le->range.first = begin->begin;
|
|
||||||
f_le->range.second = pos->begin;
|
|
||||||
f_le->name = "lessOrEquals";
|
f_le->name = "lessOrEquals";
|
||||||
f_le->arguments = args_le;
|
f_le->arguments = args_le;
|
||||||
f_le->children.emplace_back(f_le->arguments);
|
f_le->children.emplace_back(f_le->arguments);
|
||||||
@ -294,8 +283,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
|||||||
args_and->children.emplace_back(f_ge);
|
args_and->children.emplace_back(f_ge);
|
||||||
args_and->children.emplace_back(f_le);
|
args_and->children.emplace_back(f_le);
|
||||||
|
|
||||||
f_and->range.first = begin->begin;
|
|
||||||
f_and->range.second = pos->begin;
|
|
||||||
f_and->name = "and";
|
f_and->name = "and";
|
||||||
f_and->arguments = args_and;
|
f_and->arguments = args_and;
|
||||||
f_and->children.emplace_back(f_and->arguments);
|
f_and->children.emplace_back(f_and->arguments);
|
||||||
@ -315,8 +302,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect
|
|||||||
ASTPtr elem_then;
|
ASTPtr elem_then;
|
||||||
ASTPtr elem_else;
|
ASTPtr elem_else;
|
||||||
|
|
||||||
Pos begin = pos;
|
|
||||||
|
|
||||||
if (!elem_parser.parse(pos, elem_cond, expected))
|
if (!elem_parser.parse(pos, elem_cond, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -339,8 +324,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect
|
|||||||
/// function arguments
|
/// function arguments
|
||||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||||
|
|
||||||
function->range.first = begin->begin;
|
|
||||||
function->range.second = pos->begin;
|
|
||||||
function->name = "if";
|
function->name = "if";
|
||||||
function->arguments = exp_list;
|
function->arguments = exp_list;
|
||||||
function->children.push_back(exp_list);
|
function->children.push_back(exp_list);
|
||||||
@ -348,8 +331,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect
|
|||||||
exp_list->children.push_back(elem_cond);
|
exp_list->children.push_back(elem_cond);
|
||||||
exp_list->children.push_back(elem_then);
|
exp_list->children.push_back(elem_then);
|
||||||
exp_list->children.push_back(elem_else);
|
exp_list->children.push_back(elem_else);
|
||||||
exp_list->range.first = begin->begin;
|
|
||||||
exp_list->range.second = pos->begin;
|
|
||||||
|
|
||||||
node = function;
|
node = function;
|
||||||
}
|
}
|
||||||
@ -423,7 +404,6 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
|||||||
bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
/// try to find any of the valid operators
|
/// try to find any of the valid operators
|
||||||
Pos begin = pos;
|
|
||||||
const char ** it;
|
const char ** it;
|
||||||
for (it = operators; *it; it += 2)
|
for (it = operators; *it; it += 2)
|
||||||
{
|
{
|
||||||
@ -471,15 +451,11 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex
|
|||||||
/// function arguments
|
/// function arguments
|
||||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||||
|
|
||||||
function->range.first = begin->begin;
|
|
||||||
function->range.second = pos->begin;
|
|
||||||
function->name = it[1];
|
function->name = it[1];
|
||||||
function->arguments = exp_list;
|
function->arguments = exp_list;
|
||||||
function->children.push_back(exp_list);
|
function->children.push_back(exp_list);
|
||||||
|
|
||||||
exp_list->children.push_back(elem);
|
exp_list->children.push_back(elem);
|
||||||
exp_list->range.first = begin->begin;
|
|
||||||
exp_list->range.second = pos->begin;
|
|
||||||
|
|
||||||
node = function;
|
node = function;
|
||||||
}
|
}
|
||||||
@ -595,8 +571,6 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
|
|
||||||
bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
Pos begin = pos;
|
|
||||||
|
|
||||||
/// If no INTERVAL keyword, go to nested parser.
|
/// If no INTERVAL keyword, go to nested parser.
|
||||||
if (!ParserKeyword("INTERVAL").ignore(pos, expected))
|
if (!ParserKeyword("INTERVAL").ignore(pos, expected))
|
||||||
return next_parser.parse(pos, node, expected);
|
return next_parser.parse(pos, node, expected);
|
||||||
@ -620,15 +594,11 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec
|
|||||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||||
|
|
||||||
/// the first argument of the function is the previous element, the second is the next one
|
/// the first argument of the function is the previous element, the second is the next one
|
||||||
function->range.first = begin->begin;
|
|
||||||
function->range.second = pos->begin;
|
|
||||||
function->name = function_name;
|
function->name = function_name;
|
||||||
function->arguments = exp_list;
|
function->arguments = exp_list;
|
||||||
function->children.push_back(exp_list);
|
function->children.push_back(exp_list);
|
||||||
|
|
||||||
exp_list->children.push_back(expr);
|
exp_list->children.push_back(expr);
|
||||||
exp_list->range.first = begin->begin;
|
|
||||||
exp_list->range.second = pos->begin;
|
|
||||||
|
|
||||||
node = function;
|
node = function;
|
||||||
return true;
|
return true;
|
||||||
|
@ -7,7 +7,6 @@
|
|||||||
|
|
||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Parsers/StringRange.h>
|
|
||||||
#include <Parsers/IdentifierQuotingStyle.h>
|
#include <Parsers/IdentifierQuotingStyle.h>
|
||||||
|
|
||||||
|
|
||||||
@ -54,7 +53,6 @@ class IAST : public std::enable_shared_from_this<IAST>
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ASTs children;
|
ASTs children;
|
||||||
StringRange range;
|
|
||||||
|
|
||||||
SemanticPtr semantic;
|
SemanticPtr semantic;
|
||||||
|
|
||||||
@ -209,11 +207,7 @@ public:
|
|||||||
|
|
||||||
virtual void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const
|
virtual void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const
|
||||||
{
|
{
|
||||||
throw Exception("Unknown element in AST: " + getID()
|
throw Exception("Unknown element in AST: " + getID(), ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
|
||||||
+ ((range.first && (range.second > range.first))
|
|
||||||
? " '" + std::string(range.first, range.second - range.first) + "'"
|
|
||||||
: ""),
|
|
||||||
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void cloneChildren();
|
void cloneChildren();
|
||||||
|
@ -22,8 +22,6 @@ bool IParserBase::parse(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
node = nullptr;
|
node = nullptr;
|
||||||
pos = begin;
|
pos = begin;
|
||||||
}
|
}
|
||||||
else if (node)
|
|
||||||
node->range = StringRange(begin, pos);
|
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -635,13 +635,13 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
|
|||||||
|
|
||||||
DataTypePtr key_expr_type; /// Type of expression containing key column
|
DataTypePtr key_expr_type; /// Type of expression containing key column
|
||||||
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
|
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
|
||||||
size_t key_column_num; /// Number of a key column (inside key_column_names array)
|
size_t key_column_num = -1; /// Number of a key column (inside key_column_names array)
|
||||||
MonotonicFunctionsChain chain;
|
MonotonicFunctionsChain chain;
|
||||||
bool is_set_const = false;
|
bool is_set_const = false;
|
||||||
bool is_constant_transformed = false;
|
bool is_constant_transformed = false;
|
||||||
|
|
||||||
if (prepared_sets.count(args[1]->range)
|
if (prepared_sets.count(args[1]->getTreeHash())
|
||||||
&& tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
|
&& tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->getTreeHash()], key_column_num))
|
||||||
{
|
{
|
||||||
key_arg_pos = 0;
|
key_arg_pos = 0;
|
||||||
is_set_const = true;
|
is_set_const = true;
|
||||||
@ -671,6 +671,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
|
|||||||
else
|
else
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
if (key_column_num == static_cast<size_t>(-1))
|
||||||
|
throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
std::string func_name = func->name;
|
std::string func_name = func->name;
|
||||||
|
|
||||||
/// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5"
|
/// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5"
|
||||||
@ -1015,7 +1018,7 @@ bool KeyCondition::mayBeTrueInParallelogram(const std::vector<Range> & parallelo
|
|||||||
{
|
{
|
||||||
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
|
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
|
||||||
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
|
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
|
||||||
PreparedSets::const_iterator it = prepared_sets.find(args[1]->range);
|
PreparedSets::const_iterator it = prepared_sets.find(args[1]->getTreeHash());
|
||||||
if (in_func && it != prepared_sets.end())
|
if (in_func && it != prepared_sets.end())
|
||||||
{
|
{
|
||||||
rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types));
|
rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types));
|
||||||
|
@ -334,7 +334,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const
|
|||||||
if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) ||
|
if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) ||
|
||||||
(primary_key_columns.count(second_arg_name) && isConstant(args[0])) ||
|
(primary_key_columns.count(second_arg_name) && isConstant(args[0])) ||
|
||||||
(primary_key_columns.count(first_arg_name)
|
(primary_key_columns.count(first_arg_name)
|
||||||
&& (prepared_sets.count(args[1]->range) || typeid_cast<const ASTSubquery *>(args[1].get()))))
|
&& (typeid_cast<const ASTSubquery *>(args[1].get()) || prepared_sets.count(args[1]->getTreeHash()))))
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Storages/MutationCommands.h>
|
#include <Storages/MutationCommands.h>
|
||||||
|
#include <map>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,24 +1,14 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Interpreters/PreparedSets.h>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
#include <unordered_map>
|
|
||||||
#include <Parsers/StringRange.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class IAST;
|
|
||||||
using ASTPtr = std::shared_ptr<IAST>;
|
|
||||||
|
|
||||||
class ExpressionActions;
|
class ExpressionActions;
|
||||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||||
|
|
||||||
class Set;
|
|
||||||
using SetPtr = std::shared_ptr<Set>;
|
|
||||||
|
|
||||||
/// Information about calculated sets in right hand side of IN.
|
|
||||||
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointersHash, StringRangePointersEqualTo>;
|
|
||||||
|
|
||||||
struct PrewhereInfo
|
struct PrewhereInfo
|
||||||
{
|
{
|
||||||
/// Actions which are executed in order to alias columns are used for prewhere actions.
|
/// Actions which are executed in order to alias columns are used for prewhere actions.
|
||||||
|
Loading…
Reference in New Issue
Block a user