dbms: GLOBAL JOINs: development [#METR-11370].

This commit is contained in:
Alexey Milovidov 2014-07-04 00:32:42 +04:00
parent c8fde6537f
commit 2fa4a2cf60
4 changed files with 31 additions and 23 deletions

View File

@ -37,7 +37,7 @@ public:
Strictness strictness = Any;
Kind kind = Inner;
ASTPtr subquery; /// Подзапрос - "правая" таблица для соединения.
ASTPtr table; /// "Правая" таблица для соединения - подзапрос или имя таблицы.
ASTPtr using_expr_list; /// По каким столбцам выполнять соединение.
ASTJoin() {}
@ -66,7 +66,7 @@ public:
ASTJoin * res = new ASTJoin(*this);
res->children.clear();
if (subquery) { res->subquery = subquery->clone(); res->children.push_back(res->subquery); }
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;

View File

@ -52,21 +52,13 @@ static std::string * tryGetAlias(ASTPtr & ast)
static void setAlias(ASTPtr & ast, const std::string & alias)
{
if (ASTFunction * node = typeid_cast<ASTFunction *>(&*ast))
{
node->alias = alias;
}
else if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(&*ast))
{
node->alias = alias;
}
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(&*ast))
{
node->alias = alias;
}
else
{
throw Exception("Can't set alias of " + ast->getColumnName(), ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
}
}
@ -247,7 +239,8 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
current_asts.insert(ast);
replaced = true;
}
/// может быть указано in t, где t - таблица, что равносильно select * from t.
/// может быть указано IN t, где t - таблица, что равносильно IN (SELECT * FROM t).
if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn")
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(&*node->arguments->children[1]))
right->kind = ASTIdentifier::Table;
@ -351,6 +344,16 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
addExternalStorage(node);
}
if (ASTJoin * node = typeid_cast<ASTJoin *>(&*ast))
{
/// может быть указано JOIN t, где t - таблица, что равносильно JOIN (SELECT * FROM t).
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(&*node->table))
right->kind = ASTIdentifier::Table;
/* if (do_global && node->locality == ASTJoin::Global)
addExternalStorage(node->table);*/
}
current_asts.erase(initial_ast);
current_asts.erase(ast);
finished_asts[initial_ast] = ast;
@ -433,10 +436,12 @@ void ExpressionAnalyzer::addExternalStorage(ASTFunction * node)
/// Если подзапрос или имя таблицы для селекта
if (typeid_cast<const ASTSubquery *>(&*arg) || typeid_cast<const ASTIdentifier *>(&*arg))
{
/** Для подзапроса в секции IN не действуют ограничения на максимальный размер результата.
* Так как результат этого поздапроса - ещё не результат всего запроса.
* Вместо этого работают ограничения max_rows_in_set, max_bytes_in_set, set_overflow_mode.
*/
/** Для подзапроса в секции IN/JOIN не действуют ограничения на максимальный размер результата.
* Так как результат этого поздапроса - ещё не результат всего запроса.
* Вместо этого работают ограничения
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
* max_rows_in_join, max_bytes_in_join, join_overflow_mode.
*/
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.limits.max_result_rows = 0;
@ -467,7 +472,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTFunction * node)
bool parse_res = parser.parse(pos, end, subquery, expected);
if (!parse_res)
throw Exception("Error in parsing SELECT query while creating set for table " + table->name + ".",
throw Exception("Error in parsing SELECT query while creating set or join for table " + table->name + ".",
ErrorCodes::LOGICAL_ERROR);
}
else
@ -497,7 +502,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTFunction * node)
sets_with_subqueries[ast_set->getColumnName()] = ast_set->set;
}
else
throw Exception("GLOBAL [NOT] IN supports only SELECT data.", ErrorCodes::BAD_ARGUMENTS);
throw Exception("IN/JOIN supports only SELECT subqueries.", ErrorCodes::BAD_ARGUMENTS);
}
@ -1277,8 +1282,9 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
for (const auto & name_type : columns_added_by_join)
required_joined_columns.push_back(name_type.first);
/// TODO: поддержка идентификаторов вместо подзапросов.
InterpreterSelectQuery interpreter(
typeid_cast<ASTJoin &>(*select_query->join).subquery->children[0], subquery_context,
typeid_cast<ASTJoin &>(*select_query->join).table->children[0], subquery_context,
required_joined_columns,
QueryProcessingStage::Complete, subquery_depth + 1);
@ -1648,7 +1654,7 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
auto & node = typeid_cast<ASTJoin &>(*select_query->join);
auto & keys = typeid_cast<ASTExpressionList &>(*node.using_expr_list);
auto & select = node.subquery->children[0];
auto & table = node.table->children[0]; /// TODO: поддержка идентификаторов.
size_t num_join_keys = keys.children.size();
@ -1661,7 +1667,7 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
throw Exception("Duplicate column in USING list", ErrorCodes::DUPLICATE_COLUMN);
}
Block nested_result_sample = ExpressionAnalyzer(select, context, subquery_depth + 1).getSelectSampleBlock();
Block nested_result_sample = ExpressionAnalyzer(table, context, subquery_depth + 1).getSelectSampleBlock();
size_t nested_result_columns = nested_result_sample.columns();
for (size_t i = 0; i < nested_result_columns; ++i)

View File

@ -27,6 +27,7 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expecte
ParserNotEmptyExpressionList exp_list;
ParserSubquery subquery;
ParserIdentifier identifier;
ws.ignore(pos, end);
@ -66,7 +67,8 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expecte
ws.ignore(pos, end);
if (!subquery.parse(pos, end, join->subquery, expected))
if (!identifier.parse(pos, end, join->table, expected)
&& !subquery.parse(pos, end, join->table, expected))
return false;
ws.ignore(pos, end);
@ -81,7 +83,7 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expecte
ws.ignore(pos, end);
join->children.push_back(join->subquery);
join->children.push_back(join->table);
join->children.push_back(join->using_expr_list);
return true;

View File

@ -754,7 +754,7 @@ void formatAST(const ASTJoin & ast, std::ostream & s, size_t indent, bool hilite
<< "JOIN"
<< (hilite ? hilite_none : "");
formatAST(*ast.subquery, s, indent, hilite, one_line, need_parens);
formatAST(*ast.table, s, indent, hilite, one_line, need_parens);
s << (hilite ? hilite_keyword : "") << " USING " << (hilite ? hilite_none : "");