ClickHouse/src/Storages/MergeTree/KeyCondition.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

2748 lines
95 KiB
C++
Raw Normal View History

#include <Storages/MergeTree/KeyCondition.h>
#include <Storages/MergeTree/BoolMask.h>
#include <DataTypes/DataTypesNumber.h>
2020-09-05 14:12:47 +00:00
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/getLeastSupertype.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
2020-11-06 03:50:58 +00:00
#include <Interpreters/castColumn.h>
#include <Interpreters/misc.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsConversion.h>
2021-06-23 12:19:22 +00:00
#include <Functions/indexHint.h>
#include <Functions/CastOverloadResolver.h>
#include <Functions/IFunction.h>
2020-06-14 18:42:10 +00:00
#include <Common/FieldVisitorsAccurateComparison.h>
2021-06-14 04:13:35 +00:00
#include <Common/FieldVisitorToString.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
2021-06-03 12:26:02 +00:00
#include <Columns/ColumnSet.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/Set.h>
2018-02-26 03:37:08 +00:00
#include <Parsers/queryToString.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
2020-11-09 19:07:38 +00:00
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
2021-05-21 14:56:36 +00:00
#include <Storages/KeyDescription.h>
#include <cassert>
2020-11-09 11:58:11 +00:00
#include <stack>
2021-04-27 04:22:32 +00:00
#include <limits>
2012-12-05 12:44:55 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_TYPE_OF_FIELD;
}
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
String Range::toString() const
{
2020-11-09 19:07:38 +00:00
WriteBufferFromOwnString str;
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
str << (left_included ? '[' : '(') << applyVisitor(FieldVisitorToString(), left) << ", ";
str << applyVisitor(FieldVisitorToString(), right) << (right_included ? ']' : ')');
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
return str.str();
}
2021-11-09 13:34:55 +00:00
/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string.
2021-12-08 02:40:59 +00:00
String extractFixedPrefixFromLikePattern(const String & like_pattern)
{
String fixed_prefix;
const char * pos = like_pattern.data();
const char * end = pos + like_pattern.size();
while (pos < end)
{
switch (*pos)
{
case '%':
2017-12-02 03:25:52 +00:00
[[fallthrough]];
case '_':
2021-11-09 13:34:55 +00:00
return fixed_prefix;
case '\\':
++pos;
if (pos == end)
break;
2017-12-02 03:25:52 +00:00
[[fallthrough]];
default:
fixed_prefix += *pos;
break;
}
++pos;
}
return fixed_prefix;
}
2017-03-12 19:18:07 +00:00
/** For a given string, get a minimum string that is strictly greater than all strings with this prefix,
* or return an empty string if there are no such strings.
*/
static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & prefix)
{
2021-04-27 04:22:32 +00:00
/** Increment the last byte of the prefix by one. But if it is max (255), then remove it and increase the previous one.
2017-03-13 18:01:46 +00:00
* Example (for convenience, suppose that the maximum value of byte is `z`)
* abcx -> abcy
* abcz -> abd
2017-03-12 19:18:07 +00:00
* zzz -> empty string
* z -> empty string
*/
String res = prefix;
2021-04-27 04:22:32 +00:00
while (!res.empty() && static_cast<UInt8>(res.back()) == std::numeric_limits<UInt8>::max())
res.pop_back();
if (res.empty())
return res;
res.back() = static_cast<char>(1 + static_cast<UInt8>(res.back()));
return res;
}
2021-06-21 16:17:05 +00:00
static void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool legacy = false)
{
switch (node.type)
{
case (ActionsDAG::ActionType::INPUT):
2021-06-21 16:17:05 +00:00
writeString(node.result_name, out);
break;
case (ActionsDAG::ActionType::COLUMN):
{
/// If it was created from ASTLiteral, then result_name can be an alias.
/// We need to convert value back to string here.
if (const auto * column_const = typeid_cast<const ColumnConst *>(node.column.get()))
writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out);
/// It may be possible that column is ColumnSet
else
writeString(node.result_name, out);
break;
}
2021-06-21 16:17:05 +00:00
case (ActionsDAG::ActionType::ALIAS):
appendColumnNameWithoutAlias(*node.children.front(), out, legacy);
break;
case (ActionsDAG::ActionType::ARRAY_JOIN):
writeCString("arrayJoin(", out);
appendColumnNameWithoutAlias(*node.children.front(), out, legacy);
writeChar(')', out);
break;
case (ActionsDAG::ActionType::FUNCTION):
{
auto name = node.function_base->getName();
if (legacy && name == "modulo")
writeCString("moduleLegacy", out);
else
writeString(name, out);
writeChar('(', out);
bool first = true;
for (const auto * arg : node.children)
{
if (!first)
writeCString(", ", out);
first = false;
appendColumnNameWithoutAlias(*arg, out, legacy);
}
writeChar(')', out);
}
}
}
static std::string getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool legacy = false)
{
WriteBufferFromOwnString out;
appendColumnNameWithoutAlias(node, out, legacy);
return std::move(out.str());
}
2021-06-02 16:56:24 +00:00
class KeyCondition::Tree
{
public:
2021-06-22 07:26:45 +00:00
explicit Tree(const IAST * ast_) : ast(ast_) { assert(ast); }
explicit Tree(const ActionsDAG::Node * dag_) : dag(dag_) { assert(dag); }
2021-06-02 16:56:24 +00:00
std::string getColumnName() const
{
if (ast)
return ast->getColumnNameWithoutAlias();
else
2021-06-21 16:17:05 +00:00
return getColumnNameWithoutAlias(*dag);
2021-06-02 16:56:24 +00:00
}
2021-06-21 16:17:05 +00:00
std::string getColumnNameLegacy() const
{
if (ast)
{
auto adjusted_ast = ast->clone();
KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast);
return adjusted_ast->getColumnNameWithoutAlias();
}
else
return getColumnNameWithoutAlias(*dag, true);
}
2021-06-02 16:56:24 +00:00
bool isFunction() const
{
if (ast)
return typeid_cast<const ASTFunction *>(ast);
else
return dag->type == ActionsDAG::ActionType::FUNCTION;
}
bool isConstant() const
{
if (ast)
return typeid_cast<const ASTLiteral *>(ast);
else
return dag->column && isColumnConst(*dag->column);
2021-06-02 16:56:24 +00:00
}
ColumnWithTypeAndName getConstant() const
{
if (!isConstant())
throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a constant");
ColumnWithTypeAndName res;
if (ast)
{
const auto * literal = assert_cast<const ASTLiteral *>(ast);
res.type = applyVisitor(FieldToDataType(), literal->value);
res.column = res.type->createColumnConst(0, literal->value);
}
else
{
res.type = dag->result_type;
res.column = dag->column;
}
return res;
}
2022-08-04 15:23:10 +00:00
bool tryGetConstant(const Block & block_with_constants, Field & out_value, DataTypePtr & out_type) const
2021-06-03 12:26:02 +00:00
{
if (ast)
{
// Constant expr should use alias names if any
String column_name = ast->getColumnName();
if (const auto * lit = ast->as<ASTLiteral>())
{
/// By default block_with_constants has only one column named "_dummy".
/// If block contains only constants it's may not be preprocessed by
// ExpressionAnalyzer, so try to look up in the default column.
if (!block_with_constants.has(column_name))
column_name = "_dummy";
/// Simple literal
out_value = lit->value;
out_type = block_with_constants.getByName(column_name).type;
2022-07-19 17:33:14 +00:00
/// If constant is not Null, we can assume it's type is not Nullable as well.
if (!out_value.isNull())
out_type = removeNullable(out_type);
2021-06-03 12:26:02 +00:00
return true;
}
else if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column))
{
/// An expression which is dependent on constants only
const auto & expr_info = block_with_constants.getByName(column_name);
out_value = (*expr_info.column)[0];
out_type = expr_info.type;
2022-07-19 17:33:14 +00:00
if (!out_value.isNull())
out_type = removeNullable(out_type);
2021-06-03 12:26:02 +00:00
return true;
}
}
else
{
if (dag->column && isColumnConst(*dag->column))
{
out_value = (*dag->column)[0];
out_type = dag->result_type;
2022-07-19 17:33:14 +00:00
if (!out_value.isNull())
out_type = removeNullable(out_type);
2021-06-03 12:26:02 +00:00
return true;
}
}
return false;
}
ConstSetPtr tryGetPreparedSet(
2022-08-10 13:43:55 +00:00
const PreparedSetsPtr & sets,
2021-06-03 12:26:02 +00:00
const std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> & indexes_mapping,
const DataTypes & data_types) const
{
2022-08-10 13:43:55 +00:00
if (sets && ast)
2021-06-03 12:26:02 +00:00
{
if (ast->as<ASTSubquery>() || ast->as<ASTTableIdentifier>())
2022-08-10 13:43:55 +00:00
return sets->get(PreparedSetKey::forSubquery(*ast));
/// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information
/// about types in left argument of the IN operator. Instead, we manually iterate through all the sets
/// and find the one for the right arg based on the AST structure (getTreeHash), after that we check
/// that the types it was prepared with are compatible with the types of the primary key.
auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set)
2021-06-03 12:26:02 +00:00
{
assert(indexes_mapping.size() == data_types.size());
for (size_t i = 0; i < indexes_mapping.size(); ++i)
{
if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i]))
return false;
}
2021-06-03 12:26:02 +00:00
return true;
};
2021-06-03 12:26:02 +00:00
2022-08-10 13:43:55 +00:00
for (const auto & set : sets->getByTreeHash(ast->getTreeHash()))
{
if (types_match(set))
return set;
2021-06-03 12:26:02 +00:00
}
}
2022-08-10 13:43:55 +00:00
else if (dag->column)
2021-06-03 12:26:02 +00:00
{
2022-08-10 13:43:55 +00:00
const IColumn * col = dag->column.get();
if (const auto * col_const = typeid_cast<const ColumnConst *>(col))
col = &col_const->getDataColumn();
if (const auto * col_set = typeid_cast<const ColumnSet *>(col))
2021-06-03 12:26:02 +00:00
{
2022-08-10 13:43:55 +00:00
auto set = col_set->getData();
if (set->isCreated())
return set;
2021-06-03 12:26:02 +00:00
}
}
return nullptr;
}
2021-06-02 16:56:24 +00:00
FunctionTree asFunction() const;
protected:
const IAST * ast = nullptr;
const ActionsDAG::Node * dag = nullptr;
};
class KeyCondition::FunctionTree : public KeyCondition::Tree
{
public:
std::string getFunctionName() const
{
if (ast)
return assert_cast<const ASTFunction *>(ast)->name;
else
return dag->function_base->getName();
}
size_t numArguments() const
{
if (ast)
{
const auto * func = assert_cast<const ASTFunction *>(ast);
2021-06-21 17:28:15 +00:00
return func->arguments ? func->arguments->children.size() : 0;
2021-06-02 16:56:24 +00:00
}
else
return dag->children.size();
}
Tree getArgumentAt(size_t idx) const
{
if (ast)
return Tree(assert_cast<const ASTFunction *>(ast)->arguments->children[idx].get());
else
return Tree(dag->children[idx]);
}
private:
using Tree::Tree;
friend class Tree;
};
KeyCondition::FunctionTree KeyCondition::Tree::asFunction() const
{
if (!isFunction())
throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a function");
if (ast)
return KeyCondition::FunctionTree(ast);
else
return KeyCondition::FunctionTree(dag);
}
2017-03-12 19:18:07 +00:00
/// A dictionary containing actions to the corresponding functions to turn them into `RPNElement`
const KeyCondition::AtomMap KeyCondition::atom_map
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
{
{
"notEquals",
[] (RPNElement & out, const Field & value)
{
out.function = RPNElement::FUNCTION_NOT_IN_RANGE;
out.range = Range(value);
return true;
}
},
{
"equals",
[] (RPNElement & out, const Field & value)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range(value);
return true;
}
},
{
"less",
[] (RPNElement & out, const Field & value)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createRightBounded(value, false);
return true;
}
},
{
"greater",
[] (RPNElement & out, const Field & value)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createLeftBounded(value, false);
return true;
}
},
{
"lessOrEquals",
[] (RPNElement & out, const Field & value)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createRightBounded(value, true);
return true;
}
},
{
"greaterOrEquals",
[] (RPNElement & out, const Field & value)
{
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range::createLeftBounded(value, true);
return true;
}
},
{
"in",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_IN_SET;
return true;
}
},
{
"notIn",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_NOT_IN_SET;
return true;
}
},
2020-11-21 08:25:45 +00:00
{
"globalIn",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_IN_SET;
return true;
}
},
{
"globalNotIn",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_NOT_IN_SET;
return true;
}
},
{
"nullIn",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_IN_SET;
return true;
}
},
{
"notNullIn",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_NOT_IN_SET;
return true;
}
},
{
"globalNullIn",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_IN_SET;
return true;
}
},
{
"globalNotNullIn",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_NOT_IN_SET;
return true;
}
},
{
"empty",
2020-05-29 21:32:35 +00:00
[] (RPNElement & out, const Field & value)
{
2020-05-29 21:32:35 +00:00
if (value.getType() != Field::Types::String)
return false;
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = Range("");
return true;
}
},
{
"notEmpty",
2020-05-29 21:32:35 +00:00
[] (RPNElement & out, const Field & value)
{
2020-05-29 21:32:35 +00:00
if (value.getType() != Field::Types::String)
return false;
out.function = RPNElement::FUNCTION_NOT_IN_RANGE;
out.range = Range("");
return true;
}
},
{
"like",
2019-07-07 15:52:20 +00:00
[] (RPNElement & out, const Field & value)
{
if (value.getType() != Field::Types::String)
return false;
String prefix = extractFixedPrefixFromLikePattern(value.get<const String &>());
if (prefix.empty())
return false;
String right_bound = firstStringThatIsGreaterThanAllStringsWithPrefix(prefix);
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = !right_bound.empty()
2019-07-07 15:56:15 +00:00
? Range(prefix, true, right_bound, false)
: Range::createLeftBounded(prefix, true);
2019-07-07 15:48:34 +00:00
return true;
}
},
{
"startsWith",
[] (RPNElement & out, const Field & value)
{
if (value.getType() != Field::Types::String)
return false;
2019-07-07 21:28:03 +00:00
String prefix = value.get<const String &>();
2019-07-07 15:48:34 +00:00
if (prefix.empty())
return false;
String right_bound = firstStringThatIsGreaterThanAllStringsWithPrefix(prefix);
out.function = RPNElement::FUNCTION_IN_RANGE;
out.range = !right_bound.empty()
2019-07-07 15:56:15 +00:00
? Range(prefix, true, right_bound, false)
: Range::createLeftBounded(prefix, true);
return true;
}
},
{
"isNotNull",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_IS_NOT_NULL;
// isNotNull means (-Inf, +Inf), which is the default Range
out.range = Range();
return true;
}
},
{
"isNull",
[] (RPNElement & out, const Field &)
{
out.function = RPNElement::FUNCTION_IS_NULL;
2021-09-23 16:39:15 +00:00
// isNull means +Inf (NULLS_LAST) or -Inf (NULLS_FIRST),
2021-11-01 19:59:57 +00:00
// which is equivalent to not in Range (-Inf, +Inf)
2021-09-23 16:39:15 +00:00
out.range = Range();
return true;
}
}
};
static const std::map<std::string, std::string> inverse_relations = {
{"equals", "notEquals"},
{"notEquals", "equals"},
{"less", "greaterOrEquals"},
{"greaterOrEquals", "less"},
{"greater", "lessOrEquals"},
{"lessOrEquals", "greater"},
{"in", "notIn"},
{"notIn", "in"},
{"globalIn", "globalNotIn"},
{"globalNotIn", "globalIn"},
{"nullIn", "notNullIn"},
{"notNullIn", "nullIn"},
{"globalNullIn", "globalNotNullIn"},
{"globalNullNotIn", "globalNullIn"},
{"isNull", "isNotNull"},
{"isNotNull", "isNull"},
{"like", "notLike"},
{"notLike", "like"},
{"empty", "notEmpty"},
{"notEmpty", "empty"},
};
bool isLogicalOperator(const String & func_name)
{
2021-02-28 07:25:56 +00:00
return (func_name == "and" || func_name == "or" || func_name == "not" || func_name == "indexHint");
}
/// The node can be one of:
2021-02-28 07:25:56 +00:00
/// - Logical operator (AND, OR, NOT and indexHint() - logical NOOP)
/// - An "atom" (relational operator, constant, expression)
/// - A logical constant expression
/// - Any other function
ASTPtr cloneASTWithInversionPushDown(const ASTPtr node, const bool need_inversion = false)
{
const ASTFunction * func = node->as<ASTFunction>();
if (func && isLogicalOperator(func->name))
{
if (func->name == "not")
{
return cloneASTWithInversionPushDown(func->arguments->children.front(), !need_inversion);
}
const auto result_node = makeASTFunction(func->name);
2021-02-28 07:25:56 +00:00
/// indexHint() is a special case - logical NOOP function
if (result_node->name != "indexHint" && need_inversion)
{
result_node->name = (result_node->name == "and") ? "or" : "and";
}
if (func->arguments)
{
for (const auto & child : func->arguments->children)
{
result_node->arguments->children.push_back(cloneASTWithInversionPushDown(child, need_inversion));
}
}
return result_node;
}
2020-04-22 06:22:14 +00:00
auto cloned_node = node->clone();
if (func && inverse_relations.find(func->name) != inverse_relations.cend())
{
if (need_inversion)
{
cloned_node->as<ASTFunction>()->name = inverse_relations.at(func->name);
}
return cloned_node;
}
return need_inversion ? makeASTFunction("not", cloned_node) : cloned_node;
}
2021-06-21 16:17:05 +00:00
static const ActionsDAG::Node & cloneASTWithInversionPushDown(
const ActionsDAG::Node & node,
ActionsDAG & inverted_dag,
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> to_inverted,
const ContextPtr & context,
const bool need_inversion)
{
{
auto it = to_inverted.find(&node);
if (it != to_inverted.end())
return *it->second;
}
const ActionsDAG::Node * res = nullptr;
switch (node.type)
{
case (ActionsDAG::ActionType::INPUT):
{
2021-06-22 10:28:56 +00:00
/// Note: inputs order is not important here. Will match columns by names.
2021-06-21 16:17:05 +00:00
res = &inverted_dag.addInput({node.column, node.result_type, node.result_name});
break;
}
2021-06-22 10:28:56 +00:00
case (ActionsDAG::ActionType::COLUMN):
{
res = &inverted_dag.addColumn({node.column, node.result_type, node.result_name});
break;
}
2021-06-21 16:17:05 +00:00
case (ActionsDAG::ActionType::ALIAS):
{
/// Ignore aliases
const auto & alias = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion);
to_inverted[&node] = &alias;
return alias;
}
case (ActionsDAG::ActionType::ARRAY_JOIN):
{
const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, false);
2022-08-04 15:23:10 +00:00
res = &inverted_dag.addArrayJoin(arg, {});
2021-06-21 16:17:05 +00:00
break;
}
case (ActionsDAG::ActionType::FUNCTION):
{
auto name = node.function_base->getName();
if (name == "not")
{
const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, !need_inversion);
to_inverted[&node] = &arg;
return arg;
}
if (name == "materialize")
{
/// Ignore materialize
const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion);
to_inverted[&node] = &arg;
return arg;
}
2021-06-23 12:19:22 +00:00
if (name == "indexHint")
{
ActionsDAG::NodeRawConstPtrs children;
if (const auto * adaptor = typeid_cast<const FunctionToOverloadResolverAdaptor *>(node.function_builder.get()))
{
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction()))
{
const auto & index_hint_dag = index_hint->getActions();
2022-08-09 13:03:59 +00:00
children = index_hint_dag->getOutputs();
2021-06-23 12:19:22 +00:00
for (auto & arg : children)
arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion);
}
}
const auto & func = inverted_dag.addFunction(node.function_builder, children, "");
to_inverted[&node] = &func;
return func;
}
if (need_inversion && (name == "and" || name == "or"))
2021-06-21 16:17:05 +00:00
{
ActionsDAG::NodeRawConstPtrs children(node.children);
for (auto & arg : children)
arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion);
FunctionOverloadResolverPtr function_builder;
2021-06-23 12:19:22 +00:00
if (name == "and")
2021-06-21 16:17:05 +00:00
function_builder = FunctionFactory::instance().get("or", context);
else if (name == "or")
function_builder = FunctionFactory::instance().get("and", context);
assert(function_builder);
2021-06-22 10:28:56 +00:00
/// We match columns by name, so it is important to fill name correctly.
/// So, use empty string to make it automatically.
2021-06-21 16:17:05 +00:00
const auto & func = inverted_dag.addFunction(function_builder, children, "");
to_inverted[&node] = &func;
return func;
}
ActionsDAG::NodeRawConstPtrs children(node.children);
for (auto & arg : children)
arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, false);
auto it = inverse_relations.find(name);
if (it != inverse_relations.end())
{
const auto & func_name = need_inversion ? it->second : it->first;
auto function_builder = FunctionFactory::instance().get(func_name, context);
const auto & func = inverted_dag.addFunction(function_builder, children, "");
to_inverted[&node] = &func;
return func;
}
res = &inverted_dag.addFunction(node.function_builder, children, "");
}
}
if (need_inversion)
res = &inverted_dag.addFunction(FunctionFactory::instance().get("not", context), {res}, "");
to_inverted[&node] = res;
return *res;
}
2021-06-22 10:28:56 +00:00
static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context)
2021-06-21 16:17:05 +00:00
{
2021-06-22 10:28:56 +00:00
auto res = std::make_shared<ActionsDAG>();
2021-06-21 16:17:05 +00:00
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> to_inverted;
2021-06-22 10:28:56 +00:00
for (auto & node : nodes)
node = &cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false);
if (nodes.size() > 1)
2021-06-21 16:17:05 +00:00
{
2021-06-22 10:28:56 +00:00
auto function_builder = FunctionFactory::instance().get("and", context);
nodes = {&res->addFunction(function_builder, std::move(nodes), "")};
2021-06-21 16:17:05 +00:00
}
2022-08-09 13:03:59 +00:00
res->getOutputs().swap(nodes);
2021-06-21 16:17:05 +00:00
return res;
}
Squashed commit of the following: commit c567d4e1fe8d54e6363e47548f1e3927cc5ee78f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:35:01 2017 +0300 Style [#METR-2944]. commit 26bf3e1228e03f46c29b13edb0e3770bd453e3f1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:33:11 2017 +0300 Miscellaneous [#METR-2944]. commit eb946f4c6fd4bb0e9e5c7fb1468d36be3dfca5a5 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:30:19 2017 +0300 Miscellaneous [#METR-2944]. commit 78c867a14744b5af2db8d37caf7804fc2057ea51 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 20:11:41 2017 +0300 Miscellaneous [#METR-2944]. commit 6604c5c83cfcedc81c8da4da026711920d5963b4 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:56:15 2017 +0300 Miscellaneous [#METR-2944]. commit 23fbf05c1d4bead636458ec21b05a101b1152e33 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:47:52 2017 +0300 Miscellaneous [#METR-2944]. commit 98772faf11a7d450d473f7fa84f8a9ae24f7b59b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:46:05 2017 +0300 Miscellaneous [#METR-2944]. commit 3dc636ab9f9359dbeac2e8d997ae563d4ca147e2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:39:46 2017 +0300 Miscellaneous [#METR-2944]. commit 3e16aee95482f374ee3eda1a4dbe9ba5cdce02e8 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:38:03 2017 +0300 Miscellaneous [#METR-2944]. commit ae7e7e90eb1f82bd0fe0f887708d08b9e7755612 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Jan 6 19:34:15 2017 +0300 Miscellaneous [#METR-2944].
2017-01-06 17:41:19 +00:00
inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); }
inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); }
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
/** Calculate expressions, that depend only on constants.
* For index to work when something like "WHERE Date = toDate(now())" is written.
*/
Block KeyCondition::getBlockWithConstants(
const ASTPtr & query, const TreeRewriterResultPtr & syntax_analyzer_result, ContextPtr context)
{
Block result
{
2017-12-18 04:07:26 +00:00
{ DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared<DataTypeUInt8>(), "_dummy" }
};
const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActions();
expr_for_constant_folding->execute(result);
return result;
}
2021-06-15 13:47:37 +00:00
static NameSet getAllSubexpressionNames(const ExpressionActions & key_expr)
{
NameSet names;
for (const auto & action : key_expr.getActions())
names.insert(action.node->result_name);
return names;
}
KeyCondition::KeyCondition(
2021-06-21 16:17:05 +00:00
const ASTPtr & query,
2022-07-28 09:40:09 +00:00
const ASTs & additional_filter_asts,
2021-06-21 16:17:05 +00:00
TreeRewriterResultPtr syntax_analyzer_result,
PreparedSetsPtr prepared_sets_,
ContextPtr context,
2018-06-30 21:35:01 +00:00
const Names & key_column_names,
2020-09-21 10:13:01 +00:00
const ExpressionActionsPtr & key_expr_,
bool single_point_,
bool strict_)
2021-06-15 13:47:37 +00:00
: key_expr(key_expr_)
, key_subexpr_names(getAllSubexpressionNames(*key_expr))
, prepared_sets(prepared_sets_)
2021-06-15 13:47:37 +00:00
, single_point(single_point_)
, strict(strict_)
2012-12-05 12:44:55 +00:00
{
2018-06-30 21:35:01 +00:00
for (size_t i = 0, size = key_column_names.size(); i < size; ++i)
2012-12-05 12:44:55 +00:00
{
const auto & name = key_column_names[i];
if (!key_columns.contains(name))
key_columns[name] = i;
2012-12-05 12:44:55 +00:00
}
2019-03-08 08:29:48 +00:00
/** Evaluation of expressions that depend only on constants.
* For the index to be used, if it is written, for example `WHERE Date = toDate(now())`.
*/
2021-06-21 16:17:05 +00:00
Block block_with_constants = getBlockWithConstants(query, syntax_analyzer_result, context);
2019-03-08 08:29:48 +00:00
for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source)
array_joined_columns.insert(name);
2021-06-21 16:17:05 +00:00
const ASTSelectQuery & select = query->as<ASTSelectQuery &>();
2022-07-28 09:40:09 +00:00
ASTs filters;
if (select.where())
filters.push_back(select.where());
if (select.prewhere())
filters.push_back(select.prewhere());
for (const auto & filter_ast : additional_filter_asts)
filters.push_back(filter_ast);
if (!filters.empty())
2019-03-08 08:29:48 +00:00
{
ASTPtr filter_query;
2022-07-28 09:40:09 +00:00
if (filters.size() == 1)
{
filter_query = filters.front();
}
else
2022-07-28 09:40:09 +00:00
{
auto function = std::make_shared<ASTFunction>();
function->name = "and";
function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments);
function->arguments->children = std::move(filters);
filter_query = function;
}
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
* when parsing the AST into internal RPN representation.
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
* are pushed down and applied (when possible) to leaf nodes.
*/
2021-06-21 16:17:05 +00:00
auto ast = cloneASTWithInversionPushDown(filter_query);
traverseAST(Tree(ast.get()), context, block_with_constants);
}
else
{
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
}
}
KeyCondition::KeyCondition(
2021-06-22 10:28:56 +00:00
ActionDAGNodes dag_nodes,
2021-06-22 14:45:22 +00:00
TreeRewriterResultPtr syntax_analyzer_result,
PreparedSetsPtr prepared_sets_,
2021-06-21 16:17:05 +00:00
ContextPtr context,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr_,
bool single_point_,
bool strict_)
: key_expr(key_expr_)
, key_subexpr_names(getAllSubexpressionNames(*key_expr))
, prepared_sets(prepared_sets_)
2021-06-21 16:17:05 +00:00
, single_point(single_point_)
, strict(strict_)
{
for (size_t i = 0, size = key_column_names.size(); i < size; ++i)
{
2022-07-25 12:30:01 +00:00
const auto & name = key_column_names[i];
if (!key_columns.contains(name))
2021-06-21 16:17:05 +00:00
key_columns[name] = i;
}
2021-06-22 14:45:22 +00:00
for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source)
array_joined_columns.insert(name);
2021-06-22 10:28:56 +00:00
if (!dag_nodes.nodes.empty())
2021-06-21 16:17:05 +00:00
{
2021-06-22 10:28:56 +00:00
auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context);
2022-09-13 18:50:01 +00:00
// std::cerr << "========== inverted dag: " << inverted_dag->dumpDAG() << std::endl;
2021-06-22 15:52:14 +00:00
2021-06-21 16:17:05 +00:00
Block empty;
2022-08-09 13:03:59 +00:00
for (const auto * node : inverted_dag->getOutputs())
2021-06-21 16:17:05 +00:00
traverseAST(Tree(node), context, empty);
2019-03-08 08:29:48 +00:00
}
else
{
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
}
2012-12-05 12:44:55 +00:00
}
bool KeyCondition::addCondition(const String & column, const Range & range)
{
if (!key_columns.contains(column))
return false;
rpn.emplace_back(RPNElement::FUNCTION_IN_RANGE, key_columns[column], range);
2015-03-27 03:37:46 +00:00
rpn.emplace_back(RPNElement::FUNCTION_AND);
return true;
}
/** Computes value of constant expression and its data type.
* Returns false, if expression isn't constant.
*/
2019-02-20 16:24:46 +00:00
bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type)
2012-12-05 12:44:55 +00:00
{
2022-08-04 15:23:10 +00:00
return Tree(expr.get()).tryGetConstant(block_with_constants, out_value, out_type);
2012-12-05 12:44:55 +00:00
}
static Field applyFunctionForField(
const FunctionBasePtr & func,
const DataTypePtr & arg_type,
const Field & arg_value)
{
ColumnsWithTypeAndName columns
{
{ arg_type->createColumnConst(1, arg_value), arg_type, "x" },
};
2020-10-19 18:37:44 +00:00
auto col = func->execute(columns, func->getResultType(), 1);
return (*col)[0];
}
2020-08-02 20:55:20 +00:00
/// The case when arguments may have types different than in the primary key.
2020-08-02 21:01:39 +00:00
static std::pair<Field, DataTypePtr> applyFunctionForFieldOfUnknownType(
const FunctionBasePtr & func,
2020-08-02 20:55:20 +00:00
const DataTypePtr & arg_type,
const Field & arg_value)
{
2020-10-19 18:37:44 +00:00
ColumnsWithTypeAndName arguments{{ arg_type->createColumnConst(1, arg_value), arg_type, "x" }};
DataTypePtr return_type = func->getResultType();
auto col = func->execute(arguments, return_type, 1);
2020-10-19 18:37:44 +00:00
Field result = (*col)[0];
2020-08-02 21:01:39 +00:00
return {std::move(result), std::move(return_type)};
2020-08-02 20:55:20 +00:00
}
2020-11-06 03:50:58 +00:00
/// Same as above but for binary operators
static std::pair<Field, DataTypePtr> applyBinaryFunctionForFieldOfUnknownType(
const FunctionOverloadResolverPtr & func,
const DataTypePtr & arg_type,
const Field & arg_value,
const DataTypePtr & arg_type2,
const Field & arg_value2)
{
ColumnsWithTypeAndName arguments{
{arg_type->createColumnConst(1, arg_value), arg_type, "x"}, {arg_type2->createColumnConst(1, arg_value2), arg_type2, "y"}};
FunctionBasePtr func_base = func->build(arguments);
DataTypePtr return_type = func_base->getResultType();
auto col = func_base->execute(arguments, return_type, 1);
Field result = (*col)[0];
return {std::move(result), std::move(return_type)};
}
2020-07-21 11:02:58 +00:00
static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field)
{
/// Fallback for fields without block reference.
if (field.isExplicit())
return applyFunctionForField(func, current_type, field);
String result_name = "_" + func->getName() + "_" + toString(field.column_idx);
const auto & columns = field.columns;
size_t result_idx = columns->size();
for (size_t i = 0; i < result_idx; ++i)
{
if ((*columns)[i].name == result_name)
result_idx = i;
}
if (result_idx == columns->size())
{
ColumnsWithTypeAndName args{(*columns)[field.column_idx]};
2020-10-19 18:37:44 +00:00
field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name});
(*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size());
}
return {field.columns, field.row_idx, result_idx};
}
2021-06-02 16:56:24 +00:00
void KeyCondition::traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants)
2019-03-08 08:29:48 +00:00
{
RPNElement element;
2021-06-02 16:56:24 +00:00
if (node.isFunction())
2019-03-08 08:29:48 +00:00
{
2021-06-02 16:56:24 +00:00
auto func = node.asFunction();
if (tryParseLogicalOperatorFromAST(func, element))
2019-03-08 08:29:48 +00:00
{
2021-06-02 16:56:24 +00:00
size_t num_args = func.numArguments();
for (size_t i = 0; i < num_args; ++i)
2019-03-08 08:29:48 +00:00
{
2021-06-02 16:56:24 +00:00
traverseAST(func.getArgumentAt(i), context, block_with_constants);
2019-03-08 08:29:48 +00:00
/** The first part of the condition is for the correct support of `and` and `or` functions of arbitrary arity
* - in this case `n - 1` elements are added (where `n` is the number of arguments).
*/
if (i != 0 || element.function == RPNElement::FUNCTION_NOT)
2020-03-18 02:02:24 +00:00
rpn.emplace_back(element);
2019-03-08 08:29:48 +00:00
}
return;
}
}
if (!tryParseAtomFromAST(node, context, block_with_constants, element))
2019-03-08 08:29:48 +00:00
{
element.function = RPNElement::FUNCTION_UNKNOWN;
}
rpn.emplace_back(std::move(element));
}
2021-08-28 15:33:18 +00:00
/** The key functional expression constraint may be inferred from a plain column in the expression.
* For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`,
* it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())`
* condition also holds, so the index may be used to select only parts satisfying this condition.
*
* To check the assumption, we'd need to assert that the inverse function to this transformation is also monotonic, however the
* inversion isn't exported (or even viable for not strictly monotonic functions such as `toStartOfHour()`).
* Instead, we can qualify only functions that do not transform the range (for example rounding),
* which while not strictly monotonic, are monotonic everywhere on the input range.
*/
bool KeyCondition::transformConstantWithValidFunctions(
const String & expr_name,
2021-06-15 13:47:37 +00:00
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
2021-08-28 15:33:18 +00:00
DataTypePtr & out_type,
std::function<bool(IFunctionBase &, const IDataType &)> always_monotonic) const
2021-05-24 23:39:56 +00:00
{
const auto & sample_block = key_expr->getSampleBlock();
2021-06-03 14:44:59 +00:00
2021-08-28 15:33:18 +00:00
for (const auto & node : key_expr->getNodes())
{
auto it = key_columns.find(node.result_name);
if (it != key_columns.end())
2021-06-03 14:44:59 +00:00
{
std::stack<const ActionsDAG::Node *> chain;
2021-06-03 14:44:59 +00:00
2021-08-28 15:33:18 +00:00
const auto * cur_node = &node;
bool is_valid_chain = true;
2021-06-03 14:44:59 +00:00
while (is_valid_chain)
{
if (cur_node->result_name == expr_name)
break;
2021-06-03 14:44:59 +00:00
chain.push(cur_node);
2021-08-28 15:33:18 +00:00
if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2)
{
2021-08-28 15:33:18 +00:00
is_valid_chain = always_monotonic(*cur_node->function_base, *cur_node->result_type);
2021-06-03 14:44:59 +00:00
2021-08-28 15:33:18 +00:00
const ActionsDAG::Node * next_node = nullptr;
for (const auto * arg : cur_node->children)
2021-06-07 13:41:40 +00:00
{
2021-08-28 15:33:18 +00:00
if (arg->column && isColumnConst(*arg->column))
continue;
if (next_node)
2021-06-07 13:41:40 +00:00
is_valid_chain = false;
2021-08-28 15:33:18 +00:00
next_node = arg;
}
2021-08-28 15:33:18 +00:00
if (!next_node)
is_valid_chain = false;
2021-06-07 13:41:40 +00:00
cur_node = next_node;
}
else if (cur_node->type == ActionsDAG::ActionType::ALIAS)
cur_node = cur_node->children.front();
else
is_valid_chain = false;
}
2021-08-28 15:33:18 +00:00
if (is_valid_chain)
2021-06-03 14:44:59 +00:00
{
auto const_type = cur_node->result_type;
auto const_column = out_type->createColumnConst(1, out_value);
2022-03-18 08:18:29 +00:00
auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0];
2022-03-18 08:18:29 +00:00
if (const_value.isNull())
return false;
while (!chain.empty())
{
const auto * func = chain.top();
chain.pop();
if (func->type != ActionsDAG::ActionType::FUNCTION)
continue;
2021-08-28 15:33:18 +00:00
if (func->children.size() == 1)
{
std::tie(const_value, const_type)
= applyFunctionForFieldOfUnknownType(func->function_base, const_type, const_value);
}
else if (func->children.size() == 2)
{
const auto * left = func->children[0];
const auto * right = func->children[1];
if (left->column && isColumnConst(*left->column))
{
auto left_arg_type = left->result_type;
auto left_arg_value = (*left->column)[0];
std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType(
func->function_builder, left_arg_type, left_arg_value, const_type, const_value);
}
else
{
auto right_arg_type = right->result_type;
auto right_arg_value = (*right->column)[0];
std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType(
func->function_builder, const_type, const_value, right_arg_type, right_arg_value);
}
}
}
2021-06-03 14:44:59 +00:00
out_key_column_num = it->second;
out_key_column_type = sample_block.getByName(it->first).type;
2021-06-03 14:44:59 +00:00
out_value = const_value;
out_type = const_type;
return true;
2021-06-03 14:44:59 +00:00
}
}
}
2022-07-19 17:33:14 +00:00
return false;
}
2021-08-28 15:33:18 +00:00
bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
2022-07-19 17:33:14 +00:00
const Tree & node,
2021-08-28 15:33:18 +00:00
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
DataTypePtr & out_type)
{
2022-07-19 17:33:14 +00:00
String expr_name = node.getColumnName();
2021-08-28 15:33:18 +00:00
if (array_joined_columns.contains(expr_name))
2021-08-28 15:33:18 +00:00
return false;
if (!key_subexpr_names.contains(expr_name))
2021-08-28 15:33:18 +00:00
return false;
if (out_value.isNull())
return false;
return transformConstantWithValidFunctions(
expr_name, out_key_column_num, out_key_column_type, out_value, out_type, [](IFunctionBase & func, const IDataType & type)
{
if (!func.hasInformationAboutMonotonicity())
return false;
else
{
/// Range is irrelevant in this case.
auto monotonicity = func.getMonotonicityForRange(type, Field(), Field());
if (!monotonicity.is_always_monotonic)
return false;
}
return true;
});
}
2020-11-06 03:50:58 +00:00
/// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)`
bool KeyCondition::canConstantBeWrappedByFunctions(
2022-07-19 17:33:14 +00:00
const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type)
2020-11-06 03:50:58 +00:00
{
2022-07-19 17:33:14 +00:00
String expr_name = node.getColumnName();
if (array_joined_columns.contains(expr_name))
return false;
if (!key_subexpr_names.contains(expr_name))
{
/// Let's check another one case.
/// If our storage was created with moduloLegacy in partition key,
/// We can assume that `modulo(...) = const` is the same as `moduloLegacy(...) = const`.
/// Replace modulo to moduloLegacy in AST and check if we also have such a column.
///
2021-06-15 18:42:26 +00:00
/// We do not check this in canConstantBeWrappedByMonotonicFunctions.
2022-07-19 17:33:14 +00:00
/// The case `f(modulo(...))` for totally monotonic `f ` is considered to be rare.
2021-06-15 18:42:26 +00:00
///
/// Note: for negative values, we can filter more partitions then needed.
2022-07-19 17:33:14 +00:00
expr_name = node.getColumnNameLegacy();
if (!key_subexpr_names.contains(expr_name))
return false;
}
2020-11-06 11:18:42 +00:00
if (out_value.isNull())
return false;
2021-08-28 15:33:18 +00:00
return transformConstantWithValidFunctions(
expr_name, out_key_column_num, out_key_column_type, out_value, out_type, [](IFunctionBase & func, const IDataType &)
2020-11-06 03:50:58 +00:00
{
2021-08-28 15:33:18 +00:00
return func.isDeterministic();
});
2020-11-06 03:50:58 +00:00
}
2018-07-02 18:57:14 +00:00
bool KeyCondition::tryPrepareSetIndex(
2021-06-03 12:26:02 +00:00
const FunctionTree & func,
ContextPtr context,
RPNElement & out,
size_t & out_key_column_num)
2018-02-08 14:15:21 +00:00
{
2021-06-03 12:26:02 +00:00
const auto & left_arg = func.getArgumentAt(0);
2018-07-02 18:57:14 +00:00
out_key_column_num = 0;
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> indexes_mapping;
DataTypes data_types;
2021-06-03 12:26:02 +00:00
auto get_key_tuple_position_mapping = [&](const Tree & node, size_t tuple_index)
{
MergeTreeSetIndex::KeyTuplePositionMapping index_mapping;
index_mapping.tuple_index = tuple_index;
DataTypePtr data_type;
if (isKeyPossiblyWrappedByMonotonicFunctions(
node, context, index_mapping.key_index, data_type, index_mapping.functions))
{
indexes_mapping.push_back(index_mapping);
data_types.push_back(data_type);
if (out_key_column_num < index_mapping.key_index)
out_key_column_num = index_mapping.key_index;
}
};
2019-08-19 16:51:50 +00:00
size_t left_args_count = 1;
2021-06-03 12:26:02 +00:00
if (left_arg.isFunction())
{
2021-06-03 12:26:02 +00:00
/// Note: in case of ActionsDAG, tuple may be a constant.
/// In this case, there is no keys in tuple. So, we don't have to check it.
auto left_arg_tuple = left_arg.asFunction();
if (left_arg_tuple.getFunctionName() == "tuple")
{
left_args_count = left_arg_tuple.numArguments();
for (size_t i = 0; i < left_args_count; ++i)
get_key_tuple_position_mapping(left_arg_tuple.getArgumentAt(i), i);
}
2021-06-21 17:28:15 +00:00
else
get_key_tuple_position_mapping(left_arg, 0);
}
else
get_key_tuple_position_mapping(left_arg, 0);
if (indexes_mapping.empty())
return false;
2021-06-03 12:26:02 +00:00
const auto right_arg = func.getArgumentAt(1);
2022-07-18 15:53:30 +00:00
2021-06-03 12:26:02 +00:00
auto prepared_set = right_arg.tryGetPreparedSet(prepared_sets, indexes_mapping, data_types);
if (!prepared_set)
2022-07-18 15:53:30 +00:00
return false;
/// The index can be prepared if the elements of the set were saved in advance.
if (!prepared_set->hasExplicitSetElements())
return false;
2019-08-19 16:51:50 +00:00
prepared_set->checkColumnsNumber(left_args_count);
for (size_t i = 0; i < indexes_mapping.size(); ++i)
prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]);
2019-08-19 16:51:50 +00:00
2018-06-30 21:35:01 +00:00
out.set_index = std::make_shared<MergeTreeSetIndex>(prepared_set->getSetElements(), std::move(indexes_mapping));
return true;
}
2021-01-15 11:36:07 +00:00
/** Allow to use two argument function with constant argument to be analyzed as a single argument function.
* In other words, it performs "currying" (binding of arguments).
* This is needed, for example, to support correct analysis of `toDate(time, 'UTC')`.
*/
class FunctionWithOptionalConstArg : public IFunctionBase
{
public:
enum Kind
{
NO_CONST = 0,
LEFT_CONST,
RIGHT_CONST,
};
2020-12-30 14:58:43 +00:00
explicit FunctionWithOptionalConstArg(const FunctionBasePtr & func_) : func(func_) {}
FunctionWithOptionalConstArg(const FunctionBasePtr & func_, const ColumnWithTypeAndName & const_arg_, Kind kind_)
: func(func_), const_arg(const_arg_), kind(kind_)
{
}
String getName() const override { return func->getName(); }
const DataTypes & getArgumentTypes() const override { return func->getArgumentTypes(); }
const DataTypePtr & getResultType() const override { return func->getResultType(); }
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & arguments) const override { return func->prepare(arguments); }
ColumnPtr
2020-12-31 03:06:15 +00:00
execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const override
{
if (kind == Kind::LEFT_CONST)
{
ColumnsWithTypeAndName new_arguments;
new_arguments.reserve(arguments.size() + 1);
new_arguments.push_back(const_arg);
for (const auto & arg : arguments)
new_arguments.push_back(arg);
return func->prepare(new_arguments)->execute(new_arguments, result_type, input_rows_count, dry_run);
}
else if (kind == Kind::RIGHT_CONST)
{
auto new_arguments = arguments;
new_arguments.push_back(const_arg);
return func->prepare(new_arguments)->execute(new_arguments, result_type, input_rows_count, dry_run);
}
else
return func->prepare(arguments)->execute(arguments, result_type, input_rows_count, dry_run);
}
bool isDeterministic() const override { return func->isDeterministic(); }
bool isDeterministicInScopeOfQuery() const override { return func->isDeterministicInScopeOfQuery(); }
bool hasInformationAboutMonotonicity() const override { return func->hasInformationAboutMonotonicity(); }
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override { return func->isSuitableForShortCircuitArgumentsExecution(arguments); }
IFunctionBase::Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
{
return func->getMonotonicityForRange(type, left, right);
}
2021-04-15 17:30:04 +00:00
Kind getKind() const { return kind; }
const ColumnWithTypeAndName & getConstArg() const { return const_arg; }
private:
FunctionBasePtr func;
ColumnWithTypeAndName const_arg;
Kind kind = Kind::NO_CONST;
};
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
2021-06-02 16:56:24 +00:00
const Tree & node,
ContextPtr context,
size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type,
MonotonicFunctionsChain & out_functions_chain)
2012-12-05 12:44:55 +00:00
{
2021-06-02 16:56:24 +00:00
std::vector<FunctionTree> chain_not_tested_for_monotonicity;
DataTypePtr key_column_type;
if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity))
return false;
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
{
2021-06-03 12:26:02 +00:00
auto function = *it;
auto func_builder = FunctionFactory::instance().tryGet(function.getFunctionName(), context);
2021-02-28 07:25:56 +00:00
if (!func_builder)
return false;
2020-09-05 14:12:47 +00:00
ColumnsWithTypeAndName arguments;
ColumnWithTypeAndName const_arg;
FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST;
2021-06-03 12:26:02 +00:00
if (function.numArguments() == 2)
2020-09-05 14:12:47 +00:00
{
2021-06-03 12:26:02 +00:00
if (function.getArgumentAt(0).isConstant())
2020-09-05 14:12:47 +00:00
{
2021-06-03 12:26:02 +00:00
const_arg = function.getArgumentAt(0).getConstant();
arguments.push_back(const_arg);
2020-09-05 14:12:47 +00:00
arguments.push_back({ nullptr, key_column_type, "" });
kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST;
2020-09-05 14:12:47 +00:00
}
2021-06-03 12:26:02 +00:00
else if (function.getArgumentAt(1).isConstant())
2020-09-05 14:12:47 +00:00
{
arguments.push_back({ nullptr, key_column_type, "" });
2021-06-03 12:26:02 +00:00
const_arg = function.getArgumentAt(1).getConstant();
arguments.push_back(const_arg);
kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST;
2020-09-05 14:12:47 +00:00
}
}
else
arguments.push_back({ nullptr, key_column_type, "" });
2018-02-02 08:33:36 +00:00
auto func = func_builder->build(arguments);
2020-09-21 10:13:01 +00:00
/// If we know the given range only contains one value, then we treat all functions as positive monotonic.
if (!func || (!single_point && !func->hasInformationAboutMonotonicity()))
return false;
2020-10-19 18:37:44 +00:00
key_column_type = func->getResultType();
if (kind == FunctionWithOptionalConstArg::Kind::NO_CONST)
out_functions_chain.push_back(func);
else
out_functions_chain.push_back(std::make_shared<FunctionWithOptionalConstArg>(func, const_arg, kind));
}
out_key_res_column_type = key_column_type;
return true;
}
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
2021-06-02 16:56:24 +00:00
const Tree & node,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
2021-06-02 16:56:24 +00:00
std::vector<FunctionTree> & out_functions_chain)
{
/** By itself, the key column can be a functional expression. for example, `intHash32(UserID)`.
2017-03-13 18:01:46 +00:00
* Therefore, use the full name of the expression for search.
*/
const auto & sample_block = key_expr->getSampleBlock();
2021-03-16 06:07:30 +00:00
// Key columns should use canonical names for index analysis
2021-06-02 16:56:24 +00:00
String name = node.getColumnName();
if (array_joined_columns.contains(name))
return false;
auto it = key_columns.find(name);
if (key_columns.end() != it)
{
out_key_column_num = it->second;
out_key_column_type = sample_block.getByName(it->first).type;
return true;
}
2021-06-02 16:56:24 +00:00
if (node.isFunction())
{
2021-06-02 16:56:24 +00:00
auto func = node.asFunction();
2021-06-02 16:56:24 +00:00
size_t num_args = func.numArguments();
if (num_args > 2 || num_args == 0)
return false;
out_functions_chain.push_back(func);
2020-09-05 14:12:47 +00:00
bool ret = false;
2021-06-02 16:56:24 +00:00
if (num_args == 2)
2020-09-05 14:12:47 +00:00
{
2021-06-02 16:56:24 +00:00
if (func.getArgumentAt(0).isConstant())
2020-09-05 14:12:47 +00:00
{
2021-06-02 16:56:24 +00:00
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain);
2020-09-05 14:12:47 +00:00
}
2021-06-02 16:56:24 +00:00
else if (func.getArgumentAt(1).isConstant())
2020-09-05 14:12:47 +00:00
{
2021-06-02 16:56:24 +00:00
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
2020-09-05 14:12:47 +00:00
}
}
else
{
2021-06-02 16:56:24 +00:00
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
2020-09-05 14:12:47 +00:00
}
return ret;
}
return false;
}
2021-06-21 16:17:05 +00:00
static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const KeyCondition::Tree & node)
{
try
{
src_value = convertFieldToType(src_value, *desired_type, src_type.get());
}
catch (...)
{
throw Exception("Key expression contains comparison between inconvertible types: " +
desired_type->getName() + " and " + src_type->getName() +
2021-06-21 16:17:05 +00:00
" inside " + node.getColumnName(),
ErrorCodes::BAD_TYPE_OF_FIELD);
}
}
2021-06-02 16:56:24 +00:00
bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out)
{
/** Functions < > = != <= >= in `notIn` isNull isNotNull, where one argument is a constant, and the other is one of columns of key,
2017-11-15 19:47:49 +00:00
* or itself, wrapped in a chain of possibly-monotonic functions,
2017-03-13 18:01:46 +00:00
* or constant expression - number.
*/
Field const_value;
DataTypePtr const_type;
2021-06-02 16:56:24 +00:00
if (node.isFunction())
2012-12-05 12:44:55 +00:00
{
2021-06-02 16:56:24 +00:00
auto func = node.asFunction();
size_t num_args = func.numArguments();
DataTypePtr key_expr_type; /// Type of expression containing key column
2019-01-18 16:30:35 +00:00
size_t key_column_num = -1; /// Number of a key column (inside key_column_names array)
MonotonicFunctionsChain chain;
2021-06-02 16:56:24 +00:00
std::string func_name = func.getFunctionName();
if (atom_map.find(func_name) == std::end(atom_map))
return false;
2021-06-02 16:56:24 +00:00
if (num_args == 1)
2012-12-05 12:44:55 +00:00
{
2021-06-02 16:56:24 +00:00
if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain)))
return false;
if (key_column_num == static_cast<size_t>(-1))
throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR);
2012-12-05 12:44:55 +00:00
}
2021-06-02 16:56:24 +00:00
else if (num_args == 2)
{
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
bool is_set_const = false;
bool is_constant_transformed = false;
/// We don't look for inversed key transformations when strict is true, which is required for trivial count().
/// Consider the following test case:
///
/// create table test1(p DateTime, k int) engine MergeTree partition by toDate(p) order by k;
/// insert into test1 values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3);
/// select count() from test1 where p > toDateTime('2020-09-01 10:00:00');
///
2021-04-24 02:34:35 +00:00
/// toDate(DateTime) is always monotonic, but we cannot relax the predicates to be
/// >= toDate(toDateTime('2020-09-01 10:00:00')), which returns 3 instead of the right count: 2.
2021-04-20 06:15:28 +00:00
bool strict_condition = strict;
2021-04-20 06:15:28 +00:00
/// If we use this key condition to prune partitions by single value, we cannot relax conditions for NOT.
if (single_point
&& (func_name == "notLike" || func_name == "notIn" || func_name == "globalNotIn" || func_name == "notNullIn"
|| func_name == "globalNotNullIn" || func_name == "notEquals" || func_name == "notEmpty"))
2021-04-20 06:15:28 +00:00
strict_condition = true;
2020-11-06 11:18:42 +00:00
if (functionIsInOrGlobalInOperator(func_name))
{
2021-06-03 12:26:02 +00:00
if (tryPrepareSetIndex(func, context, out, key_column_num))
2020-11-06 11:18:42 +00:00
{
key_arg_pos = 0;
is_set_const = true;
}
else
return false;
}
2022-08-04 15:23:10 +00:00
else if (func.getArgumentAt(1).tryGetConstant(block_with_constants, const_value, const_type))
{
2021-06-03 12:27:38 +00:00
if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain))
2020-11-06 03:50:58 +00:00
{
key_arg_pos = 0;
}
2021-04-20 06:15:28 +00:00
else if (
!strict_condition
2021-06-03 12:27:38 +00:00
&& canConstantBeWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, const_value, const_type))
2020-11-06 03:50:58 +00:00
{
key_arg_pos = 0;
is_constant_transformed = true;
}
else if (
2021-04-20 06:15:28 +00:00
single_point && func_name == "equals" && !strict_condition
2021-06-03 12:27:38 +00:00
&& canConstantBeWrappedByFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, const_value, const_type))
2020-11-06 03:50:58 +00:00
{
key_arg_pos = 0;
is_constant_transformed = true;
}
else
return false;
}
2022-08-04 15:23:10 +00:00
else if (func.getArgumentAt(0).tryGetConstant(block_with_constants, const_value, const_type))
{
2021-06-21 16:17:05 +00:00
if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), context, key_column_num, key_expr_type, chain))
2020-11-06 03:50:58 +00:00
{
key_arg_pos = 1;
}
2021-04-20 06:15:28 +00:00
else if (
!strict_condition
2021-06-21 16:17:05 +00:00
&& canConstantBeWrappedByMonotonicFunctions(func.getArgumentAt(1), key_column_num, key_expr_type, const_value, const_type))
2020-11-06 03:50:58 +00:00
{
key_arg_pos = 1;
is_constant_transformed = true;
}
else if (
2021-04-20 06:15:28 +00:00
single_point && func_name == "equals" && !strict_condition
2021-06-21 16:17:05 +00:00
&& canConstantBeWrappedByFunctions(func.getArgumentAt(1), key_column_num, key_expr_type, const_value, const_type))
2020-11-06 03:50:58 +00:00
{
key_arg_pos = 0;
is_constant_transformed = true;
}
else
return false;
}
else
return false;
2019-01-18 16:30:35 +00:00
if (key_column_num == static_cast<size_t>(-1))
throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR);
/// Replace <const> <sign> <data> on to <data> <-sign> <const>
if (key_arg_pos == 1)
{
if (func_name == "less")
func_name = "greater";
else if (func_name == "greater")
func_name = "less";
else if (func_name == "greaterOrEquals")
func_name = "lessOrEquals";
else if (func_name == "lessOrEquals")
func_name = "greaterOrEquals";
else if (func_name == "in" || func_name == "notIn" ||
func_name == "like" || func_name == "notLike" ||
func_name == "ilike" || func_name == "notIlike" ||
func_name == "startsWith")
{
/// "const IN data_column" doesn't make sense (unlike "data_column IN const")
return false;
}
}
key_expr_type = recursiveRemoveLowCardinality(key_expr_type);
DataTypePtr key_expr_type_not_null;
bool key_expr_type_is_nullable = false;
if (const auto * nullable_type = typeid_cast<const DataTypeNullable *>(key_expr_type.get()))
{
key_expr_type_is_nullable = true;
key_expr_type_not_null = nullable_type->getNestedType();
}
else
key_expr_type_not_null = key_expr_type;
bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST
|| ((isNativeInteger(key_expr_type_not_null) || isDateTime(key_expr_type_not_null))
&& (isNativeInteger(const_type) || isDateTime(const_type))); /// Native integers and DateTime are accurately compared without cast.
if (!cast_not_needed && !key_expr_type_not_null->equals(*const_type))
{
if (const_value.getType() == Field::Types::String)
{
const_value = convertFieldToType(const_value, *key_expr_type_not_null);
if (const_value.isNull())
return false;
// No need to set is_constant_transformed because we're doing exact conversion
}
else
{
DataTypePtr common_type = tryGetLeastSupertype(DataTypes{key_expr_type_not_null, const_type});
2022-01-20 13:56:12 +00:00
if (!common_type)
return false;
if (!const_type->equals(*common_type))
{
castValueToType(common_type, const_value, const_type, node);
// Need to set is_constant_transformed unless we're doing exact conversion
if (!key_expr_type_not_null->equals(*common_type))
is_constant_transformed = true;
}
if (!key_expr_type_not_null->equals(*common_type))
{
2021-11-25 16:36:37 +00:00
auto common_type_maybe_nullable = (key_expr_type_is_nullable && !common_type->isNullable())
? DataTypePtr(std::make_shared<DataTypeNullable>(common_type))
: common_type;
ColumnsWithTypeAndName arguments{
{nullptr, key_expr_type, ""},
{DataTypeString().createColumnConst(1, common_type_maybe_nullable->getName()), common_type_maybe_nullable, ""}};
2021-08-07 08:11:40 +00:00
FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver<CastType::nonAccurate>::createImpl();
auto func_cast = func_builder_cast->build(arguments);
/// If we know the given range only contains one value, then we treat all functions as positive monotonic.
2021-11-25 16:36:37 +00:00
if (!single_point && !func_cast->hasInformationAboutMonotonicity())
return false;
chain.push_back(func_cast);
}
}
}
/// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5"
if (is_constant_transformed)
{
if (func_name == "less")
func_name = "lessOrEquals";
else if (func_name == "greater")
func_name = "greaterOrEquals";
}
}
else
return false;
const auto atom_it = atom_map.find(func_name);
out.key_column = key_column_num;
out.monotonic_functions_chain = std::move(chain);
return atom_it->second(out, const_value);
2012-12-05 12:44:55 +00:00
}
2022-08-04 15:23:10 +00:00
else if (node.tryGetConstant(block_with_constants, const_value, const_type))
{
/// For cases where it says, for example, `WHERE 0 AND something`
if (const_value.getType() == Field::Types::UInt64)
{
out.function = const_value.safeGet<UInt64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
return true;
}
else if (const_value.getType() == Field::Types::Int64)
{
out.function = const_value.safeGet<Int64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
return true;
}
else if (const_value.getType() == Field::Types::Float64)
{
2022-09-11 01:21:34 +00:00
out.function = const_value.safeGet<Float64>() != 0.0 ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
return true;
}
}
2012-12-05 12:44:55 +00:00
return false;
}
2021-06-02 16:56:24 +00:00
bool KeyCondition::tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out)
2019-03-08 08:29:48 +00:00
{
/// Functions AND, OR, NOT.
2021-02-28 07:25:56 +00:00
/// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses
/// (or, the same thing - calling the function `and` from one argument).
2019-03-08 08:29:48 +00:00
2021-06-02 16:56:24 +00:00
if (func.getFunctionName() == "not")
2019-03-08 08:29:48 +00:00
{
2021-06-02 16:56:24 +00:00
if (func.numArguments() != 1)
2019-03-08 08:29:48 +00:00
return false;
out.function = RPNElement::FUNCTION_NOT;
}
else
{
2021-06-02 16:56:24 +00:00
if (func.getFunctionName() == "and" || func.getFunctionName() == "indexHint")
2019-03-08 08:29:48 +00:00
out.function = RPNElement::FUNCTION_AND;
2021-06-02 16:56:24 +00:00
else if (func.getFunctionName() == "or")
2019-03-08 08:29:48 +00:00
out.function = RPNElement::FUNCTION_OR;
else
return false;
}
return true;
}
String KeyCondition::toString() const
2012-12-05 12:44:55 +00:00
{
String res;
for (size_t i = 0; i < rpn.size(); ++i)
{
if (i)
res += ", ";
res += rpn[i].toString();
}
return res;
}
2021-04-15 17:30:04 +00:00
KeyCondition::Description KeyCondition::getDescription() const
{
2021-04-16 09:42:23 +00:00
/// This code may seem to be too difficult.
/// Here we want to convert RPN back to tree, and also simplify some logical expressions like `and(x, true) -> x`.
2021-04-15 17:30:04 +00:00
Description description;
2021-04-16 09:42:23 +00:00
/// That's a binary tree. Explicit.
/// Build and optimize it simultaneously.
2021-04-15 17:30:04 +00:00
struct Node
{
enum class Type
{
2021-04-16 09:42:23 +00:00
/// Leaf, which is RPNElement.
2021-04-15 17:30:04 +00:00
Leaf,
2021-04-16 09:42:23 +00:00
/// Leafs, which are logical constants.
2021-04-15 17:30:04 +00:00
True,
False,
2021-04-16 09:42:23 +00:00
/// Binary operators.
2021-04-15 17:30:04 +00:00
And,
Or,
};
2021-04-27 04:22:32 +00:00
Type type{};
2021-04-15 17:30:04 +00:00
/// Only for Leaf
const RPNElement * element = nullptr;
2021-04-16 09:42:23 +00:00
/// This means that logical NOT is applied to leaf.
2021-04-15 17:30:04 +00:00
bool negate = false;
std::unique_ptr<Node> left = nullptr;
std::unique_ptr<Node> right = nullptr;
};
2021-04-16 09:42:23 +00:00
/// The algorithm is the same as in KeyCondition::checkInHyperrectangle
/// We build a pair of trees on stack. For checking if key condition may be true, and if it may be false.
/// We need only `can_be_true` in result.
2021-04-15 17:30:04 +00:00
struct Frame
{
std::unique_ptr<Node> can_be_true;
std::unique_ptr<Node> can_be_false;
};
2021-04-16 09:42:23 +00:00
/// Combine two subtrees using logical operator.
2021-04-15 17:30:04 +00:00
auto combine = [](std::unique_ptr<Node> left, std::unique_ptr<Node> right, Node::Type type)
{
2021-04-16 09:42:23 +00:00
/// Simplify operators with for one constant condition.
2021-04-15 17:30:04 +00:00
if (type == Node::Type::And)
{
/// false AND right
if (left->type == Node::Type::False)
return left;
/// left AND false
if (right->type == Node::Type::False)
return right;
/// true AND right
if (left->type == Node::Type::True)
return right;
/// left AND true
if (right->type == Node::Type::True)
return left;
}
if (type == Node::Type::Or)
{
/// false OR right
if (left->type == Node::Type::False)
return right;
/// left OR false
if (right->type == Node::Type::False)
return left;
/// true OR right
if (left->type == Node::Type::True)
return left;
/// left OR true
if (right->type == Node::Type::True)
return right;
}
return std::make_unique<Node>(Node{
.type = type,
.left = std::move(left),
.right = std::move(right)
});
};
std::vector<Frame> rpn_stack;
for (const auto & element : rpn)
{
if (element.function == RPNElement::FUNCTION_UNKNOWN)
{
auto can_be_true = std::make_unique<Node>(Node{.type = Node::Type::True});
auto can_be_false = std::make_unique<Node>(Node{.type = Node::Type::True});
rpn_stack.emplace_back(Frame{.can_be_true = std::move(can_be_true), .can_be_false = std::move(can_be_false)});
}
else if (
element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|| element.function == RPNElement::FUNCTION_IS_NULL
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL
2021-04-15 17:30:04 +00:00
|| element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
auto can_be_true = std::make_unique<Node>(Node{.type = Node::Type::Leaf, .element = &element, .negate = false});
auto can_be_false = std::make_unique<Node>(Node{.type = Node::Type::Leaf, .element = &element, .negate = true});
rpn_stack.emplace_back(Frame{.can_be_true = std::move(can_be_true), .can_be_false = std::move(can_be_false)});
}
else if (element.function == RPNElement::FUNCTION_NOT)
{
assert(!rpn_stack.empty());
std::swap(rpn_stack.back().can_be_true, rpn_stack.back().can_be_false);
}
else if (element.function == RPNElement::FUNCTION_AND)
{
assert(!rpn_stack.empty());
auto arg1 = std::move(rpn_stack.back());
rpn_stack.pop_back();
assert(!rpn_stack.empty());
auto arg2 = std::move(rpn_stack.back());
Frame frame;
frame.can_be_true = combine(std::move(arg1.can_be_true), std::move(arg2.can_be_true), Node::Type::And);
frame.can_be_false = combine(std::move(arg1.can_be_false), std::move(arg2.can_be_false), Node::Type::Or);
rpn_stack.back() = std::move(frame);
}
else if (element.function == RPNElement::FUNCTION_OR)
{
assert(!rpn_stack.empty());
auto arg1 = std::move(rpn_stack.back());
rpn_stack.pop_back();
assert(!rpn_stack.empty());
auto arg2 = std::move(rpn_stack.back());
Frame frame;
frame.can_be_true = combine(std::move(arg1.can_be_true), std::move(arg2.can_be_true), Node::Type::Or);
frame.can_be_false = combine(std::move(arg1.can_be_false), std::move(arg2.can_be_false), Node::Type::And);
rpn_stack.back() = std::move(frame);
}
else if (element.function == RPNElement::ALWAYS_FALSE)
{
auto can_be_true = std::make_unique<Node>(Node{.type = Node::Type::False});
auto can_be_false = std::make_unique<Node>(Node{.type = Node::Type::True});
rpn_stack.emplace_back(Frame{.can_be_true = std::move(can_be_true), .can_be_false = std::move(can_be_false)});
}
else if (element.function == RPNElement::ALWAYS_TRUE)
{
auto can_be_true = std::make_unique<Node>(Node{.type = Node::Type::True});
auto can_be_false = std::make_unique<Node>(Node{.type = Node::Type::False});
rpn_stack.emplace_back(Frame{.can_be_true = std::move(can_be_true), .can_be_false = std::move(can_be_false)});
}
else
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
}
if (rpn_stack.size() != 1)
throw Exception("Unexpected stack size in KeyCondition::checkInRange", ErrorCodes::LOGICAL_ERROR);
std::vector<std::string_view> key_names(key_columns.size());
std::vector<bool> is_key_used(key_columns.size(), false);
for (const auto & key : key_columns)
key_names[key.second] = key.first;
2021-04-16 09:42:23 +00:00
WriteBufferFromOwnString buf;
std::function<void(const Node *)> describe;
describe = [&describe, &key_names, &is_key_used, &buf](const Node * node)
2021-04-15 17:30:04 +00:00
{
switch (node->type)
{
case Node::Type::Leaf:
{
is_key_used[node->element->key_column] = true;
2021-04-16 09:42:23 +00:00
/// Note: for condition with double negation, like `not(x not in set)`,
/// we can replace it to `x in set` here.
/// But I won't do it, because `cloneASTWithInversionPushDown` already push down `not`.
/// So, this seem to be impossible for `can_be_true` tree.
2021-04-15 17:30:04 +00:00
if (node->negate)
2021-04-16 09:42:23 +00:00
buf << "not(";
buf << node->element->toString(key_names[node->element->key_column], true);
2021-04-15 17:30:04 +00:00
if (node->negate)
2021-04-16 09:42:23 +00:00
buf << ")";
break;
2021-04-15 17:30:04 +00:00
}
case Node::Type::True:
2021-04-16 09:42:23 +00:00
buf << "true";
break;
2021-04-15 17:30:04 +00:00
case Node::Type::False:
2021-04-16 09:42:23 +00:00
buf << "false";
break;
2021-04-15 17:30:04 +00:00
case Node::Type::And:
2021-04-16 09:42:23 +00:00
buf << "and(";
describe(node->left.get());
buf << ", ";
describe(node->right.get());
buf << ")";
break;
2021-04-15 17:30:04 +00:00
case Node::Type::Or:
2021-04-16 09:42:23 +00:00
buf << "or(";
describe(node->left.get());
buf << ", ";
describe(node->right.get());
buf << ")";
break;
2021-04-15 17:30:04 +00:00
}
};
2021-04-16 09:42:23 +00:00
describe(rpn_stack.front().can_be_true.get());
description.condition = std::move(buf.str());
2021-04-15 17:30:04 +00:00
for (size_t i = 0; i < key_names.size(); ++i)
if (is_key_used[i])
description.used_keys.emplace_back(key_names[i]);
return description;
}
/** Index is the value of key every `index_granularity` rows.
2017-03-12 19:18:07 +00:00
* This value is called a "mark". That is, the index consists of marks.
*
* The key is the tuple.
* The data is sorted by key in the sense of lexicographic order over tuples.
*
2017-03-12 19:18:07 +00:00
* A pair of marks specifies a segment with respect to the order over the tuples.
* Denote it like this: [ x1 y1 z1 .. x2 y2 z2 ],
* where x1 y1 z1 - tuple - value of key in left border of segment;
* x2 y2 z2 - tuple - value of key in right boundary of segment.
2017-03-12 19:18:07 +00:00
* In this section there are data between these marks.
*
2017-03-12 19:18:07 +00:00
* Or, the last mark specifies the range open on the right: [ a b c .. + inf )
*
2017-03-12 19:18:07 +00:00
* The set of all possible tuples can be considered as an n-dimensional space, where n is the size of the tuple.
* A range of tuples specifies some subset of this space.
*
2021-04-13 15:47:11 +00:00
* Hyperrectangles will be the subrange of an n-dimensional space that is a direct product of one-dimensional ranges.
2021-04-13 19:06:24 +00:00
* In this case, the one-dimensional range can be:
2021-04-13 15:47:11 +00:00
* a point, a segment, an open interval, a half-open interval;
* unlimited on the left, unlimited on the right ...
*
2021-04-13 15:47:11 +00:00
* The range of tuples can always be represented as a combination (union) of hyperrectangles.
2020-03-10 14:56:55 +00:00
* For example, the range [ x1 y1 .. x2 y2 ] given x1 != x2 is equal to the union of the following three hyperrectangles:
* [x1] x [y1 .. +inf)
* (x1 .. x2) x (-inf .. +inf)
* [x2] x (-inf .. y2]
*
2020-03-10 14:56:55 +00:00
* Or, for example, the range [ x1 y1 .. +inf ] is equal to the union of the following two hyperrectangles:
* [x1] x [y1 .. +inf)
* (x1 .. +inf) x (-inf .. +inf)
2017-03-12 19:18:07 +00:00
* It's easy to see that this is a special case of the variant above.
*
2020-03-10 14:56:55 +00:00
* This is important because it is easy for us to check the feasibility of the condition over the hyperrectangle,
2017-03-12 19:18:07 +00:00
* and therefore, feasibility of condition on the range of tuples will be checked by feasibility of condition
2020-03-10 14:56:55 +00:00
* over at least one hyperrectangle from which this range consists.
*/
template <typename F>
2020-03-10 14:56:55 +00:00
static BoolMask forAnyHyperrectangle(
size_t key_size,
const FieldRef * left_keys,
const FieldRef * right_keys,
bool left_bounded,
bool right_bounded,
2020-03-10 14:56:55 +00:00
std::vector<Range> & hyperrectangle,
size_t prefix_size,
BoolMask initial_mask,
F && callback)
2012-12-05 12:44:55 +00:00
{
if (!left_bounded && !right_bounded)
2020-03-10 14:56:55 +00:00
return callback(hyperrectangle);
if (left_bounded && right_bounded)
2012-12-05 12:44:55 +00:00
{
2017-03-12 19:18:07 +00:00
/// Let's go through the matching elements of the key.
while (prefix_size < key_size)
2012-12-05 12:44:55 +00:00
{
if (left_keys[prefix_size] == right_keys[prefix_size])
2012-12-06 09:45:09 +00:00
{
2017-03-13 18:01:46 +00:00
/// Point ranges.
hyperrectangle[prefix_size] = Range(left_keys[prefix_size]);
++prefix_size;
2012-12-06 09:45:09 +00:00
}
else
break;
2012-12-05 12:44:55 +00:00
}
}
if (prefix_size == key_size)
2020-03-10 14:56:55 +00:00
return callback(hyperrectangle);
if (prefix_size + 1 == key_size)
{
if (left_bounded && right_bounded)
hyperrectangle[prefix_size] = Range(left_keys[prefix_size], true, right_keys[prefix_size], true);
else if (left_bounded)
hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], true);
else if (right_bounded)
hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], true);
2020-03-10 14:56:55 +00:00
return callback(hyperrectangle);
}
/// (x1 .. x2) x (-inf .. +inf)
if (left_bounded && right_bounded)
hyperrectangle[prefix_size] = Range(left_keys[prefix_size], false, right_keys[prefix_size], false);
else if (left_bounded)
hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], false);
else if (right_bounded)
hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], false);
2016-03-31 17:42:57 +00:00
for (size_t i = prefix_size + 1; i < key_size; ++i)
2020-03-10 14:56:55 +00:00
hyperrectangle[i] = Range();
BoolMask result = initial_mask;
2020-03-10 14:56:55 +00:00
result = result | callback(hyperrectangle);
/// There are several early-exit conditions (like the one below) hereinafter.
/// They are important; in particular, if initial_mask == BoolMask::consider_only_can_be_true
/// (which happens when this routine is called from KeyCondition::mayBeTrueXXX),
/// they provide significant speedup, which may be observed on merge_tree_huge_pk performance test.
2020-01-29 23:36:39 +00:00
if (result.isComplete())
return result;
/// [x1] x [y1 .. +inf)
if (left_bounded)
{
hyperrectangle[prefix_size] = Range(left_keys[prefix_size]);
result = result | forAnyHyperrectangle(key_size, left_keys, right_keys, true, false, hyperrectangle, prefix_size + 1, initial_mask, callback);
2020-01-29 23:36:39 +00:00
if (result.isComplete())
return result;
}
/// [x2] x (-inf .. y2]
if (right_bounded)
{
hyperrectangle[prefix_size] = Range(right_keys[prefix_size]);
result = result | forAnyHyperrectangle(key_size, left_keys, right_keys, false, true, hyperrectangle, prefix_size + 1, initial_mask, callback);
2020-01-29 23:36:39 +00:00
if (result.isComplete())
return result;
}
return result;
}
BoolMask KeyCondition::checkInRange(
size_t used_key_size,
const FieldRef * left_keys,
const FieldRef * right_keys,
const DataTypes & data_types,
BoolMask initial_mask) const
{
std::vector<Range> key_ranges(used_key_size, Range());
// std::cerr << "Checking for: [";
// for (size_t i = 0; i != used_key_size; ++i)
// std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), left_keys[i]);
// std::cerr << " ... ";
// for (size_t i = 0; i != used_key_size; ++i)
// std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), right_keys[i]);
// std::cerr << "]\n";
return forAnyHyperrectangle(used_key_size, left_keys, right_keys, true, true, key_ranges, 0, initial_mask,
2020-03-10 14:56:55 +00:00
[&] (const std::vector<Range> & key_ranges_hyperrectangle)
2016-03-31 17:42:57 +00:00
{
2020-03-10 14:56:55 +00:00
auto res = checkInHyperrectangle(key_ranges_hyperrectangle, data_types);
// std::cerr << "Hyperrectangle: ";
// for (size_t i = 0, size = key_ranges.size(); i != size; ++i)
// std::cerr << (i != 0 ? " x " : "") << key_ranges[i].toString();
// std::cerr << ": " << res.can_be_true << "\n";
2016-03-31 17:42:57 +00:00
return res;
});
}
std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange(
Range key_range,
2020-07-21 11:02:58 +00:00
const MonotonicFunctionsChain & functions,
2020-09-21 10:13:01 +00:00
DataTypePtr current_type,
bool single_point)
{
2020-07-21 11:02:58 +00:00
for (const auto & func : functions)
{
/// We check the monotonicity of each function on a specific range.
2020-09-21 10:13:01 +00:00
/// If we know the given range only contains one value, then we treat all functions as positive monotonic.
IFunction::Monotonicity monotonicity = single_point
? IFunction::Monotonicity{true}
: func->getMonotonicityForRange(*current_type.get(), key_range.left, key_range.right);
if (!monotonicity.is_monotonic)
{
return {};
}
/// If we apply function to open interval, we can get empty intervals in result.
/// E.g. for ('2020-01-03', '2020-01-20') after applying 'toYYYYMM' we will get ('202001', '202001').
/// To avoid this we make range left and right included.
/// Any function that treats NULL specially is not monotonic.
/// Thus we can safely use isNull() as an -Inf/+Inf indicator here.
if (!key_range.left.isNull())
{
key_range.left = applyFunction(func, current_type, key_range.left);
key_range.left_included = true;
}
if (!key_range.right.isNull())
{
key_range.right = applyFunction(func, current_type, key_range.right);
key_range.right_included = true;
}
2020-10-19 18:37:44 +00:00
current_type = func->getResultType();
if (!monotonicity.is_positive)
key_range.invert();
}
return key_range;
}
Optimize PK lookup for queries that match exact PK range Existing code that looks up marks that match the query has a pathological case, when most of the part does in fact match the query. The code works by recursively splitting a part into ranges and then discarding the ranges that definitely do not match the query, based on primary key. The problem is that it requires visiting every mark that matches the query, making the complexity of this sort of look up O(n). For queries that match exact range on the primary key, we can find both left and right parts of the range with O(log 2) complexity. This change implements exactly that. To engage this optimization, the query must: * Have a prefix list of the primary key. * Have only range or single set element constraints for columns. * Have only AND as a boolean operator. Consider a table with `(service, timestamp)` as the primary key. The following conditions will be optimized: * `service = 'foo'` * `service = 'foo' and timestamp >= now() - 3600` * `service in ('foo')` * `service in ('foo') and timestamp >= now() - 3600 and timestamp <= now` The following will fall back to previous lookup algorithm: * `timestamp >= now() - 3600` * `service in ('foo', 'bar') and timestamp >= now() - 3600` * `service = 'foo'` Note that the optimization won't engage when PK has a range expression followed by a point expression, since in that case the range is not continuous. Trace query logging provides the following messages types of messages, each representing a different kind of PK usage for a part: ``` Used optimized inclusion search over index for part 20200711_5710108_5710108_0 with 9 steps Used generic exclusion search over index for part 20200711_5710118_5710228_5 with 1495 steps Not using index on part 20200710_5710473_5710473_0 ``` Number of steps translates to computational complexity. Here's a comparison for before and after for a query over 24h of data: ``` Read 4562944 rows, 148.05 MiB in 45.19249672 sec., 100966 rows/sec., 3.28 MiB/sec. Read 4183040 rows, 135.78 MiB in 0.196279627 sec., 21311636 rows/sec., 691.75 MiB/sec. ``` This is especially useful for queries that read data in order and terminate early to return "last X things" matching a query. See #11564 for more thoughts on this.
2020-07-07 18:10:44 +00:00
// Returns whether the condition is one continuous range of the primary key,
// where every field is matched by range or a single element set.
// This allows to use a more efficient lookup with no extra reads.
bool KeyCondition::matchesExactContinuousRange() const
{
// Not implemented yet.
if (hasMonotonicFunctionsChain())
return false;
enum Constraint
{
POINT,
RANGE,
UNKNOWN,
};
std::vector<Constraint> column_constraints(key_columns.size(), Constraint::UNKNOWN);
for (const auto & element : rpn)
{
if (element.function == RPNElement::Function::FUNCTION_AND)
{
continue;
}
if (element.function == RPNElement::Function::FUNCTION_IN_SET && element.set_index && element.set_index->size() == 1)
{
column_constraints[element.key_column] = Constraint::POINT;
continue;
}
if (element.function == RPNElement::Function::FUNCTION_IN_RANGE)
{
if (element.range.left == element.range.right)
{
column_constraints[element.key_column] = Constraint::POINT;
}
if (column_constraints[element.key_column] != Constraint::POINT)
{
column_constraints[element.key_column] = Constraint::RANGE;
}
continue;
}
if (element.function == RPNElement::Function::FUNCTION_UNKNOWN)
{
continue;
}
Optimize PK lookup for queries that match exact PK range Existing code that looks up marks that match the query has a pathological case, when most of the part does in fact match the query. The code works by recursively splitting a part into ranges and then discarding the ranges that definitely do not match the query, based on primary key. The problem is that it requires visiting every mark that matches the query, making the complexity of this sort of look up O(n). For queries that match exact range on the primary key, we can find both left and right parts of the range with O(log 2) complexity. This change implements exactly that. To engage this optimization, the query must: * Have a prefix list of the primary key. * Have only range or single set element constraints for columns. * Have only AND as a boolean operator. Consider a table with `(service, timestamp)` as the primary key. The following conditions will be optimized: * `service = 'foo'` * `service = 'foo' and timestamp >= now() - 3600` * `service in ('foo')` * `service in ('foo') and timestamp >= now() - 3600 and timestamp <= now` The following will fall back to previous lookup algorithm: * `timestamp >= now() - 3600` * `service in ('foo', 'bar') and timestamp >= now() - 3600` * `service = 'foo'` Note that the optimization won't engage when PK has a range expression followed by a point expression, since in that case the range is not continuous. Trace query logging provides the following messages types of messages, each representing a different kind of PK usage for a part: ``` Used optimized inclusion search over index for part 20200711_5710108_5710108_0 with 9 steps Used generic exclusion search over index for part 20200711_5710118_5710228_5 with 1495 steps Not using index on part 20200710_5710473_5710473_0 ``` Number of steps translates to computational complexity. Here's a comparison for before and after for a query over 24h of data: ``` Read 4562944 rows, 148.05 MiB in 45.19249672 sec., 100966 rows/sec., 3.28 MiB/sec. Read 4183040 rows, 135.78 MiB in 0.196279627 sec., 21311636 rows/sec., 691.75 MiB/sec. ``` This is especially useful for queries that read data in order and terminate early to return "last X things" matching a query. See #11564 for more thoughts on this.
2020-07-07 18:10:44 +00:00
return false;
}
auto min_constraint = column_constraints[0];
if (min_constraint > Constraint::RANGE)
{
return false;
}
for (size_t i = 1; i < key_columns.size(); ++i)
{
if (column_constraints[i] < min_constraint)
{
return false;
}
if (column_constraints[i] == Constraint::RANGE && min_constraint == Constraint::RANGE)
{
return false;
}
min_constraint = column_constraints[i];
}
return true;
}
2020-03-10 14:56:55 +00:00
BoolMask KeyCondition::checkInHyperrectangle(
const std::vector<Range> & hyperrectangle,
const DataTypes & data_types) const
{
2012-12-05 12:44:55 +00:00
std::vector<BoolMask> rpn_stack;
2020-03-09 01:50:33 +00:00
for (const auto & element : rpn)
2012-12-05 12:44:55 +00:00
{
if (element.function == RPNElement::FUNCTION_UNKNOWN)
{
rpn_stack.emplace_back(true, true);
2012-12-05 12:44:55 +00:00
}
else if (element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
2012-12-05 12:44:55 +00:00
{
2020-03-10 14:56:55 +00:00
const Range * key_range = &hyperrectangle[element.key_column];
/// The case when the column is wrapped in a chain of possibly monotonic functions.
Range transformed_range;
if (!element.monotonic_functions_chain.empty())
{
std::optional<Range> new_range = applyMonotonicFunctionsChainToRange(
*key_range,
element.monotonic_functions_chain,
2020-09-21 10:13:01 +00:00
data_types[element.key_column],
single_point
);
if (!new_range)
{
rpn_stack.emplace_back(true, true);
continue;
}
transformed_range = *new_range;
key_range = &transformed_range;
}
bool intersects = element.range.intersectsRange(*key_range);
bool contains = element.range.containsRange(*key_range);
rpn_stack.emplace_back(intersects, !contains);
if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
rpn_stack.back() = !rpn_stack.back();
}
else if (
element.function == RPNElement::FUNCTION_IS_NULL
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL)
{
const Range * key_range = &hyperrectangle[element.key_column];
/// No need to apply monotonic functions as nulls are kept.
bool intersects = element.range.intersectsRange(*key_range);
bool contains = element.range.containsRange(*key_range);
2021-09-23 16:39:15 +00:00
rpn_stack.emplace_back(intersects, !contains);
2021-09-23 16:39:15 +00:00
if (element.function == RPNElement::FUNCTION_IS_NULL)
rpn_stack.back() = !rpn_stack.back();
}
else if (
element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
if (!element.set_index)
2018-04-18 20:27:27 +00:00
throw Exception("Set for IN is not created yet", ErrorCodes::LOGICAL_ERROR);
rpn_stack.emplace_back(element.set_index->checkInRange(hyperrectangle, data_types, single_point));
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
rpn_stack.back() = !rpn_stack.back();
2014-03-26 10:56:21 +00:00
}
2012-12-05 12:44:55 +00:00
else if (element.function == RPNElement::FUNCTION_NOT)
{
assert(!rpn_stack.empty());
2012-12-05 12:44:55 +00:00
rpn_stack.back() = !rpn_stack.back();
}
else if (element.function == RPNElement::FUNCTION_AND)
{
assert(!rpn_stack.empty());
auto arg1 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.back() = arg1 & arg2;
}
else if (element.function == RPNElement::FUNCTION_OR)
{
assert(!rpn_stack.empty());
auto arg1 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
2012-12-05 12:44:55 +00:00
rpn_stack.back() = arg1 | arg2;
}
else if (element.function == RPNElement::ALWAYS_FALSE)
{
rpn_stack.emplace_back(false, true);
}
else if (element.function == RPNElement::ALWAYS_TRUE)
{
rpn_stack.emplace_back(true, false);
}
2012-12-05 12:44:55 +00:00
else
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
2012-12-05 12:44:55 +00:00
}
2012-12-05 12:44:55 +00:00
if (rpn_stack.size() != 1)
throw Exception("Unexpected stack size in KeyCondition::checkInRange", ErrorCodes::LOGICAL_ERROR);
return rpn_stack[0];
}
bool KeyCondition::mayBeTrueInRange(
size_t used_key_size,
const FieldRef * left_keys,
const FieldRef * right_keys,
const DataTypes & data_types) const
2012-12-06 09:45:09 +00:00
{
return checkInRange(used_key_size, left_keys, right_keys, data_types, BoolMask::consider_only_can_be_true).can_be_true;
2012-12-06 09:45:09 +00:00
}
2021-04-15 17:30:04 +00:00
String KeyCondition::RPNElement::toString() const { return toString("column " + std::to_string(key_column), false); }
String KeyCondition::RPNElement::toString(std::string_view column_name, bool print_constants) const
{
2021-04-15 17:30:04 +00:00
auto print_wrapped_column = [this, &column_name, print_constants](WriteBuffer & buf)
{
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
2021-04-15 17:30:04 +00:00
{
2020-11-09 19:07:38 +00:00
buf << (*it)->getName() << "(";
2021-04-15 17:30:04 +00:00
if (print_constants)
{
if (const auto * func = typeid_cast<const FunctionWithOptionalConstArg *>(it->get()))
{
if (func->getKind() == FunctionWithOptionalConstArg::Kind::LEFT_CONST)
buf << applyVisitor(FieldVisitorToString(), (*func->getConstArg().column)[0]) << ", ";
}
}
}
2021-04-15 17:30:04 +00:00
buf << column_name;
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
2021-04-15 17:30:04 +00:00
{
if (print_constants)
{
if (const auto * func = typeid_cast<const FunctionWithOptionalConstArg *>(it->get()))
{
if (func->getKind() == FunctionWithOptionalConstArg::Kind::RIGHT_CONST)
buf << ", " << applyVisitor(FieldVisitorToString(), (*func->getConstArg().column)[0]);
}
}
2020-11-09 19:07:38 +00:00
buf << ")";
2021-04-15 17:30:04 +00:00
}
};
2020-11-09 19:07:38 +00:00
WriteBufferFromOwnString buf;
switch (function)
{
case FUNCTION_AND:
return "and";
case FUNCTION_OR:
return "or";
case FUNCTION_NOT:
return "not";
case FUNCTION_UNKNOWN:
return "unknown";
case FUNCTION_NOT_IN_SET:
case FUNCTION_IN_SET:
{
2020-11-09 19:07:38 +00:00
buf << "(";
print_wrapped_column(buf);
buf << (function == FUNCTION_IN_SET ? " in " : " notIn ");
2018-04-20 17:34:09 +00:00
if (!set_index)
2020-11-09 19:07:38 +00:00
buf << "unknown size set";
2018-04-20 17:34:09 +00:00
else
2020-11-09 19:07:38 +00:00
buf << set_index->size() << "-element set";
buf << ")";
return buf.str();
}
case FUNCTION_IN_RANGE:
case FUNCTION_NOT_IN_RANGE:
{
2020-11-09 19:07:38 +00:00
buf << "(";
print_wrapped_column(buf);
buf << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString();
buf << ")";
return buf.str();
}
case FUNCTION_IS_NULL:
case FUNCTION_IS_NOT_NULL:
{
buf << "(";
print_wrapped_column(buf);
buf << (function == FUNCTION_IS_NULL ? " isNull" : " isNotNull");
buf << ")";
return buf.str();
}
case ALWAYS_FALSE:
return "false";
case ALWAYS_TRUE:
return "true";
}
__builtin_unreachable();
}
bool KeyCondition::alwaysUnknownOrTrue() const
{
return unknownOrAlwaysTrue(false);
}
bool KeyCondition::anyUnknownOrAlwaysTrue() const
{
return unknownOrAlwaysTrue(true);
}
bool KeyCondition::unknownOrAlwaysTrue(bool unknown_any) const
{
std::vector<UInt8> rpn_stack;
for (const auto & element : rpn)
{
if (element.function == RPNElement::FUNCTION_UNKNOWN)
{
/// If unknown_any is true, return instantly,
/// to avoid processing it with FUNCTION_AND, and change the outcome.
if (unknown_any)
return true;
/// Otherwise, it may be AND'ed via FUNCTION_AND
rpn_stack.push_back(true);
}
else if (element.function == RPNElement::ALWAYS_TRUE)
{
rpn_stack.push_back(true);
}
else if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET
|| element.function == RPNElement::FUNCTION_IS_NULL
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL
|| element.function == RPNElement::ALWAYS_FALSE)
{
rpn_stack.push_back(false);
}
else if (element.function == RPNElement::FUNCTION_NOT)
{
}
else if (element.function == RPNElement::FUNCTION_AND)
{
assert(!rpn_stack.empty());
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 & arg2;
}
else if (element.function == RPNElement::FUNCTION_OR)
{
assert(!rpn_stack.empty());
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
rpn_stack.back() = arg1 | arg2;
}
else
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
}
if (rpn_stack.size() != 1)
throw Exception("Unexpected stack size in KeyCondition::unknownOrAlwaysTrue", ErrorCodes::LOGICAL_ERROR);
return rpn_stack[0];
}
bool KeyCondition::alwaysFalse() const
{
/// 0: always_false, 1: always_true, 2: non_const
std::vector<UInt8> rpn_stack;
for (const auto & element : rpn)
{
if (element.function == RPNElement::ALWAYS_TRUE)
{
rpn_stack.push_back(1);
}
else if (element.function == RPNElement::ALWAYS_FALSE)
{
rpn_stack.push_back(0);
}
else if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET
|| element.function == RPNElement::FUNCTION_IS_NULL
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL
|| element.function == RPNElement::FUNCTION_UNKNOWN)
{
rpn_stack.push_back(2);
}
else if (element.function == RPNElement::FUNCTION_NOT)
{
assert(!rpn_stack.empty());
auto & arg = rpn_stack.back();
if (arg == 0)
arg = 1;
else if (arg == 1)
arg = 0;
}
else if (element.function == RPNElement::FUNCTION_AND)
{
assert(!rpn_stack.empty());
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
if (arg1 == 0 || arg2 == 0)
rpn_stack.back() = 0;
else if (arg1 == 1 && arg2 == 1)
rpn_stack.back() = 1;
else
rpn_stack.back() = 2;
}
else if (element.function == RPNElement::FUNCTION_OR)
{
assert(!rpn_stack.empty());
auto arg1 = rpn_stack.back();
rpn_stack.pop_back();
auto arg2 = rpn_stack.back();
if (arg1 == 1 || arg2 == 1)
rpn_stack.back() = 1;
else if (arg1 == 0 && arg2 == 0)
rpn_stack.back() = 0;
else
rpn_stack.back() = 2;
}
else
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
}
if (rpn_stack.size() != 1)
throw Exception("Unexpected stack size in KeyCondition::alwaysFalse", ErrorCodes::LOGICAL_ERROR);
return rpn_stack[0] == 0;
}
size_t KeyCondition::getMaxKeyColumn() const
{
size_t res = 0;
for (const auto & element : rpn)
{
if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_IS_NULL
|| element.function == RPNElement::FUNCTION_IS_NOT_NULL
|| element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
if (element.key_column > res)
res = element.key_column;
}
}
return res;
}
bool KeyCondition::hasMonotonicFunctionsChain() const
{
for (const auto & element : rpn)
if (!element.monotonic_functions_chain.empty()
|| (element.set_index && element.set_index->hasMonotonicFunctionsChain()))
return true;
return false;
}
2012-12-05 12:44:55 +00:00
}