Compatibility with new parser and AST of tables list [#METR-2944].

This commit is contained in:
Alexey Milovidov 2016-07-22 23:39:28 +03:00
parent 1ef87202f6
commit 130ec8b58b
17 changed files with 312 additions and 346 deletions

View File

@ -2,7 +2,7 @@
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Parsers/ASTTablesInSelectQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Storages/IStorage.h>
#include <DB/Storages/StorageDistributed.h>
@ -69,11 +69,13 @@ std::string getNameFromInSubqueryAttributes(IAST::Attributes attributes)
/// Проверить, указана ли таблица в секции FROM.
bool isQueryFromTable(const ASTSelectQuery & query)
{
if (query.table)
auto query_table = query.table();
if (query_table)
{
if (typeid_cast<const ASTSelectQuery *>(query.table.get()) != nullptr)
if (typeid_cast<const ASTSelectQuery *>(query_table.get()) != nullptr)
return false;
else if (typeid_cast<const ASTFunction *>(query.table.get()) != nullptr)
else if (typeid_cast<const ASTFunction *>(query_table.get()) != nullptr)
return false;
else
return true;
@ -165,7 +167,7 @@ public:
to_preprocess.pop_back();
ASTFunction * function;
ASTJoin * join;
ASTTableJoin * join;
ASTSelectQuery * sub_select_query;
if ((function = typeid_cast<ASTFunction *>(node)) != nullptr)
@ -178,12 +180,12 @@ public:
node->attributes |= attributes;
}
}
else if ((join = typeid_cast<ASTJoin *>(node)) != nullptr)
else if ((join = typeid_cast<ASTTableJoin *>(node)) != nullptr)
{
/// Найдена секция JOIN.
node->enclosing_in_or_join = node;
node->attributes |= IAST::IsJoin;
if (join->locality == ASTJoin::Global)
if (join->locality == ASTTableJoin::Locality::Global)
node->attributes |= IAST::IsGlobal;
}
else if ((node != static_cast<IAST *>(select_query))
@ -250,8 +252,8 @@ private:
{
if (enclosing_in_or_join.attributes & IAST::IsJoin)
{
auto & join = static_cast<ASTJoin &>(enclosing_in_or_join);
join.locality = ASTJoin::Global;
auto & join = static_cast<ASTTableJoin &>(enclosing_in_or_join);
join.locality = ASTTableJoin::Locality::Global;
}
else if (enclosing_in_or_join.attributes & (IAST::IsIn | IAST::IsNotIn))
{
@ -268,27 +270,9 @@ private:
/// Преобразовать распределённую таблицу в соответствующую удалённую таблицу.
auto & distributed_storage = static_cast<TStorageDistributed &>(*subquery_table_storage);
if (!sub_select_query.database)
{
sub_select_query.database = std::make_shared<ASTIdentifier>(StringRange(), distributed_storage.getRemoteDatabaseName(),
ASTIdentifier::Database);
/// Поскольку был создан новый узел для БД, необходимо его вставить в список
/// потомков этого подзапроса. См. ParserSelectQuery для структуры потомков.
if (sub_select_query.children.size() < 2)
throw Exception("InJoinSubqueriesPreprocessor: Internal error", ErrorCodes::LOGICAL_ERROR);
auto it = ++sub_select_query.children.begin();
sub_select_query.children.insert(it, sub_select_query.database);
}
else
{
auto & db_name = typeid_cast<ASTIdentifier &>(*sub_select_query.database).name;
db_name = distributed_storage.getRemoteDatabaseName();
}
auto & table_name = typeid_cast<ASTIdentifier &>(*sub_select_query.table).name;
table_name = distributed_storage.getRemoteTableName();
sub_select_query.replaceDatabaseAndTable(
distributed_storage.getRemoteDatabaseName(),
distributed_storage.getRemoteTableName());
}
else
throw Exception("InJoinSubqueriesPreprocessor: Internal error", ErrorCodes::LOGICAL_ERROR);
@ -296,18 +280,21 @@ private:
StoragePtr getDistributedSubqueryStorage(const ASTSelectQuery & sub_select_query) const
{
if (!sub_select_query.table)
auto database = sub_select_query.database();
auto table = sub_select_query.table();
if (!table)
return {};
const auto identifier = typeid_cast<const ASTIdentifier *>(sub_select_query.table.get());
const auto identifier = typeid_cast<const ASTIdentifier *>(table.get());
if (identifier == nullptr)
return {};
const std::string & table_name = identifier->name;
std::string database_name;
if (sub_select_query.database)
database_name = typeid_cast<const ASTIdentifier &>(*sub_select_query.database).name;
if (database)
database_name = typeid_cast<const ASTIdentifier &>(*database).name;
else
database_name = "";

View File

@ -2,7 +2,7 @@
#include <common/logger_useful.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Parsers/ASTTablesInSelectQuery.h>
#include <DB/Interpreters/AggregationCommon.h>
@ -62,7 +62,7 @@ class Join
{
public:
Join(const Names & key_names_left_, const Names & key_names_right_,
const Limits & limits, ASTJoin::Kind kind_, ASTJoin::Strictness strictness_)
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_)
: kind(kind_), strictness(strictness_),
key_names_left(key_names_left_),
key_names_right(key_names_right_),
@ -107,7 +107,7 @@ public:
/// Считает суммарный размер в байтах буфферов всех Join'ов + размер string_pool'а
size_t getTotalByteCount() const;
ASTJoin::Kind getKind() const { return kind; }
ASTTableJoin::Kind getKind() const { return kind; }
/// Ссылка на строку в блоке.
@ -188,8 +188,8 @@ public:
private:
friend class NonJoinedBlockInputStream;
ASTJoin::Kind kind;
ASTJoin::Strictness strictness;
ASTTableJoin::Kind kind;
ASTTableJoin::Strictness strictness;
/// Имена ключевых столбцов (по которым производится соединение) в "левой" таблице.
const Names key_names_left;
@ -247,10 +247,10 @@ private:
void init(Type type_);
template <ASTJoin::Strictness STRICTNESS, typename Maps>
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
void insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block);
template <ASTJoin::Kind KIND, ASTJoin::Strictness STRICTNESS, typename Maps>
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
void joinBlockImpl(Block & block, const Maps & maps) const;
void joinBlockImplCross(Block & block) const;

View File

@ -1,123 +0,0 @@
#pragma once
#include <DB/Parsers/IAST.h>
namespace DB
{
/** [GLOBAL] ANY|ALL INNER|LEFT JOIN (subquery) USING (tuple)
* Замечание: ARRAY JOIN к этому не относится.
*/
class ASTJoin : public IAST
{
public:
/// Алгоритм работы при распределённой обработке запроса.
enum Locality
{
Local, /// Выполнить JOIN, используя соответствующие данные в пределах каждого удалённого сервера.
Global /// Собрать соединяемые данные со всех удалённых серверов, объединить их, затем отправить на каждый удалённый сервер.
};
/// Оптимизация JOIN-а для большинства простых случаев.
enum Strictness
{
Any, /// Если в "правой" таблице несколько подходящих строк - выбрать из них одну, любую.
All /// Если в "правой" таблице несколько подходящих строк - размножить строки по ним (обычная семантика JOIN-а).
};
/// Способ соединения.
enum Kind
{
Inner, /// Оставить только записи, для которых в "правой" таблице есть соответствующая.
Left, /// Если в "правой" таблице нет соответствующих записей, заполнить столбцы значениями "по-умолчанию".
Right,
Full,
Cross /// Прямое произведение. strictness и using_expr_list не используются.
};
Locality locality = Local;
Strictness strictness = Any;
Kind kind = Inner;
ASTPtr table; /// "Правая" таблица для соединения - подзапрос или имя таблицы.
ASTPtr using_expr_list; /// По каким столбцам выполнять соединение.
ASTPtr on_expr; /// Условие соединения. Поддерживается либо USING либо ON, но не одновременно.
ASTJoin() = default;
ASTJoin(const StringRange range_) : IAST(range_) {}
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override
{
String res;
{
WriteBufferFromString wb(res);
if (locality == Global)
writeString("Global", wb);
writeString(strictness == Any ? "Any" : "All", wb);
writeString(
kind == Inner ? "Inner"
: (kind == Left ? "Left"
: (kind == Right ? "Right"
: (kind == Full ? "Full"
: "Cross"))), wb);
writeString("Join", wb);
}
return res;
};
ASTPtr clone() const override
{
auto res = std::make_shared<ASTJoin>(*this);
res->children.clear();
if (table) { res->table = table->clone(); res->children.push_back(res->table); }
if (using_expr_list) { res->using_expr_list = using_expr_list->clone(); res->children.push_back(res->using_expr_list); }
return res;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
frame.need_parens = false;
settings.ostr << (settings.hilite ? hilite_keyword : "");
if (locality == ASTJoin::Global)
settings.ostr << "GLOBAL ";
if (kind != ASTJoin::Cross)
settings.ostr << (strictness == ASTJoin::Any ? "ANY " : "ALL ");
settings.ostr << (kind == ASTJoin::Inner ? "INNER "
: (kind == ASTJoin::Left ? "LEFT "
: (kind == ASTJoin::Right ? "RIGHT "
: (kind == ASTJoin::Cross ? "CROSS "
: "FULL OUTER "))));
settings.ostr << "JOIN "
<< (settings.hilite ? hilite_none : "");
table->formatImpl(settings, state, frame);
if (using_expr_list)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " USING " << (settings.hilite ? hilite_none : "");
using_expr_list->formatImpl(settings, state, frame);
}
else if (on_expr)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : "");
on_expr->formatImpl(settings, state, frame);
}
}
};
}

View File

@ -404,11 +404,13 @@ private:
void determineArrayJoinedNames(ASTSelectQuery & select)
{
auto array_join_expression_list = select.array_join_expression_list();
/// much simplified code from ExpressionAnalyzer::getArrayJoinedColumns()
if (!select.array_join_expression_list)
if (!array_join_expression_list)
return;
for (const auto & ast : select.array_join_expression_list->children)
for (const auto & ast : array_join_expression_list->children)
array_joined_names.emplace(ast->getAliasOrColumnName());
}

View File

@ -21,7 +21,7 @@ public:
const String & path_,
const String & name_,
const Names & key_names_,
ASTJoin::Kind kind_, ASTJoin::Strictness strictness_,
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
@ -40,12 +40,12 @@ public:
JoinPtr & getJoin() { return join; }
/// Убедиться, что структура данных подходит для осуществления JOIN такого типа.
void assertCompatible(ASTJoin::Kind kind_, ASTJoin::Strictness strictness_) const;
void assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const;
private:
const Names & key_names;
ASTJoin::Kind kind; /// LEFT | INNER ...
ASTJoin::Strictness strictness; /// ANY | ALL
ASTTableJoin::Kind kind; /// LEFT | INNER ...
ASTTableJoin::Strictness strictness; /// ANY | ALL
JoinPtr join;
@ -53,7 +53,7 @@ private:
const String & path_,
const String & name_,
const Names & key_names_,
ASTJoin::Kind kind_, ASTJoin::Strictness strictness_,
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,

View File

