mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
get rid of DNF and related features in ORs in JOIN
This commit is contained in:
parent
1dc7fc5962
commit
7ebc16c1b3
@ -77,7 +77,6 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
static JoinIdentifierPos getTableForIdentifiers(const ASTPtr & ast, bool throw_on_table_mix, const Data & data);
|
||||
private:
|
||||
static void visit(const ASTFunction & func, const ASTPtr & ast, Data & data);
|
||||
static void visit(const ASTIdentifier & ident, const ASTPtr & ast, Data & data);
|
||||
@ -85,7 +84,7 @@ private:
|
||||
static void getIdentifiers(const ASTPtr & ast, std::vector<const ASTIdentifier *> & out);
|
||||
static JoinIdentifierPosPair getTableNumbers(const ASTPtr & left_ast, const ASTPtr & right_ast, Data & data);
|
||||
static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases);
|
||||
|
||||
static JoinIdentifierPos getTableForIdentifiers(const ASTPtr & ast, bool throw_on_table_mix, const Data & data);
|
||||
};
|
||||
|
||||
/// Parse JOIN ON expression and collect ASTs for joined columns.
|
||||
|
@ -36,7 +36,6 @@ struct RequiredSourceColumnsData
|
||||
|
||||
bool has_table_join = false;
|
||||
bool has_array_join = false;
|
||||
bool converted_to_dnf = false;
|
||||
|
||||
bool addColumnAliasIfAny(const IAST & ast);
|
||||
void addColumnIdentifier(const ASTIdentifier & node);
|
||||
|
@ -178,14 +178,8 @@ void RequiredSourceColumnsMatcher::visit(const ASTFunction & node, const ASTPtr
|
||||
void RequiredSourceColumnsMatcher::visit(const ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
for (const auto & child : node.children)
|
||||
{
|
||||
const auto * join_node = child->as<ASTTableJoin>();
|
||||
if (join_node)
|
||||
{
|
||||
if (child->as<ASTTableJoin>())
|
||||
data.has_table_join = true;
|
||||
data.converted_to_dnf = join_node->converted_to_dnf;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// ASTIdentifiers here are tables. Do not visit them as generic ones.
|
||||
|
@ -125,113 +125,6 @@ void TableJoin::addUsingKey(const ASTPtr & ast)
|
||||
addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast);
|
||||
}
|
||||
|
||||
void TableJoin::newClauseIfPopulated()
|
||||
{
|
||||
const auto & clause = clauses.back();
|
||||
if (!clause.key_names_left.empty() || !clause.key_names_right.empty() ||
|
||||
clause.on_filter_condition_left || clause.on_filter_condition_right)
|
||||
{
|
||||
clauses.emplace_back();
|
||||
}
|
||||
if (getStorageJoin() && clauses.size() > 1)
|
||||
throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
bool compatibleFilerConditions(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r)
|
||||
{
|
||||
bool has_left = l.on_filter_condition_left || r.on_filter_condition_left;
|
||||
bool has_right = l.on_filter_condition_right || r.on_filter_condition_right;
|
||||
return !(has_left && has_right);
|
||||
}
|
||||
|
||||
bool equalTableColumns(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r)
|
||||
{
|
||||
return l.key_names_left == r.key_names_left &&
|
||||
l.key_names_right == r.key_names_right &&
|
||||
compatibleFilerConditions(l, r);
|
||||
}
|
||||
|
||||
void joinASTbyOR(ASTPtr & to, const ASTPtr & from)
|
||||
{
|
||||
if (from == nullptr)
|
||||
return;
|
||||
|
||||
if (to == nullptr)
|
||||
{
|
||||
to = from;
|
||||
}
|
||||
else if (const auto * func = to->as<ASTFunction>(); func && func->name == "or")
|
||||
{
|
||||
/// already have `or` in condition, just add new argument
|
||||
func->arguments->children.push_back(from);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// already have some conditions, unite it with `or`
|
||||
to = makeASTFunction("or", to, from);
|
||||
}
|
||||
}
|
||||
|
||||
/// from's conditions added to to's ones
|
||||
void addConditionsToClause(TableJoin::JoinOnClause & to, const TableJoin::JoinOnClause & from)
|
||||
{
|
||||
assert(compatibleFilerConditions(to, from));
|
||||
|
||||
joinASTbyOR(to.on_filter_condition_left, from.on_filter_condition_left);
|
||||
joinASTbyOR(to.on_filter_condition_right, from.on_filter_condition_right);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
bool operator<(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r)
|
||||
{
|
||||
return l.key_names_left < r.key_names_left ||
|
||||
(l.key_names_left == r.key_names_left && l.key_names_right < r.key_names_right) ||
|
||||
(l.key_names_left == r.key_names_left && l.key_names_right == r.key_names_right && l.on_filter_condition_left && !r.on_filter_condition_left);
|
||||
}
|
||||
|
||||
void TableJoin::optimizeClauses()
|
||||
{
|
||||
if (clauses.size() > 1)
|
||||
{
|
||||
std::sort(std::begin(clauses), std::end(clauses));
|
||||
|
||||
auto to_it = clauses.begin();
|
||||
|
||||
for (auto from_it = to_it + 1; from_it != clauses.end(); ++from_it)
|
||||
{
|
||||
if (!equalTableColumns(*from_it, *to_it))
|
||||
{
|
||||
if (++to_it != from_it)
|
||||
{
|
||||
*to_it = std::move(*from_it);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
addConditionsToClause(*to_it, *from_it);
|
||||
}
|
||||
}
|
||||
const Clauses::size_type new_size = std::distance(clauses.begin(), to_it) + 1;
|
||||
if (clauses.size() != new_size)
|
||||
{
|
||||
LOG_TRACE(
|
||||
&Poco::Logger::get("TableJoin"), "optimizeClauses trim clauses, size {} => {}", clauses.size(), new_size);
|
||||
clauses.resize(new_size);
|
||||
}
|
||||
|
||||
if (clauses.size() > MAX_DISJUNCTS)
|
||||
{
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
"Maximum number of keys that join tables via OR is {} (after normalization), consider reducing",
|
||||
MAX_DISJUNCTS);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void TableJoin::addDisjunct()
|
||||
{
|
||||
clauses.emplace_back();
|
||||
@ -619,7 +512,6 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
|
||||
right_key_name, rtype->second->getName(),
|
||||
ex.message());
|
||||
}
|
||||
|
||||
if (!allow_right && !common_type->equals(*rtype->second))
|
||||
{
|
||||
throw DB::Exception(ErrorCodes::TYPE_MISMATCH,
|
||||
|
@ -222,15 +222,9 @@ public:
|
||||
|
||||
void resetCollected();
|
||||
void addUsingKey(const ASTPtr & ast);
|
||||
void newClauseIfPopulated();
|
||||
|
||||
/// if several disjuncts have exactly the same table columns
|
||||
/// we can eliminate redundant disjuncts ORing filter conditions
|
||||
/// This is needed for queries like
|
||||
/// SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.a = t2.a AND t1.c > 0) OR (t1.a = t2.a AND t1.b > 0);
|
||||
/// to be compatible with merge joins and to reduce number of hashmaps if hashjoin.
|
||||
void optimizeClauses();
|
||||
void addDisjunct();
|
||||
|
||||
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
|
||||
|
||||
/* Conditions for left/right table from JOIN ON section.
|
||||
|
@ -508,212 +508,6 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul
|
||||
out_table_join = table_join;
|
||||
}
|
||||
|
||||
/// Convert to Disjunctive Normal Form https://en.wikipedia.org/wiki/Disjunctive_normal_form
|
||||
/// based on sample https://github.com/ilejn/ndf
|
||||
/// Keep join conditions as is.
|
||||
class DNF
|
||||
{
|
||||
bool node_added = false;
|
||||
const CollectJoinOnKeysVisitor::Data & data;
|
||||
|
||||
void normTree(ASTPtr node)
|
||||
{
|
||||
auto * func = node->as<ASTFunction>();
|
||||
if (func && func->children.size() == 1)
|
||||
{
|
||||
for (bool touched = true; touched;)
|
||||
{
|
||||
touched = false;
|
||||
|
||||
ASTs new_children;
|
||||
const auto * func_args = func->arguments->as<ASTExpressionList>();
|
||||
for (const auto & child : func_args->children)
|
||||
{
|
||||
auto * child_func = child->as<ASTFunction>();
|
||||
if (child_func && func->children.size() == 1
|
||||
&& ((func->name == "or" && child_func->name == "or") || (func->name == "and" && child_func->name == "and")))
|
||||
{
|
||||
std::copy(child_func->arguments->children.begin(),
|
||||
child_func->arguments->children.end(),
|
||||
std::back_inserter(new_children));
|
||||
touched = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
new_children.push_back(child);
|
||||
}
|
||||
}
|
||||
|
||||
func->arguments->children = std::move(new_children);
|
||||
}
|
||||
|
||||
for (auto & child : func->arguments->children)
|
||||
{
|
||||
normTree(child);
|
||||
}
|
||||
}
|
||||
// LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of normTree: {}", node->dumpTree());
|
||||
}
|
||||
|
||||
ASTPtr distribute(ASTPtr node)
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
const auto * function = node->as<ASTFunction>();
|
||||
|
||||
if (function && function->children.size() == 1)
|
||||
{
|
||||
if (function->name == "and")
|
||||
{
|
||||
auto * func_args = function->arguments->as<ASTExpressionList>();
|
||||
if (!func_args)
|
||||
{
|
||||
return node;
|
||||
}
|
||||
|
||||
ASTs distr_lst;
|
||||
for (const auto & arg : func_args->children)
|
||||
{
|
||||
distr_lst.push_back(distribute(arg));
|
||||
}
|
||||
func_args->children = distr_lst;
|
||||
|
||||
auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [this](ASTPtr arg)
|
||||
{
|
||||
const auto * f = arg->as<ASTFunction>();
|
||||
if (f && f->name == "or" && f->children.size() == 1)
|
||||
{
|
||||
const bool throw_on_table_mix = false;
|
||||
auto pos = CollectJoinOnKeysMatcher::getTableForIdentifiers(std::make_shared<ASTFunction>(*f),
|
||||
throw_on_table_mix,
|
||||
data);
|
||||
return pos != JoinIdentifierPos::Left && pos != JoinIdentifierPos::Right;
|
||||
}
|
||||
return false;
|
||||
|
||||
});
|
||||
if (or_child == func_args->children.end())
|
||||
{
|
||||
return node;
|
||||
}
|
||||
|
||||
ASTs rest_children;
|
||||
|
||||
for (const auto & arg : func_args->children)
|
||||
{
|
||||
// LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash());
|
||||
|
||||
// if (arg->getTreeHash() != (*or_child)->getTreeHash())
|
||||
if (arg.get() != (*or_child).get())
|
||||
{
|
||||
rest_children.push_back(arg);
|
||||
}
|
||||
}
|
||||
if (rest_children.empty())
|
||||
{
|
||||
return node;
|
||||
}
|
||||
|
||||
const auto * or_child_function = (*or_child)->as<ASTFunction>();
|
||||
if (!or_child_function)
|
||||
{
|
||||
return node;
|
||||
}
|
||||
|
||||
auto rest = rest_children.size() > 1 ?
|
||||
makeASTFunction("and", rest_children):
|
||||
rest_children[0];
|
||||
|
||||
const auto * or_child_expression_list = or_child_function->children[0]->as<ASTExpressionList>();
|
||||
assert(or_child_expression_list);
|
||||
|
||||
if (or_child_expression_list)
|
||||
{
|
||||
|
||||
ASTs lst;
|
||||
for (const auto & arg : or_child_expression_list->children)
|
||||
{
|
||||
ASTs arg_rest_lst;
|
||||
arg_rest_lst.push_back(arg);
|
||||
arg_rest_lst.push_back(rest);
|
||||
|
||||
auto and_node = makeASTFunction("and", arg_rest_lst);
|
||||
lst.push_back(distribute(and_node));
|
||||
}
|
||||
if (lst.empty())
|
||||
{
|
||||
return node;
|
||||
}
|
||||
|
||||
auto ret = lst.size()>1 ?
|
||||
makeASTFunction("or", lst) :
|
||||
lst[0];
|
||||
|
||||
node_added = true;
|
||||
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
else if (function->name == "or")
|
||||
{
|
||||
const auto * expression_list = function->children[0]->as<ASTExpressionList>();
|
||||
if (!expression_list)
|
||||
{
|
||||
return node;
|
||||
}
|
||||
|
||||
ASTs lst;
|
||||
for (const auto & arg : expression_list->children)
|
||||
{
|
||||
lst.push_back(distribute(arg));
|
||||
}
|
||||
|
||||
auto ret = lst.size() > 1
|
||||
? makeASTFunction("or", lst)
|
||||
: lst[0];
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
|
||||
return node;
|
||||
}
|
||||
|
||||
|
||||
public:
|
||||
explicit DNF(const CollectJoinOnKeysVisitor::Data & data_)
|
||||
: data(data_)
|
||||
{
|
||||
}
|
||||
|
||||
void process(const ASTSelectQuery & select_query, const TablesWithColumns & tables)
|
||||
{
|
||||
|
||||
const ASTTablesInSelectQueryElement * node = select_query.join();
|
||||
if (!node || tables.size() < 2)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
auto & table_join = node->table_join->as<ASTTableJoin &>();
|
||||
if (!table_join.on_expression || table_join.strictness == ASTTableJoin::Strictness::Asof)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
normTree(table_join.on_expression);
|
||||
|
||||
auto distributed_expression = distribute(table_join.on_expression);
|
||||
|
||||
normTree(distributed_expression);
|
||||
LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of toDNF: {}, node_added {}",
|
||||
distributed_expression->dumpTree(), node_added);
|
||||
|
||||
table_join.on_expression = distributed_expression;
|
||||
|
||||
table_join.converted_to_dnf = node_added;
|
||||
}
|
||||
};
|
||||
|
||||
/// Find the columns that are obtained by JOIN.
|
||||
void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_join,
|
||||
const TablesWithColumns & tables, const Aliases & aliases)
|
||||
@ -761,7 +555,6 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_
|
||||
"Cannot get JOIN keys from JOIN ON section: '{}'",
|
||||
queryToString(table_join.on_expression));
|
||||
}
|
||||
analyzed_join.optimizeClauses();
|
||||
|
||||
if (is_asof)
|
||||
{
|
||||
@ -934,10 +727,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
|
||||
if (required.count(name))
|
||||
{
|
||||
/// Optimisation: do not add columns needed only in JOIN ON section.
|
||||
/// Does not work well if AST was altered, the problem here is DNFing applied to table_join.on_query,
|
||||
/// not to original query, so calculation of 'how many times a column occurred in a query' is not affected by DNFing,
|
||||
/// but calculation of 'how many times it occurred in join' is affected.
|
||||
if (columns_context.converted_to_dnf || columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name))
|
||||
if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name))
|
||||
analyzed_join->addJoinedColumn(joined_column);
|
||||
|
||||
required.erase(name);
|
||||
@ -1179,21 +969,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
|
||||
normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true);
|
||||
|
||||
if (table_join)
|
||||
{
|
||||
auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as<ASTTableJoin>() : nullptr;
|
||||
if (table_join_ast)
|
||||
{
|
||||
CollectJoinOnKeysVisitor::Data data{*result.analyzed_join,
|
||||
tables_with_columns[0],
|
||||
tables_with_columns[1],
|
||||
result.aliases,
|
||||
table_join_ast->strictness == ASTTableJoin::Strictness::Asof};
|
||||
DNF(data).process(*select_query, tables_with_columns);
|
||||
}
|
||||
}
|
||||
|
||||
/// Remove unneeded columns according to 'required_result_columns'.
|
||||
/// Remove unneeded columns according to 'required_result_columns'.
|
||||
/// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside.
|
||||
/// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost)
|
||||
/// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations.
|
||||
|
@ -99,8 +99,6 @@ struct ASTTableJoin : public IAST
|
||||
Strictness strictness = Strictness::Unspecified;
|
||||
Kind kind = Kind::Inner;
|
||||
|
||||
bool converted_to_dnf = false;
|
||||
|
||||
/// Condition. One of fields is non-nullptr.
|
||||
ASTPtr using_expression_list;
|
||||
ASTPtr on_expression;
|
||||
|
@ -17,9 +17,7 @@ SET join_algorithm = 'auto';
|
||||
SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 }
|
||||
-- works for a = b OR a = b because of equivalent disjunct optimization
|
||||
|
||||
SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb; -- { serverError 48 }
|
||||
|
||||
|
||||
SET join_algorithm = 'hash';
|
||||
|
||||
SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb; -- { serverError 403 }
|
||||
-- conditions for different table joined via OR
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError 403 }
|
||||
|
@ -29,5 +29,7 @@ select tab2.*, tab3.* from tab2 any join tab3 on a2 = a3 or b2 = b3;
|
||||
select '==';
|
||||
select tab2.*, tab3.* from tab2 any join tab3 on b2 = b3 or a2 = a3;
|
||||
|
||||
SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb;
|
||||
|
||||
drop table tab2;
|
||||
drop table tab3;
|
||||
|
@ -1,12 +1,17 @@
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.b = t2.b and t1.c = t2.b and t1.d = t2.b or t1.e = t2.e;
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b or t1.e = t2.e and t1.a=t2.a and t2.f=t1.f;
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f);
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e;
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f);
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f);
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f);
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f));
|
||||
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f);
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.e = t2.e;
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.e = t2.e and t1.a=t2.a and t2.f=t1.f;
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b AND t1.e = t2.e OR t1.c = t2.b AND t1.e = t2.e OR t1.d = t2.b AND t1.f=t2.f OR t1.c = t2.b AND t1.f=t2.f;
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f));
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f);
|
||||
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION }
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; -- { serverError INVALID_JOIN_ON_EXPRESSION }
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION }
|
||||
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f); -- { serverError INVALID_JOIN_ON_EXPRESSION }
|
||||
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
SET max_threads = 1;
|
||||
@ -24,7 +29,7 @@ insert into tab3 values (5, 4);
|
||||
insert into tab3 values (100, 4);
|
||||
|
||||
select 'join on OR/AND chain';
|
||||
select a2, b2, a3, b3 from tab2 any left join tab3 on (a2=a3 or b2=b3) and a2 +1 = b3 + 0;
|
||||
select a2, b2, a3, b3 from tab2 any left join tab3 on a2=a3 and a2 +1 = b3 + 0 or b2=b3 and a2 +1 = b3 + 0 ;
|
||||
|
||||
drop table tab2;
|
||||
drop table tab3;
|
||||
|
@ -18,6 +18,6 @@ select '17 full', * from (select 1 as x, 2 as y) t1 full join (select 1 as xx, 2
|
||||
|
||||
select count(1) from (select * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(555)) t2 on x = xx or y = yy);
|
||||
|
||||
select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.c=t2.c or t1.a = t2.a;
|
||||
select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.c = t2.c or t1.a = t2.a order by t1.c, t2.c;
|
||||
|
||||
select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.a = t2.a or t1.c=t2.c;
|
||||
select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.a = t2.a or t1.c = t2.c order by t1.c, t2.c;
|
||||
|
@ -70,32 +70,4 @@ t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 3 333 333 3 100 BBB BBB
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
t22 1 111 111 2 1 AAA AAA
|
||||
1 1 1 1 1 1
|
||||
1 1 1 1 1 1
|
||||
2 222 2 2 AAA AAA
|
||||
2 222 222 2 AAA AAA
|
||||
2 222 222 2 AAA a
|
||||
3 333 333 3 BBB BBB
|
||||
2 222 2 2 AAA AAA
|
||||
2 222 222 2 AAA AAA
|
||||
2 222 222 2 AAA a
|
||||
2 222 222 2 AAA AAA
|
||||
2 222 222 2 AAA a
|
||||
3 333 333 3 BBB BBB
|
||||
2 222 2 2 AAA AAA
|
||||
2 222 2 2 AAA a
|
||||
2 222 222 2 AAA AAA
|
||||
2 222 222 2 AAA a
|
||||
3 333 333 3 BBB BBB
|
||||
{% endfor -%}
|
||||
|
@ -73,7 +73,8 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverErro
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id;
|
||||
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id;
|
||||
-- non-equi condition containing columns from different tables doesn't supported yet
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 }
|
||||
@ -82,32 +83,16 @@ SELECT '--';
|
||||
-- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key
|
||||
SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333';
|
||||
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id);
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2);
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd;
|
||||
SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB');
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111'));
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id);
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2);
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111'));
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 }
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd;
|
||||
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd;
|
||||
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0;
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t2.key2='BBB'));
|
||||
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t1.key2='BBB'));
|
||||
|
||||
select t1.*, t2.* from t1 inner join t2 on t1.id = t2.id AND (((toInt32(t1.key) > 222 OR toInt32(t1.key2) > 5) AND (length(t2.key) > 0 OR length(t2.key2) > 0)));
|
||||
|
||||
select t1.*, t2.* from t1 inner join t2 on t1.id = t2.id AND (((toInt32(t1.key) > 222 OR toInt32(t1.key2) > 5) AND (length(t2.key) > 0 OR length(t2.key2) > 0)) OR t1.key = '222');
|
||||
{% endfor -%}
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
|
@ -54,12 +54,7 @@
|
||||
2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }}
|
||||
2 222 222 2 AAA AAA
|
||||
3 333 333 3 BBB BBB
|
||||
2 222 222 2 AAA AAA
|
||||
2 222 222 2 AAA a
|
||||
3 333 333 3 BBB BBB
|
||||
--
|
||||
2 222 2 2 AAA a
|
||||
2 222 222 2 AAA AAA
|
||||
t22 3 333 333 3 100 BBB BBB
|
||||
1 1 1 1 1 1
|
||||
{% endfor -%}
|
||||
|
@ -35,8 +35,8 @@ SELECT '--';
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2;
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3;
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB';
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333';
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333';
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333';
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333';
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2);
|
||||
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%';
|
||||
@ -63,7 +63,6 @@ SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id =
|
||||
|
||||
SELECT '--';
|
||||
SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST;
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND t1.key2 == '333');
|
||||
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 }
|
||||
@ -73,7 +72,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverErro
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 48 }
|
||||
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 }
|
||||
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- { serverError 48 }
|
||||
-- non-equi condition containing columns from different tables doesn't supported yet
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 }
|
||||
@ -82,16 +81,8 @@ SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t
|
||||
SELECT '--';
|
||||
-- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key
|
||||
SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333';
|
||||
SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB');
|
||||
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0; -- { serverError 48 }
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError 48 }
|
||||
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON (t1.a = t2.a AND t1.b > 0) OR (t1.a = t2.a AND t1.c > 0);
|
||||
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t2.key2='BBB')); -- { serverError 48 }
|
||||
|
||||
SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t1.key2='BBB')); -- { serverError 48 }
|
||||
{% endfor -%}
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
|
Loading…
Reference in New Issue
Block a user