review changes 1 - ASTPtr, some comments

This commit is contained in:
Ilya Golshtein 2021-07-23 22:55:36 +03:00
parent c0ad89016a
commit db50015eed
6 changed files with 58 additions and 53 deletions

View File

@ -32,21 +32,22 @@ bool isRightIdentifier(JoinIdentifierPos pos)
} }
void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTFunction & func) void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & ast)
{ {
const auto * expression_list = func.children.front()->as<ASTExpressionList>(); auto * func = ast->as<ASTFunction>();
std::vector<const IAST*> v; const auto * func_args = func->arguments->as<ASTExpressionList>();
for (const auto & child : expression_list->children) std::vector<const ASTPtr> v;
for (const auto & child : func_args->children)
{ {
v.push_back(child.get()); v.push_back(child);
} }
analyzed_join.setDisjuncts(std::move(v)); analyzed_join.setDisjuncts(std::move(v));
} }
void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTFunction & func) void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTPtr & ast)
{ {
analyzed_join.addDisjunct(static_cast<const IAST*>(&func)); analyzed_join.addDisjunct(std::move(ast));
} }
void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos)
@ -107,11 +108,11 @@ void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & as
if (func.name == "or") if (func.name == "or")
{ {
// throw Exception("JOIN ON does not support OR. Unexpected '" + queryToString(ast) + "'", ErrorCodes::NOT_IMPLEMENTED); // throw Exception("JOIN ON does not support OR. Unexpected '" + queryToString(ast) + "'", ErrorCodes::NOT_IMPLEMENTED);
data.setDisjuncts(func); data.setDisjuncts(ast);
return; return;
} }
data.addDisjunct(func); data.addDisjunct(ast);
if (func.name == "and") if (func.name == "and")
return; /// go into children return; /// go into children

View File

@ -51,8 +51,8 @@ public:
void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos);
void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos,
const ASOF::Inequality & asof_inequality); const ASOF::Inequality & asof_inequality);
void setDisjuncts(const ASTFunction & or_func); void setDisjuncts(const ASTPtr & or_func);
void addDisjunct(const ASTFunction & func); void addDisjunct(const ASTPtr & func);
void asofToJoinKeys(); void asofToJoinKeys();
}; };

View File

@ -209,7 +209,8 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
if (multiple_disjuncts) if (multiple_disjuncts)
{ {
// required_right_keys_sources concept does not work well if multiple disjuncts /// required right keys concept does not work well if multiple disjuncts,
/// we need all keys
sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block); sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block);
} }
else else
@ -1052,22 +1053,23 @@ struct JoinFeatures
template <bool multiple_disjuncts> template <bool multiple_disjuncts>
class KnownRowsHolder; class KnownRowsHolder;
// TODO: helper to clean, instead of recreating /// Keep already joined rows to prevent duplication if many disjuncts
/// if for a particular pair of rows condition looks like TRUE or TRUE or TRUE
/// we want to have it once in resultset
template<> template<>
class KnownRowsHolder<true> class KnownRowsHolder<true>
{ {
public: public:
using Type = std::pair<const Block*, DB::RowRef::SizeT>; using Type = std::pair<const Block *, DB::RowRef::SizeT>;
private: private:
static const size_t MAX_LINEAR = 16; static const size_t MAX_LINEAR = 16; // threshold to switch from Array to Set
using LinearHolder = std::array<Type, MAX_LINEAR>; using ArrayHolder = std::array<Type, MAX_LINEAR>;
using LogHolder = std::set<Type>; using SetHolder = std::set<Type>;
using LogHolderPtr = std::unique_ptr<LogHolder>; using SetHolderPtr = std::unique_ptr<SetHolder>;
LinearHolder linh; ArrayHolder array_holder;
LogHolderPtr logh_ptr; SetHolderPtr set_holder_ptr;
size_t items; size_t items;
@ -1081,19 +1083,19 @@ public:
template<class InputIt> template<class InputIt>
void add(InputIt from, InputIt to) void add(InputIt from, InputIt to)
{ {
size_t new_items = std::distance(from, to); const size_t new_items = std::distance(from, to);
if (items + new_items <= MAX_LINEAR) if (items + new_items <= MAX_LINEAR)
{ {
std::copy(from, to, &linh[items]); std::copy(from, to, &array_holder[items]);
} }
else else
{ {
if (items <= MAX_LINEAR) if (items <= MAX_LINEAR)
{ {
logh_ptr = std::make_unique<LogHolder>(); set_holder_ptr = std::make_unique<SetHolder>();
logh_ptr->insert(std::cbegin(linh), std::cbegin(linh) + items); set_holder_ptr->insert(std::cbegin(array_holder), std::cbegin(array_holder) + items);
} }
logh_ptr->insert(from, to); set_holder_ptr->insert(from, to);
} }
items += new_items; items += new_items;
} }
@ -1102,8 +1104,8 @@ public:
bool isKnown(const Needle & needle) bool isKnown(const Needle & needle)
{ {
return items <= MAX_LINEAR return items <= MAX_LINEAR
? std::find(std::cbegin(linh), std::cbegin(linh) + items, needle) != std::cbegin(linh) + items ? std::find(std::cbegin(array_holder), std::cbegin(array_holder) + items, needle) != std::cbegin(array_holder) + items
: logh_ptr->find(needle) != logh_ptr->end(); : set_holder_ptr->find(needle) != set_holder_ptr->end();
} }
}; };