@ -993,7 +993,7 @@ BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRigh
{
for (const auto & action : actions)
{
if (action.join && (action.join->getKind() == ASTJoin::Full || action.join->getKind() == ASTJoin::Right))
if (action.join && (action.join->getKind() == ASTTableJoin::Kind::Full || action.join->getKind() == ASTTableJoin::Kind::Right))
{
Block left_sample_block;
for (const auto & input_elem : input_columns)

View File

@ -225,20 +225,23 @@ void ExpressionAnalyzer::analyzeAggregation()
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(columns, settings);
if (select_query && select_query->array_join_expression_list)
if (select_query && select_query->array_join_expression_list())
{
getRootActions(select_query->array_join_expression_list, true, false, temp_actions);
getRootActions(select_query->array_join_expression_list(), true, false, temp_actions);
addMultipleArrayJoinAction(temp_actions);
}
if (select_query && select_query->join)
if (select_query)
{
auto join = typeid_cast<ASTJoin &>(*select_query->join);
if (join.using_expr_list)
getRootActions(join.using_expr_list, true, false, temp_actions);
const ASTTablesInSelectQueryElement * join = select_query->join();
if (join)
{
if (static_cast<const ASTTableJoin &>(*join->table_join).using_expression_list)
getRootActions(static_cast<const ASTTableJoin &>(*join->table_join).using_expression_list, true, false, temp_actions);
addJoinAction(temp_actions, true);
}
}
getAggregates(ast, temp_actions);
@ -338,11 +341,18 @@ void ExpressionAnalyzer::initGlobalSubqueries(ASTPtr & ast)
if (do_global && (node->name == "globalIn" || node->name == "globalNotIn"))
addExternalStorage(node->arguments->children.at(1));
}
else if (ASTJoin * node = typeid_cast<ASTJoin *>(ast.get()))
else if (ASTTablesInSelectQueryElement * node = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
{
/// Для GLOBAL JOIN.
if (do_global && node->locality == ASTJoin::Global)
addExternalStorage(node->table);
if (do_global && node->table_join
&& static_cast<const ASTTableJoin &>(*node->table_join).locality == ASTTableJoin::Locality::Global)
{
auto & table_expression = static_cast<ASTTableExpression &>(*node->table_expression);
addExternalStorage(table_expression.database_and_table_name
? table_expression.database_and_table_name
: table_expression.subquery);
}
}
}
@ -422,7 +432,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
}
else if (settings.global_subqueries_method == GlobalSubqueriesMethod::PULL)
{
String host_port = getFQDNOrHostName() + ":" + toString(context.getTCPPort());
/* String host_port = getFQDNOrHostName() + ":" + toString(context.getTCPPort());
String database = "_query_" + context.getCurrentQueryId();
auto subquery = std::make_shared<ASTSubquery>();
@ -455,7 +465,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
args->children.push_back(database_lit);
auto table_lit = std::make_shared<ASTLiteral>(StringRange(), external_table_name);
args->children.push_back(table_lit);
args->children.push_back(table_lit);*/
}
else
throw Exception("Unknown global subqueries execution method", ErrorCodes::UNKNOWN_GLOBAL_SUBQUERIES_METHOD);
@ -493,7 +503,7 @@ void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels)
/// Алиасы верхнего уровня в секции ARRAY JOIN имеют особый смысл, их добавлять не будем
/// (пропустим сам expression list и его детей).
if (select && child == select->array_join_expression_list)
if (select && child == select->array_join_expression_list())
new_ignore_levels = 2;
if (!typeid_cast<ASTSelectQuery *>(child.get()))
@ -518,12 +528,15 @@ StoragePtr ExpressionAnalyzer::getTable()
{
if (const ASTSelectQuery * select = typeid_cast<const ASTSelectQuery *>(ast.get()))
{
if (select->table && !typeid_cast<const ASTSelectQuery *>(select->table.get()) && !typeid_cast<const ASTFunction *>(select->table.get()))
auto select_database = select->database();
auto select_table = select->table();
if (select_table && !typeid_cast<const ASTSelectQuery *>(select_table.get()) && !typeid_cast<const ASTFunction *>(select_table.get()))
{
String database = select->database
? typeid_cast<const ASTIdentifier &>(*select->database).name
String database = select_database
? typeid_cast<const ASTIdentifier &>(*select_database).name
: "";
const String & table = typeid_cast<const ASTIdentifier &>(*select->table).name;
const String & table = typeid_cast<const ASTIdentifier &>(*select_table).name;
return context.tryGetTable(database, table);
}
}
@ -644,12 +657,17 @@ void ExpressionAnalyzer::normalizeTreeImpl(
}
}
}
else if (ASTJoin * node = typeid_cast<ASTJoin *>(ast.get()))
else if (ASTTablesInSelectQueryElement * node = typeid_cast<ASTTablesInSelectQueryElement *>(ast.get()))
{
if (node->table_expression)
{
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(
static_cast<ASTTableExpression &>(*node->table_expression).database_and_table_name.get()))
{
/// может быть указано JOIN t, где t - таблица, что равносильно JOIN (SELECT * FROM t).
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(node->table.get()))
right->kind = ASTIdentifier::Table;
}
}
}
/// Если заменили корень поддерева вызовемся для нового корня снова - на случай, если алиас заменился на алиас.
if (replaced)
@ -756,8 +774,7 @@ void ExpressionAnalyzer::executeScalarSubqueries()
for (auto & child : ast->children)
{
/// Не опускаемся в FROM, JOIN, UNION.
if (child.get() != select_query->table.get()
&& child.get() != select_query->join.get()
if (typeid_cast<const ASTTablesInSelectQuery *>(child.get())
&& child.get() != select_query->next_union_all.get())
{
executeScalarSubqueriesImpl(child);
@ -1103,8 +1120,7 @@ static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(
StringRange{}, column.name));
select_query->table = subquery_or_table_name;
select_query->children.emplace_back(select_query->table);
select_query->replaceDatabaseAndTable("", table->name);
}
else
{
@ -1479,9 +1495,9 @@ void ExpressionAnalyzer::getRootActions(ASTPtr ast, bool no_subqueries, bool onl
void ExpressionAnalyzer::getArrayJoinedColumns()
{
if (select_query && select_query->array_join_expression_list)
if (select_query && select_query->array_join_expression_list())
{
ASTs & array_join_asts = select_query->array_join_expression_list->children;
ASTs & array_join_asts = select_query->array_join_expression_list()->children;
for (const auto & ast : array_join_asts)
{
const String nested_table_name = ast->getColumnName();
@ -1504,7 +1520,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
if (typeid_cast<const ASTSelectQuery *>(ast.get()))
continue;
if (ast != select_query->array_join_expression_list)
if (ast != select_query->array_join_expression_list())
getArrayJoinedColumnsImpl(ast);
}
@ -1512,7 +1528,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
/// чтобы получить правильное количество строк.
if (array_join_result_to_source.empty())
{
ASTPtr expr = select_query->array_join_expression_list->children.at(0);
ASTPtr expr = select_query->array_join_expression_list()->children.at(0);
String source_name = expr->getColumnName();
String result_name = expr->getAliasOrColumnName();
@ -1969,20 +1985,20 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio
result_columns.insert(result_source.first);
}
actions->add(ExpressionAction::arrayJoin(result_columns, select_query->array_join_is_left));
actions->add(ExpressionAction::arrayJoin(result_columns, select_query->array_join_is_left()));
}
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
if (!select_query->array_join_expression_list)
if (!select_query->array_join_expression_list())
return false;
initChain(chain, columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->array_join_expression_list, only_types, false, step.actions);
getRootActions(select_query->array_join_expression_list(), only_types, false, step.actions);
addMultipleArrayJoinAction(step.actions);
@ -2003,27 +2019,29 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
{
assertSelect();
if (!select_query->join)
if (!select_query->join())
return false;
initChain(chain, columns);
ExpressionActionsChain::Step & step = chain.steps.back();
ASTJoin & ast_join = typeid_cast<ASTJoin &>(*select_query->join);
if (ast_join.using_expr_list)
getRootActions(ast_join.using_expr_list, only_types, false, step.actions);
const ASTTablesInSelectQueryElement & join_element = static_cast<const ASTTablesInSelectQueryElement &>(*select_query->join());
const ASTTableJoin & join_params = static_cast<const ASTTableJoin &>(*join_element.table_join);
const ASTTableExpression & table_to_join = static_cast<const ASTTableExpression &>(*join_element.table_expression);
if (join_params.using_expression_list)
getRootActions(join_params.using_expression_list, only_types, false, step.actions);
/// Не поддерживается два JOIN-а с одинаковым подзапросом, но разными USING-ами.
String join_id = ast_join.table->getColumnName();
String join_id = join_element.getColumnName();
SubqueryForSet & subquery_for_set = subqueries_for_sets[join_id];
/// Особый случай - если справа JOIN указано имя таблицы, при чём, таблица имеет тип Join (заранее подготовленное отображение).
/// TODO В этом синтаксисе не поддерживается указание имени БД.
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(ast_join.table.get());
if (identifier)
if (table_to_join.database_and_table_name)
{
StoragePtr table = context.tryGetTable("", identifier->name);
StoragePtr table = context.tryGetTable("", static_cast<const ASTIdentifier &>(*table_to_join.database_and_table_name).name);
if (table)
{
@ -2031,7 +2049,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
if (storage_join)
{
storage_join->assertCompatible(ast_join.kind, ast_join.strictness);
storage_join->assertCompatible(join_params.kind, join_params.strictness);
/// TODO Проверять набор ключей.
JoinPtr & join = storage_join->getJoin();
@ -2042,7 +2060,9 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
if (!subquery_for_set.join)
{
JoinPtr join = std::make_shared<Join>(join_key_names_left, join_key_names_right, settings.limits, ast_join.kind, ast_join.strictness);
JoinPtr join = std::make_shared<Join>(
join_key_names_left, join_key_names_right, settings.limits,
join_params.kind, join_params.strictness);
Names required_joined_columns(join_key_names_right.begin(), join_key_names_right.end());
for (const auto & name_type : columns_added_by_join)
@ -2055,7 +2075,13 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
*/
if (!subquery_for_set.source)
{
auto interpreter = interpretSubquery(ast_join.table, context, subquery_depth, required_joined_columns);
ASTPtr table;
if (table_to_join.database_and_table_name)
table = table_to_join.database_and_table_name;
else
table = table_to_join.subquery;
auto interpreter = interpretSubquery(table, context, subquery_depth, required_joined_columns);
subquery_for_set.source = std::make_shared<LazyBlockInputStream>([interpreter]() mutable { return interpreter->execute().in; });
subquery_for_set.source_sample = interpreter->getSampleBlock();
}
@ -2070,6 +2096,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
return true;
}
bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
@ -2309,9 +2336,9 @@ void ExpressionAnalyzer::collectUsedColumns()
NameSet required;
NameSet ignored;
if (select_query && select_query->array_join_expression_list)
if (select_query && select_query->array_join_expression_list())
{
ASTs & expressions = select_query->array_join_expression_list->children;
ASTs & expressions = select_query->array_join_expression_list()->children;
for (size_t i = 0; i < expressions.size(); ++i)
{
/// Игнорируем идентификаторы верхнего уровня из секции ARRAY JOIN.
@ -2399,26 +2426,32 @@ void ExpressionAnalyzer::collectUsedColumns()
void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type)
{
if (!select_query || !select_query->join)
if (!select_query)
return;
auto & node = typeid_cast<ASTJoin &>(*select_query->join);
const ASTTablesInSelectQueryElement * node = select_query->join();
if (!node)
return;
const ASTTableJoin & table_join = static_cast<const ASTTableJoin &>(*node->table_join);
const ASTTableExpression & table_expression = static_cast<const ASTTableExpression &>(*node->table_expression);
Block nested_result_sample;
if (const auto identifier = typeid_cast<const ASTIdentifier *>(node.table.get()))
if (table_expression.database_and_table_name)
{
const auto & table = context.getTable("", identifier->name);
const auto & table = context.getTable("", static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name).name);
nested_result_sample = table->getSampleBlockNonMaterialized();
}
else if (typeid_cast<const ASTSubquery *>(node.table.get()))
else if (table_expression.subquery)
{
const auto & subquery = node.table->children.at(0);
const auto & subquery = table_expression.subquery->children.at(0);
nested_result_sample = InterpreterSelectQuery::getSampleBlock(subquery, context);
}
if (node.using_expr_list)
if (table_join.using_expression_list)
{
auto & keys = typeid_cast<ASTExpressionList &>(*node.using_expr_list);
auto & keys = typeid_cast<ASTExpressionList &>(*table_join.using_expression_list);
for (const auto & key : keys.children)
{
if (join_key_names_left.end() == std::find(join_key_names_left.begin(), join_key_names_left.end(), key->getColumnName()))
@ -2547,7 +2580,7 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(ASTPtr ast,
* Туда collectUsedColumns отправит нас отдельно.
*/
if (!typeid_cast<ASTSubquery *>(child.get()) && !typeid_cast<ASTSelectQuery *>(child.get()) &&
!(select && child == select->array_join_expression_list))
!(select && child == select->array_join_expression_list()))
getRequiredColumnsImpl(child, required_columns, ignored_names, available_joined_columns, required_joined_columns);
}
}

View File

@ -111,21 +111,23 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi
void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
{
if (query.table && typeid_cast<ASTSelectQuery *>(query.table.get()))
auto query_table = query.table();
if (query_table && typeid_cast<ASTSelectQuery *>(query_table.get()))
{
if (table_column_names.empty())
{
table_column_names = InterpreterSelectQuery::getSampleBlock(query.table, context).getColumnsList();
table_column_names = InterpreterSelectQuery::getSampleBlock(query_table, context).getColumnsList();
}
}
else
{
if (query.table && typeid_cast<const ASTFunction *>(query.table.get()))
if (query_table && typeid_cast<const ASTFunction *>(query_table.get()))
{
/// Получить табличную функцию
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast<const ASTFunction *>(query.table.get())->name, context);
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast<const ASTFunction *>(query_table.get())->name, context);
/// Выполнить ее и запомнить результат
storage = table_function_ptr->execute(query.table, context);
storage = table_function_ptr->execute(query_table, context);
}
else
{
@ -268,20 +270,23 @@ void InterpreterSelectQuery::rewriteExpressionList(const Names & required_column
void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, String & table_name)
{
auto query_database = query.database();
auto query_table = query.table();
/** Если таблица не указана - используем таблицу system.one.
* Если база данных не указана - используем текущую базу данных.
*/
if (query.database)
database_name = typeid_cast<ASTIdentifier &>(*query.database).name;
if (query.table)
table_name = typeid_cast<ASTIdentifier &>(*query.table).name;
if (query_database)
database_name = typeid_cast<ASTIdentifier &>(*query_database).name;
if (query_table)
table_name = typeid_cast<ASTIdentifier &>(*query_table).name;
if (!query.table)
if (!query_table)
{
database_name = "system";
table_name = "one";
}
else if (!query.database)
else if (!query_database)
{
if (context.tryGetTable("", table_name))
database_name = "";
@ -447,8 +452,8 @@ void InterpreterSelectQuery::executeSingleQuery()
before_join = chain.getLastActions();
chain.addStep();
auto join = typeid_cast<const ASTJoin &>(*query.join);
if (join.kind == ASTJoin::Full || join.kind == ASTJoin::Right)
const ASTTableJoin & join = static_cast<const ASTTableJoin &>(*query.join()->table_join);
if (join.kind == ASTTableJoin::Kind::Full || join.kind == ASTTableJoin::Kind::Right)
stream_with_non_joined_data = before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(settings.max_block_size);
}
@ -698,7 +703,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
}
}
if (query.table && typeid_cast<ASTSelectQuery *>(query.table.get()))
auto query_table = query.table();
if (query_table && typeid_cast<ASTSelectQuery *>(query_table.get()))
{
/** Для подзапроса не действуют ограничения на максимальный размер результата.
* Так как результат поздапроса - ещё не результат всего запроса.
@ -712,17 +718,17 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
subquery_context.setSettings(subquery_settings);
interpreter_subquery.emplace(
query.table, subquery_context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1);
query_table, subquery_context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1);
/// Если во внешнем запросе есть аггрегация, то WITH TOTALS игнорируется в подзапросе.
if (query_analyzer->hasAggregation())
interpreter_subquery->ignoreWithTotals();
}
if (query.sample_size && (!storage || !storage->supportsSampling()))
if (query.sample_size() && (!storage || !storage->supportsSampling()))
throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED);
if (query.final && (!storage || !storage->supportsFinal()))
if (query.final() && (!storage || !storage->supportsFinal()))
throw Exception(storage ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL);
if (query.prewhere_expression && (!storage || !storage->supportsPrewhere()))
@ -1174,11 +1180,12 @@ void InterpreterSelectQuery::executeLimit()
always_read_till_end = true;
}
if (!query.group_by_with_totals && query.table && typeid_cast<const ASTSelectQuery *>(query.table.get()))
auto query_table = query.table();
if (!query.group_by_with_totals && query_table && typeid_cast<const ASTSelectQuery *>(query_table.get()))
{
const ASTSelectQuery * subquery = static_cast<const ASTSelectQuery *>(query.table.get());
const ASTSelectQuery * subquery = static_cast<const ASTSelectQuery *>(query_table.get());
while (subquery->table)
while (subquery->table())
{
if (subquery->group_by_with_totals)
{
@ -1190,8 +1197,9 @@ void InterpreterSelectQuery::executeLimit()
break;
}
if (typeid_cast<const ASTSelectQuery *>(subquery->table.get()))
subquery = static_cast<const ASTSelectQuery *>(subquery->table.get());
auto subquery_table = subquery->table();
if (typeid_cast<const ASTSelectQuery *>(subquery_table.get()))
subquery = static_cast<const ASTSelectQuery *>(subquery_table.get());
else
break;
}

View File

@ -1,7 +1,6 @@
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Parsers/ASTJoin.h>
#include <DB/Interpreters/Join.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Core/ColumnNumbers.h>
@ -105,9 +104,9 @@ static size_t getTotalByteCountImpl(const Maps & maps)
/// Нужно ли использовать хэш-таблицы maps_*_full, в которых запоминается, была ли строчка присоединена.
static bool getFullness(ASTJoin::Kind kind)
static bool getFullness(ASTTableJoin::Kind kind)
{
return kind == ASTJoin::Right || kind == ASTJoin::Full;
return kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full;
}
@ -115,19 +114,19 @@ void Join::init(Type type_)
{
type = type_;
if (kind == ASTJoin::Cross)
if (kind == ASTTableJoin::Kind::Cross)
return;
if (!getFullness(kind))
{
if (strictness == ASTJoin::Any)
if (strictness == ASTTableJoin::Strictness::Any)
initImpl(maps_any, type);
else
initImpl(maps_all, type);
}
else
{
if (strictness == ASTJoin::Any)
if (strictness == ASTTableJoin::Strictness::Any)
initImpl(maps_any_full, type);
else
initImpl(maps_all_full, type);
@ -187,14 +186,14 @@ bool Join::checkSizeLimits() const
/// Вставка элемента в хэш-таблицу вида ключ -> ссылка на строку, которая затем будет использоваться при JOIN-е.
template <ASTJoin::Strictness STRICTNESS, typename Map>
template <ASTTableJoin::Strictness STRICTNESS, typename Map>
struct Inserter
{
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool);
};
template <typename Map>
struct Inserter<ASTJoin::Any, Map>
struct Inserter<ASTTableJoin::Strictness::Any, Map>
{
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool)
{
@ -225,12 +224,12 @@ struct InserterAnyString
}
};
template <> struct Inserter<ASTJoin::Any, Join::MapsAny::MapString> : InserterAnyString<Join::MapsAny::MapString> {};
template <> struct Inserter<ASTJoin::Any, Join::MapsAnyFull::MapString> : InserterAnyString<Join::MapsAnyFull::MapString> {};
template <> struct Inserter<ASTTableJoin::Strictness::Any, Join::MapsAny::MapString> : InserterAnyString<Join::MapsAny::MapString> {};
template <> struct Inserter<ASTTableJoin::Strictness::Any, Join::MapsAnyFull::MapString> : InserterAnyString<Join::MapsAnyFull::MapString> {};
template <typename Map>
struct Inserter<ASTJoin::All, Map>
struct Inserter<ASTTableJoin::Strictness::All, Map>
{
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool)
{
@ -284,11 +283,11 @@ struct InserterAllString
}
};
template <> struct Inserter<ASTJoin::All, Join::MapsAll::MapString> : InserterAllString<Join::MapsAll::MapString> {};
template <> struct Inserter<ASTJoin::All, Join::MapsAllFull::MapString> : InserterAllString<Join::MapsAllFull::MapString> {};
template <> struct Inserter<ASTTableJoin::Strictness::All, Join::MapsAll::MapString> : InserterAllString<Join::MapsAll::MapString> {};
template <> struct Inserter<ASTTableJoin::Strictness::All, Join::MapsAllFull::MapString> : InserterAllString<Join::MapsAllFull::MapString> {};
template <ASTJoin::Strictness STRICTNESS, typename Maps>
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
void Join::insertFromBlockImpl(Maps & maps, size_t rows, const ConstColumnPlainPtrs & key_columns, size_t keys_size, Block * stored_block)
{
if (type == Type::CROSS)
@ -466,22 +465,22 @@ bool Join::insertFromBlock(const Block & block)
stored_block->getByPosition(i).column = converted;
}
if (kind != ASTJoin::Cross)
if (kind != ASTTableJoin::Kind::Cross)
{
/// Заполняем нужную хэш-таблицу.
if (!getFullness(kind))
{
if (strictness == ASTJoin::Any)
insertFromBlockImpl<ASTJoin::Any>(maps_any, rows, key_columns, keys_size, stored_block);
if (strictness == ASTTableJoin::Strictness::Any)
insertFromBlockImpl<ASTTableJoin::Strictness::Any>(maps_any, rows, key_columns, keys_size, stored_block);
else
insertFromBlockImpl<ASTJoin::All>(maps_all, rows, key_columns, keys_size, stored_block);
insertFromBlockImpl<ASTTableJoin::Strictness::All>(maps_all, rows, key_columns, keys_size, stored_block);
}
else
{
if (strictness == ASTJoin::Any)
insertFromBlockImpl<ASTJoin::Any>(maps_any_full, rows, key_columns, keys_size, stored_block);
if (strictness == ASTTableJoin::Strictness::Any)
insertFromBlockImpl<ASTTableJoin::Strictness::Any>(maps_any_full, rows, key_columns, keys_size, stored_block);
else
insertFromBlockImpl<ASTJoin::All>(maps_all_full, rows, key_columns, keys_size, stored_block);
insertFromBlockImpl<ASTTableJoin::Strictness::All>(maps_all_full, rows, key_columns, keys_size, stored_block);
}
}
@ -505,11 +504,11 @@ bool Join::insertFromBlock(const Block & block)
}
template <ASTJoin::Kind KIND, ASTJoin::Strictness STRICTNESS, typename Map>
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Map>
struct Adder;
template <typename Map>
struct Adder<ASTJoin::Left, ASTJoin::Any, Map>
struct Adder<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any, Map>
{
static void add(const Map & map, const typename Map::key_type & key, size_t num_columns_to_add, ColumnPlainPtrs & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset_t & current_offset, IColumn::Offsets_t * offsets,
@ -532,7 +531,7 @@ struct Adder<ASTJoin::Left, ASTJoin::Any, Map>
};
template <typename Map>
struct Adder<ASTJoin::Inner, ASTJoin::Any, Map>
struct Adder<ASTTableJoin::Kind::Inner, ASTTableJoin::Strictness::Any, Map>
{
static void add(const Map & map, const typename Map::key_type & key, size_t num_columns_to_add, ColumnPlainPtrs & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset_t & current_offset, IColumn::Offsets_t * offsets,
@ -553,8 +552,8 @@ struct Adder<ASTJoin::Inner, ASTJoin::Any, Map>
}
};
template <ASTJoin::Kind KIND, typename Map>
struct Adder<KIND, ASTJoin::All, Map>
template <ASTTableJoin::Kind KIND, typename Map>
struct Adder<KIND, ASTTableJoin::Strictness::All, Map>
{
static void add(const Map & map, const typename Map::key_type & key, size_t num_columns_to_add, ColumnPlainPtrs & added_columns,
size_t i, IColumn::Filter * filter, IColumn::Offset_t & current_offset, IColumn::Offsets_t * offsets,
@ -579,7 +578,7 @@ struct Adder<KIND, ASTJoin::All, Map>
}
else
{
if (KIND == ASTJoin::Inner)
if (KIND == ASTTableJoin::Kind::Inner)
{
(*offsets)[i] = current_offset;
}
@ -596,7 +595,7 @@ struct Adder<KIND, ASTJoin::All, Map>
};
template <ASTJoin::Kind KIND, ASTJoin::Strictness STRICTNESS, typename Maps>
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
void Join::joinBlockImpl(Block & block, const Maps & maps) const
{
size_t keys_size = key_names_left.size();
@ -652,14 +651,14 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
/// Используется при ANY INNER JOIN
std::unique_ptr<IColumn::Filter> filter;
if ((kind == ASTJoin::Inner || kind == ASTJoin::Right) && strictness == ASTJoin::Any)
if ((kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right) && strictness == ASTTableJoin::Strictness::Any)
filter.reset(new IColumn::Filter(rows));
/// Используется при ALL ... JOIN
IColumn::Offset_t current_offset = 0;
std::unique_ptr<IColumn::Offsets_t> offsets_to_replicate;
if (strictness == ASTJoin::All)
if (strictness == ASTTableJoin::Strictness::All)
offsets_to_replicate.reset(new IColumn::Offsets_t(rows));
/** Для LEFT/INNER JOIN, сохранённые блоки не содержат ключи.
@ -830,23 +829,23 @@ void Join::joinBlock(Block & block) const
checkTypesOfKeys(block, sample_block_with_keys);
if (kind == ASTJoin::Left && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Left, ASTJoin::Any>(block, maps_any);
else if (kind == ASTJoin::Inner && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Inner, ASTJoin::Any>(block, maps_any);
else if (kind == ASTJoin::Left && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Left, ASTJoin::All>(block, maps_all);
else if (kind == ASTJoin::Inner && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Inner, ASTJoin::All>(block, maps_all);
else if (kind == ASTJoin::Full && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Left, ASTJoin::Any>(block, maps_any_full);
else if (kind == ASTJoin::Right && strictness == ASTJoin::Any)
joinBlockImpl<ASTJoin::Inner, ASTJoin::Any>(block, maps_any_full);
else if (kind == ASTJoin::Full && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Left, ASTJoin::All>(block, maps_all_full);
else if (kind == ASTJoin::Right && strictness == ASTJoin::All)
joinBlockImpl<ASTJoin::Inner, ASTJoin::All>(block, maps_all_full);
else if (kind == ASTJoin::Cross)
if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any)
joinBlockImpl<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any>(block, maps_any);
else if (kind == ASTTableJoin::Kind::Inner && strictness == ASTTableJoin::Strictness::Any)
joinBlockImpl<ASTTableJoin::Kind::Inner, ASTTableJoin::Strictness::Any>(block, maps_any);
else if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::All)
joinBlockImpl<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::All>(block, maps_all);
else if (kind == ASTTableJoin::Kind::Inner && strictness == ASTTableJoin::Strictness::All)
joinBlockImpl<ASTTableJoin::Kind::Inner, ASTTableJoin::Strictness::All>(block, maps_all);
else if (kind == ASTTableJoin::Kind::Full && strictness == ASTTableJoin::Strictness::Any)
joinBlockImpl<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any>(block, maps_any_full);
else if (kind == ASTTableJoin::Kind::Right && strictness == ASTTableJoin::Strictness::Any)
joinBlockImpl<ASTTableJoin::Kind::Inner, ASTTableJoin::Strictness::Any>(block, maps_any_full);
else if (kind == ASTTableJoin::Kind::Full && strictness == ASTTableJoin::Strictness::All)
joinBlockImpl<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::All>(block, maps_all_full);
else if (kind == ASTTableJoin::Kind::Right && strictness == ASTTableJoin::Strictness::All)
joinBlockImpl<ASTTableJoin::Kind::Inner, ASTTableJoin::Strictness::All>(block, maps_all_full);
else if (kind == ASTTableJoin::Kind::Cross)
joinBlockImplCross(block);
else
throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR);
@ -879,11 +878,11 @@ void Join::joinTotals(Block & block) const
}
template <ASTJoin::Strictness STRICTNESS, typename Mapped>
template <ASTTableJoin::Strictness STRICTNESS, typename Mapped>
struct AdderNonJoined;
template <typename Mapped>
struct AdderNonJoined<ASTJoin::Any, Mapped>
struct AdderNonJoined<ASTTableJoin::Strictness::Any, Mapped>
{
static void add(const Mapped & mapped,
size_t num_columns_left, ColumnPlainPtrs & columns_left,
@ -898,7 +897,7 @@ struct AdderNonJoined<ASTJoin::Any, Mapped>
};
template <typename Mapped>
struct AdderNonJoined<ASTJoin::All, Mapped>
struct AdderNonJoined<ASTTableJoin::Strictness::All, Mapped>
{
static void add(const Mapped & mapped,
size_t num_columns_left, ColumnPlainPtrs & columns_left,
@ -980,10 +979,10 @@ protected:
if (parent.blocks.empty())
return Block();
if (parent.strictness == ASTJoin::Any)
return createBlock<ASTJoin::Any>(parent.maps_any_full);
else if (parent.strictness == ASTJoin::All)
return createBlock<ASTJoin::All>(parent.maps_all_full);
if (parent.strictness == ASTTableJoin::Strictness::Any)
return createBlock<ASTTableJoin::Strictness::Any>(parent.maps_any_full);
else if (parent.strictness == ASTTableJoin::Strictness::All)
return createBlock<ASTTableJoin::Strictness::All>(parent.maps_all_full);
else
throw Exception("Logical error: unknown JOIN strictness (must be ANY or ALL)", ErrorCodes::LOGICAL_ERROR);
}
@ -1001,7 +1000,7 @@ private:
std::unique_ptr<void, std::function<void(void *)>> position; /// type erasure
template <ASTJoin::Strictness STRICTNESS, typename Maps>
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
Block createBlock(const Maps & maps)
{
Block block = result_sample_block.cloneEmpty();
@ -1048,7 +1047,7 @@ private:
}
template <ASTJoin::Strictness STRICTNESS, typename Map>
template <ASTTableJoin::Strictness STRICTNESS, typename Map>
size_t fillColumns(const Map & map,
size_t num_columns_left, ColumnPlainPtrs & columns_left,
size_t num_columns_right, ColumnPlainPtrs & columns_right)

View File

@ -347,6 +347,22 @@ static const ASTTableExpression * getFirstTableExpression(const ASTSelectQuery &
return static_cast<const ASTTableExpression *>(tables_element.table_expression.get());
}
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select)
{
if (!select.tables)
return {};
ASTTablesInSelectQuery & tables_in_select_query = static_cast<ASTTablesInSelectQuery &>(*select.tables);
if (tables_in_select_query.children.empty())
return {};
ASTTablesInSelectQueryElement & tables_element = static_cast<ASTTablesInSelectQueryElement &>(*tables_in_select_query.children[0]);
if (!tables_element.table_expression)
return {};
return static_cast<ASTTableExpression *>(tables_element.table_expression.get());
}
static const ASTArrayJoin * getFirstArrayJoin(const ASTSelectQuery & select)
{
if (!select.tables)
@ -493,5 +509,50 @@ const ASTTablesInSelectQueryElement * ASTSelectQuery::join() const
return getFirstTableJoin(*this);
}
void ASTSelectQuery::setDatabaseIfNeeded(const String & database_name)
{
ASTTableExpression * table_expression = getFirstTableExpression(*this);
if (!table_expression)
return;
if (!table_expression->database_and_table_name)
return;
if (table_expression->database_and_table_name->children.empty())
{
ASTPtr database = std::make_shared<ASTIdentifier>(StringRange(), database_name, ASTIdentifier::Database);
ASTPtr table = table_expression->database_and_table_name;
const String & old_name = static_cast<ASTIdentifier &>(*table_expression->database_and_table_name).name;
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(StringRange(), database_name + "." + old_name, ASTIdentifier::Table);
table_expression->database_and_table_name->children = {database, table};
}
else if (table_expression->database_and_table_name->children.size() != 2)
{
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
}
else
{
table_expression->database_and_table_name->children[0] = std::make_shared<ASTIdentifier>(
StringRange(), database_name, ASTIdentifier::Database);
}
}
void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const String & table_name)
{
ASTTableExpression * table_expression = getFirstTableExpression(*this);
if (!table_expression)
return;
ASTPtr database = std::make_shared<ASTIdentifier>(StringRange(), database_name, ASTIdentifier::Database);
ASTPtr table = std::make_shared<ASTIdentifier>(StringRange(), table_name, ASTIdentifier::Table);
table_expression->database_and_table_name = std::make_shared<ASTIdentifier>(
StringRange(), database_name + "." + table_name, ASTIdentifier::Table);
table_expression->database_and_table_name->children = {database, table};
}
};

View File

@ -240,20 +240,23 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
ASTSelectQuery & select = *typeid_cast<ASTSelectQuery*>(&*query);
if (select.sample_size)
auto select_sample_size = select.sample_size();
auto select_sample_offset = select.sample_offset();
if (select_sample_size)
{
relative_sample_size.assign(
typeid_cast<const ASTSampleRatio &>(*select.sample_size).ratio.numerator,
typeid_cast<const ASTSampleRatio &>(*select.sample_size).ratio.denominator);
typeid_cast<const ASTSampleRatio &>(*select_sample_size).ratio.numerator,
typeid_cast<const ASTSampleRatio &>(*select_sample_size).ratio.denominator);
if (relative_sample_size < 0)
throw Exception("Negative sample size", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
relative_sample_offset = 0;
if (select.sample_offset)
if (select_sample_offset)
relative_sample_offset.assign(
typeid_cast<const ASTSampleRatio &>(*select.sample_offset).ratio.numerator,
typeid_cast<const ASTSampleRatio &>(*select.sample_offset).ratio.denominator);
typeid_cast<const ASTSampleRatio &>(*select_sample_offset).ratio.numerator,
typeid_cast<const ASTSampleRatio &>(*select_sample_offset).ratio.denominator);
if (relative_sample_offset < 0)
throw Exception("Negative sample offset", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
@ -265,7 +268,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
if (relative_sample_size > 1)
{
relative_sample_size = convertAbsoluteSampleSizeToRelative(select.sample_size, approx_total_rows);
relative_sample_size = convertAbsoluteSampleSizeToRelative(select_sample_size, approx_total_rows);
LOG_DEBUG(log, "Selected relative sample size: " << relative_sample_size);
}
@ -278,7 +281,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
if (relative_sample_offset > 1)
{
relative_sample_offset = convertAbsoluteSampleSizeToRelative(select.sample_offset, approx_total_rows);
relative_sample_offset = convertAbsoluteSampleSizeToRelative(select_sample_offset, approx_total_rows);
LOG_DEBUG(log, "Selected relative sample offset: " << relative_sample_offset);
}
}
@ -512,7 +515,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
BlockInputStreams res;
if (select.final)
if (select.final())
{
/// Добавим столбцы, нужные для вычисления первичного ключа и знака.
std::vector<String> add_columns = data.getPrimaryExpression()->getRequiredColumns();

View File

@ -53,11 +53,7 @@ namespace
inline ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, const std::string & table)
{
auto modified_query_ast = query->clone();
auto & actual_query = typeid_cast<ASTSelectQuery &>(*modified_query_ast);
actual_query.database = std::make_shared<ASTIdentifier>(StringRange(), database, ASTIdentifier::Database);
actual_query.table = std::make_shared<ASTIdentifier>(StringRange(), table, ASTIdentifier::Table);
typeid_cast<ASTSelectQuery &>(*modified_query_ast).replaceDatabaseAndTable(database, table);
return modified_query_ast;
}

View File

@ -281,11 +281,11 @@ StoragePtr StorageFactory::get(
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const String strictness_str = Poco::toLower(strictness_id->name);
ASTJoin::Strictness strictness;
ASTTableJoin::Strictness strictness;
if (strictness_str == "any")
strictness = ASTJoin::Strictness::Any;
strictness = ASTTableJoin::Strictness::Any;
else if (strictness_str == "all")
strictness = ASTJoin::Strictness::All;
strictness = ASTTableJoin::Strictness::All;
else
throw Exception("First parameter of storage Join must be ANY or ALL (without quotes).", ErrorCodes::BAD_ARGUMENTS);
@ -294,15 +294,15 @@ StoragePtr StorageFactory::get(
throw Exception("Second parameter of storage Join must be LEFT or INNER (without quotes).", ErrorCodes::BAD_ARGUMENTS);
const String kind_str = Poco::toLower(kind_id->name);
ASTJoin::Kind kind;
ASTTableJoin::Kind kind;
if (kind_str == "left")
kind = ASTJoin::Kind::Left;
kind = ASTTableJoin::Kind::Left;
else if (kind_str == "inner")
kind = ASTJoin::Kind::Inner;
kind = ASTTableJoin::Kind::Inner;
else if (kind_str == "right")
kind = ASTJoin::Kind::Right;
kind = ASTTableJoin::Kind::Right;
else if (kind_str == "full")
kind = ASTJoin::Kind::Full;
kind = ASTTableJoin::Kind::Full;
else
throw Exception("Second parameter of storage Join must be LEFT or INNER or RIGHT or FULL (without quotes).", ErrorCodes::BAD_ARGUMENTS);

View File

@ -15,7 +15,7 @@ StorageJoin::StorageJoin(
const String & path_,
const String & name_,
const Names & key_names_,
ASTJoin::Kind kind_, ASTJoin::Strictness strictness_,
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
@ -23,7 +23,7 @@ StorageJoin::StorageJoin(
: StorageSetOrJoinBase{path_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_},
key_names(key_names_), kind(kind_), strictness(strictness_)
{
/// Проверяем, что ключ существует в определении таблицы.
/// Check that key exists in table definition.
const auto check_key_exists = [] (const NamesAndTypesList & columns, const String & key)
{
for (const auto & column : columns)
@ -44,9 +44,9 @@ StorageJoin::StorageJoin(
}
void StorageJoin::assertCompatible(ASTJoin::Kind kind_, ASTJoin::Strictness strictness_) const
void StorageJoin::assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const
{
/// NOTE Можно немного ослабить.
/// NOTE Could be more loose.
if (!(kind == kind_ && strictness == strictness_))
throw Exception("Table " + name + " has incompatible type of JOIN.", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN);
}

View File

@ -130,7 +130,7 @@ BlockInputStreams StorageMergeTree::read(
auto & select = typeid_cast<const ASTSelectQuery &>(*query);
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
if (settings.optimize_move_to_prewhere && select.where_expression && !select.prewhere_expression && !select.final)
if (settings.optimize_move_to_prewhere && select.where_expression && !select.prewhere_expression && !select.final())
MergeTreeWhereOptimizer{query, context, data, column_names, log};
return reader.read(column_names, query, context, settings, processed_stage, max_block_size, threads, nullptr, 0);
@ -157,7 +157,7 @@ void StorageMergeTree::rename(const String & new_path_to_db, const String & new_
table_name = new_table_name;
full_path = new_full_path;
/// TODO: Можно обновить названия логгеров у this, data, reader, writer, merger.
/// NOTE: Logger names are not updated.
}
void StorageMergeTree::alter(

View File

@ -2090,7 +2090,7 @@ BlockInputStreams StorageReplicatedMergeTree::read(
auto & select = typeid_cast<const ASTSelectQuery &>(*query);
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
if (settings.optimize_move_to_prewhere && select.where_expression && !select.prewhere_expression && !select.final)
if (settings.optimize_move_to_prewhere && select.where_expression && !select.prewhere_expression && !select.final())
MergeTreeWhereOptimizer{query, context, data, real_column_names, log};
Block virtual_columns_block;

View File

@ -48,11 +48,7 @@ StorageView::StorageView(
ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*create.select);
/// Если во внутреннем запросе не указана база данных, получить ее из контекста и записать в запрос.
if (!select.database)
{
select.database = std::make_shared<ASTIdentifier>(StringRange(), database_name_, ASTIdentifier::Database);
select.children.push_back(select.database);
}
select.setDatabaseIfNeeded(database_name);
inner_query = select;
@ -67,20 +63,24 @@ StorageView::StorageView(
void StorageView::extractDependentTable(const ASTSelectQuery & query)
{
if (!query.table)
auto query_table = query.table();
if (!query_table)
return;
if (const ASTIdentifier * ast_id = typeid_cast<const ASTIdentifier *>(query.table.get()))
if (const ASTIdentifier * ast_id = typeid_cast<const ASTIdentifier *>(query_table.get()))
{
if (!query.database)
auto query_database = query.database();
if (!query_database)
throw Exception("Logical error while creating StorageView."
" Could not retrieve database name from select query.",
DB::ErrorCodes::LOGICAL_ERROR);
select_database_name = typeid_cast<const ASTIdentifier &>(*query.database).name;
select_database_name = typeid_cast<const ASTIdentifier &>(*query_database).name;
select_table_name = ast_id->name;
}
else if (const ASTSelectQuery * ast_select = typeid_cast<const ASTSelectQuery *>(query.table.get()))
else if (const ASTSelectQuery * ast_select = typeid_cast<const ASTSelectQuery *>(query_table.get()))
{
extractDependentTable(*ast_select);
}
@ -103,10 +103,10 @@ BlockInputStreams StorageView::read(
processed_stage = QueryProcessingStage::FetchColumns;
ASTPtr inner_query_clone = getInnerQuery();
ASTSelectQuery & inner_select = static_cast<ASTSelectQuery &>(*inner_query_clone);
/* ASTSelectQuery & inner_select = static_cast<ASTSelectQuery &>(*inner_query_clone);
const ASTSelectQuery & outer_select = typeid_cast<const ASTSelectQuery &>(*query);
/// Пробрасываем внутрь SAMPLE и FINAL, если они есть во внешнем запросе и их нет во внутреннем.
/// Пробрасываем внутрь SAMPLE и FINAL, если они есть во внешнем запросе и их нет во внутреннем. TODO
if (outer_select.sample_size && !inner_select.sample_size)
{
@ -117,7 +117,7 @@ BlockInputStreams StorageView::read(
}
if (outer_select.final && !inner_select.final)
inner_select.final = outer_select.final;
inner_select.final = outer_select.final;*/
return InterpreterSelectQuery(inner_query_clone, context, column_names).executeWithoutUnion();
}