View File

@ -103,10 +103,12 @@ void TableJoin::addUsingKey(const ASTPtr & ast)
right_key = renames[right_key]; right_key = renames[right_key];
} }
/// create new disjunct when see a child of a previously descovered OR
void TableJoin::addDisjunct(const IAST* addr) void TableJoin::addDisjunct(const ASTPtr & ast)
{ {
if (std::find(disjuncts.begin(), disjuncts.end(), addr) != disjuncts.end()) const IAST * addr = ast.get();
if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end())
{ {
assert(key_names_left.size() == disjunct_num + 1); assert(key_names_left.size() == disjunct_num + 1);
@ -121,9 +123,10 @@ void TableJoin::addDisjunct(const IAST* addr)
} }
} }
void TableJoin::setDisjuncts(std::vector<const IAST*>&& disjuncts_) /// remember OR's children
void TableJoin::setDisjuncts(std::vector<const ASTPtr>&& disjuncts_)
{ {
disjuncts = disjuncts_; disjuncts = std::move(disjuncts_);
} }
void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)

View File

@ -79,7 +79,7 @@ private:
ASTsVector on_filter_condition_asts_right; ASTsVector on_filter_condition_asts_right;
private: private:
size_t disjunct_num = 0; size_t disjunct_num = 0;
std::vector<const IAST*> disjuncts; std::vector<const ASTPtr> disjuncts;
ASTs key_asts_left; ASTs key_asts_left;
ASTs key_asts_right; ASTs key_asts_right;
@ -177,8 +177,8 @@ public:
void resetCollected(); void resetCollected();
void addUsingKey(const ASTPtr & ast); void addUsingKey(const ASTPtr & ast);
void addDisjunct(const IAST*); void setDisjuncts(std::vector<const ASTPtr>&&);
void setDisjuncts(std::vector<const IAST*>&&); void addDisjunct(const ASTPtr &);
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
/* Conditions for left/right table from JOIN ON section. /* Conditions for left/right table from JOIN ON section.

View File

@ -512,24 +512,23 @@ class DNF
void normTree(ASTPtr node) void normTree(ASTPtr node)
{ {
auto *function = node->as<ASTFunction>(); auto * func = node->as<ASTFunction>();
if (func && func->children.size() == 1)
// LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "top of normTree: {}", node->dumpTree());
if (function && function->children.size() == 1)
{ {
for (bool touched = true; touched;) for (bool touched = true; touched;)
{ {
touched = false; touched = false;
ASTs new_children; ASTs new_children;
const auto * expression_list = function->children[0]->as<ASTExpressionList>(); const auto * func_args = func->arguments->as<ASTExpressionList>();
for (const auto & child : expression_list->children) for (const auto & child : func_args->children)
{ {
auto *f = child->as<ASTFunction>(); auto * child_func = child->as<ASTFunction>();
if (f && function->children.size() == 1 && ((function->name == "or" && f->name == "or") || (function->name == "and" && f->name == "and"))) if (child_func && func->children.size() == 1
&& ((func->name == "or" && child_func->name == "or") || (func->name == "and" && child_func->name == "and")))
{ {
std::copy(child->children[0]->children.begin(), std::copy(child_func->arguments->children.begin(),
child->children[0]->children.end(), child_func->arguments->children.end(),
std::back_inserter(new_children)); std::back_inserter(new_children));
touched = true; touched = true;
} }
@ -539,10 +538,10 @@ class DNF
} }
} }
function->arguments->children = std::move(new_children); func->arguments->children = std::move(new_children);
} }
for (auto & child : function->arguments->children) for (auto & child : func->arguments->children)
{ {
normTree(child); normTree(child);
} }
@ -558,25 +557,25 @@ class DNF
{ {
if (function->name == "and") if (function->name == "and")
{ {
const auto * expression_list = function->children[0]->as<ASTExpressionList>(); const auto * func_args = function->arguments->as<ASTExpressionList>();
if (!expression_list) if (!func_args)
{ {
return node; return node;
} }
auto or_child = std::find_if(expression_list->children.begin(), expression_list->children.end(), [](ASTPtr arg) auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [](ASTPtr arg)
{ {
const auto * f = arg->as<ASTFunction>(); const auto * f = arg->as<ASTFunction>();
return f && f->name == "or" && f->children.size() == 1; return f && f->name == "or" && f->children.size() == 1;
}); });
if (or_child == expression_list->children.end()) if (or_child == func_args->children.end())
{ {
return node; return node;
} }
ASTs rest_children; ASTs rest_children;
for (const auto & arg : expression_list->children) for (const auto & arg : func_args->children)
{ {
// LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash()); // LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash());