2018-04-20 00:20:36 +00:00
|
|
|
#include <Storages/MergeTree/KeyCondition.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/BoolMask.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-11-08 15:43:14 +00:00
|
|
|
#include <Interpreters/SyntaxAnalyzer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2019-10-23 13:59:03 +00:00
|
|
|
#include <Interpreters/misc.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
#include <Functions/IFunction.h>
|
2017-11-24 13:55:31 +00:00
|
|
|
#include <Common/FieldVisitors.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/convertFieldToType.h>
|
|
|
|
#include <Interpreters/Set.h>
|
2018-02-26 03:37:08 +00:00
|
|
|
#include <Parsers/queryToString.h>
|
2018-10-29 19:04:28 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
2019-01-22 12:33:56 +00:00
|
|
|
#include <Parsers/ASTSubquery.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2019-12-18 19:00:09 +00:00
|
|
|
#include <cassert>
|
|
|
|
|
2012-12-05 12:44:55 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-04-18 19:38:40 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int BAD_TYPE_OF_FIELD;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
String Range::toString() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::stringstream str;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!left_bounded)
|
|
|
|
str << "(-inf, ";
|
|
|
|
else
|
|
|
|
str << (left_included ? '[' : '(') << applyVisitor(FieldVisitorToString(), left) << ", ";
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!right_bounded)
|
|
|
|
str << "+inf)";
|
|
|
|
else
|
|
|
|
str << applyVisitor(FieldVisitorToString(), right) << (right_included ? ']' : ')');
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return str.str();
|
2017-01-14 09:00:19 +00:00
|
|
|
}
|
|
|
|
|
2015-06-24 21:24:06 +00:00
|
|
|
|
2017-03-12 19:18:07 +00:00
|
|
|
/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string.
|
2016-04-02 21:19:58 +00:00
|
|
|
static String extractFixedPrefixFromLikePattern(const String & like_pattern)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
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]];
|
2017-04-01 07:20:54 +00:00
|
|
|
case '_':
|
|
|
|
return fixed_prefix;
|
|
|
|
|
|
|
|
case '\\':
|
|
|
|
++pos;
|
|
|
|
if (pos == end)
|
|
|
|
break;
|
2017-12-02 03:25:52 +00:00
|
|
|
[[fallthrough]];
|
2017-04-01 07:20:54 +00:00
|
|
|
default:
|
|
|
|
fixed_prefix += *pos;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
++pos;
|
|
|
|
}
|
|
|
|
|
|
|
|
return fixed_prefix;
|
2016-04-02 21:19:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
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.
|
2016-04-02 21:19:58 +00:00
|
|
|
*/
|
|
|
|
static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & prefix)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Increment the last byte of the prefix by one. But if it is 255, then remove it and increase the previous one.
|
|
|
|
* Example (for convenience, suppose that the maximum value of byte is `z`)
|
|
|
|
* abcx -> abcy
|
|
|
|
* abcz -> abd
|
|
|
|
* zzz -> empty string
|
|
|
|
* z -> empty string
|
|
|
|
*/
|
2016-04-02 21:19:58 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String res = prefix;
|
2016-04-02 21:19:58 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
while (!res.empty() && static_cast<UInt8>(res.back()) == 255)
|
|
|
|
res.pop_back();
|
2016-04-02 21:19:58 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (res.empty())
|
|
|
|
return res;
|
2016-04-02 21:19:58 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
res.back() = static_cast<char>(1 + static_cast<UInt8>(res.back()));
|
|
|
|
return res;
|
2016-04-02 21:19:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-03-12 19:18:07 +00:00
|
|
|
/// A dictionary containing actions to the corresponding functions to turn them into `RPNElement`
|
2018-04-20 00:20:36 +00:00
|
|
|
const KeyCondition::AtomMap KeyCondition::atom_map
|
2017-01-14 09:00:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
"notEquals",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_NOT_IN_RANGE;
|
|
|
|
out.range = Range(value);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"equals",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_IN_RANGE;
|
|
|
|
out.range = Range(value);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"less",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_IN_RANGE;
|
|
|
|
out.range = Range::createRightBounded(value, false);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"greater",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_IN_RANGE;
|
|
|
|
out.range = Range::createLeftBounded(value, false);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"lessOrEquals",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_IN_RANGE;
|
|
|
|
out.range = Range::createRightBounded(value, true);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"greaterOrEquals",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_IN_RANGE;
|
|
|
|
out.range = Range::createLeftBounded(value, true);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"in",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field &)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_IN_SET;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"notIn",
|
2019-01-22 12:33:56 +00:00
|
|
|
[] (RPNElement & out, const Field &)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_NOT_IN_SET;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
|
|
|
"empty",
|
|
|
|
[] (RPNElement & out, const Field &)
|
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_IN_RANGE;
|
|
|
|
out.range = Range("");
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"notEmpty",
|
|
|
|
[] (RPNElement & out, const Field &)
|
|
|
|
{
|
|
|
|
out.function = RPNElement::FUNCTION_NOT_IN_RANGE;
|
|
|
|
out.range = Range("");
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
"like",
|
2019-07-07 15:52:20 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
},
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
|
|
|
"notLike",
|
|
|
|
[] (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_NOT_IN_RANGE;
|
|
|
|
out.range = !right_bound.empty()
|
|
|
|
? Range(prefix, true, right_bound, false)
|
|
|
|
: Range::createLeftBounded(prefix, true);
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
},
|
2019-07-07 15:48:34 +00:00
|
|
|
{
|
|
|
|
"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);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
2015-09-21 12:13:05 +00:00
|
|
|
};
|
|
|
|
|
2015-06-24 21:24:06 +00:00
|
|
|
|
2020-02-20 09:53:02 +00:00
|
|
|
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"},
|
|
|
|
{"like", "notLike"},
|
|
|
|
{"notLike", "like"},
|
|
|
|
{"empty", "notEmpty"},
|
|
|
|
{"notEmpty", "empty"},
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
bool isLogicalOperator(const String & func_name)
|
|
|
|
{
|
2020-03-07 19:56:38 +00:00
|
|
|
return (func_name == "and" || func_name == "or" || func_name == "not");
|
2020-02-20 09:53:02 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// The node can be one of:
|
2020-03-07 19:56:38 +00:00
|
|
|
/// - Logical operator (AND, OR, NOT)
|
2020-02-20 09:53:02 +00:00
|
|
|
/// - 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);
|
|
|
|
|
2020-03-07 19:56:38 +00:00
|
|
|
if (need_inversion)
|
2020-02-20 09:53:02 +00:00
|
|
|
{
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
|
|
|
const 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;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
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); }
|
2015-06-24 21:24:06 +00:00
|
|
|
|
|
|
|
|
2018-02-08 14:15:21 +00:00
|
|
|
FieldWithInfinity::FieldWithInfinity(const Field & field_)
|
|
|
|
: field(field_),
|
|
|
|
type(Type::NORMAL)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
FieldWithInfinity::FieldWithInfinity(Field && field_)
|
|
|
|
: field(std::move(field_)),
|
|
|
|
type(Type::NORMAL)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
FieldWithInfinity::FieldWithInfinity(const Type type_)
|
2020-03-09 01:50:33 +00:00
|
|
|
: type(type_)
|
2018-02-08 14:15:21 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
FieldWithInfinity FieldWithInfinity::getMinusInfinity()
|
|
|
|
{
|
|
|
|
return FieldWithInfinity(Type::MINUS_INFINITY);
|
|
|
|
}
|
2018-02-08 15:31:37 +00:00
|
|
|
|
2020-02-21 14:33:51 +00:00
|
|
|
FieldWithInfinity FieldWithInfinity::getPlusInfinity()
|
2018-02-08 14:15:21 +00:00
|
|
|
{
|
|
|
|
return FieldWithInfinity(Type::PLUS_INFINITY);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool FieldWithInfinity::operator<(const FieldWithInfinity & other) const
|
|
|
|
{
|
|
|
|
return type < other.type || (type == other.type && type == Type::NORMAL && field < other.field);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool FieldWithInfinity::operator==(const FieldWithInfinity & other) const
|
|
|
|
{
|
|
|
|
return type == other.type && (type != Type::NORMAL || field == other.field);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
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.
|
|
|
|
*/
|
2018-04-20 00:20:36 +00:00
|
|
|
Block KeyCondition::getBlockWithConstants(
|
2018-11-08 17:28:52 +00:00
|
|
|
const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, const Context & context)
|
2015-09-21 12:13:05 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Block result
|
|
|
|
{
|
2017-12-18 04:07:26 +00:00
|
|
|
{ DataTypeUInt8().createColumnConstWithDefaultValue(1), std::make_shared<DataTypeUInt8>(), "_dummy" }
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
2015-09-21 12:13:05 +00:00
|
|
|
|
2018-11-08 17:28:52 +00:00
|
|
|
const auto expr_for_constant_folding = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActions();
|
2015-09-21 12:13:05 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
expr_for_constant_folding->execute(result);
|
2015-09-21 12:13:05 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return result;
|
2015-09-21 12:13:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
KeyCondition::KeyCondition(
|
2017-07-15 03:48:36 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
|
|
|
const Context & context,
|
2018-06-30 21:35:01 +00:00
|
|
|
const Names & key_column_names,
|
2018-04-20 00:20:36 +00:00
|
|
|
const ExpressionActionsPtr & key_expr_)
|
2018-06-30 21:35:01 +00:00
|
|
|
: key_expr(key_expr_), prepared_sets(query_info.sets)
|
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)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-06-30 21:35:01 +00:00
|
|
|
std::string name = key_column_names[i];
|
2018-04-20 00:20:36 +00:00
|
|
|
if (!key_columns.count(name))
|
|
|
|
key_columns[name] = i;
|
2017-04-01 07:20:54 +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())`.
|
|
|
|
*/
|
|
|
|
Block block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
|
|
|
|
|
2020-02-20 09:53:02 +00:00
|
|
|
const ASTSelectQuery & select = query_info.query->as<ASTSelectQuery &>();
|
|
|
|
if (select.where() || select.prewhere())
|
2019-03-08 08:29:48 +00:00
|
|
|
{
|
2020-02-20 09:53:02 +00:00
|
|
|
ASTPtr filter_query;
|
|
|
|
if (select.where() && select.prewhere())
|
|
|
|
filter_query = makeASTFunction("and", select.where(), select.prewhere());
|
|
|
|
else
|
|
|
|
filter_query = select.where() ? select.where() : select.prewhere();
|
|
|
|
|
|
|
|
/** 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.
|
|
|
|
*/
|
|
|
|
traverseAST(cloneASTWithInversionPushDown(filter_query), context, block_with_constants);
|
2019-03-08 08:29:48 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
|
|
|
|
}
|
2012-12-05 12:44:55 +00:00
|
|
|
}
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
bool KeyCondition::addCondition(const String & column, const Range & range)
|
2012-12-12 14:25:55 +00:00
|
|
|
{
|
2018-04-20 00:20:36 +00:00
|
|
|
if (!key_columns.count(column))
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
2018-04-20 00:20:36 +00:00
|
|
|
rpn.emplace_back(RPNElement::FUNCTION_IN_RANGE, key_columns[column], range);
|
2017-04-01 07:20:54 +00:00
|
|
|
rpn.emplace_back(RPNElement::FUNCTION_AND);
|
|
|
|
return true;
|
2012-12-12 14:25:55 +00:00
|
|
|
}
|
|
|
|
|
2018-12-11 17:43:12 +00:00
|
|
|
/** Computes value of constant expression and its data type.
|
2016-11-24 12:26:47 +00:00
|
|
|
* Returns false, if expression isn't constant.
|
2015-10-28 21:00:44 +00:00
|
|
|
*/
|
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
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
String column_name = expr->getColumnName();
|
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * lit = expr->as<ASTLiteral>())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// 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;
|
|
|
|
return true;
|
|
|
|
}
|
2019-06-27 19:28:52 +00:00
|
|
|
else if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// 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;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
2012-12-05 12:44:55 +00:00
|
|
|
}
|
|
|
|
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
|
|
|
|
static void applyFunction(
|
2018-02-06 19:34:53 +00:00
|
|
|
const FunctionBasePtr & func,
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
const DataTypePtr & arg_type, const Field & arg_value,
|
|
|
|
DataTypePtr & res_type, Field & res_value)
|
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
res_type = func->getReturnType();
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
|
|
|
|
Block block
|
|
|
|
{
|
2018-02-06 19:34:53 +00:00
|
|
|
{ arg_type->createColumnConst(1, arg_value), arg_type, "x" },
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
{ nullptr, res_type, "y" }
|
|
|
|
};
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
func->execute(block, {0}, 1, 1);
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
|
|
|
|
block.safeGetByPosition(1).column->get(0, res_value);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-03-08 08:29:48 +00:00
|
|
|
void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants)
|
|
|
|
{
|
|
|
|
RPNElement element;
|
|
|
|
|
2020-02-20 09:53:02 +00:00
|
|
|
if (const auto * func = node->as<ASTFunction>())
|
2019-03-08 08:29:48 +00:00
|
|
|
{
|
2020-02-20 09:53:02 +00:00
|
|
|
if (tryParseLogicalOperatorFromAST(func, element))
|
2019-03-08 08:29:48 +00:00
|
|
|
{
|
2019-03-11 12:49:39 +00:00
|
|
|
auto & args = func->arguments->children;
|
2019-03-08 08:29:48 +00:00
|
|
|
for (size_t i = 0, size = args.size(); i < size; ++i)
|
|
|
|
{
|
|
|
|
traverseAST(args[i], context, block_with_constants);
|
|
|
|
|
|
|
|
/** 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;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-02-20 09:53:02 +00:00
|
|
|
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));
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
const ASTPtr & node,
|
2018-04-20 00:27:25 +00:00
|
|
|
size_t & out_key_column_num,
|
|
|
|
DataTypePtr & out_key_column_type,
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
Field & out_value,
|
|
|
|
DataTypePtr & out_type)
|
|
|
|
{
|
|
|
|
String expr_name = node->getColumnName();
|
2018-04-20 00:20:36 +00:00
|
|
|
const auto & sample_block = key_expr->getSampleBlock();
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
if (!sample_block.has(expr_name))
|
|
|
|
return false;
|
|
|
|
|
|
|
|
bool found_transformation = false;
|
2018-04-20 00:20:36 +00:00
|
|
|
for (const ExpressionAction & a : key_expr->getActions())
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
{
|
2018-04-20 00:27:25 +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()`,
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
* 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.
|
|
|
|
*/
|
|
|
|
const auto & action = a.argument_names;
|
|
|
|
if (a.type == ExpressionAction::Type::APPLY_FUNCTION && action.size() == 1 && a.argument_names[0] == expr_name)
|
|
|
|
{
|
2018-09-21 14:15:21 +00:00
|
|
|
if (!a.function_base->hasInformationAboutMonotonicity())
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
return false;
|
|
|
|
|
|
|
|
// Range is irrelevant in this case
|
2018-09-21 14:15:21 +00:00
|
|
|
IFunction::Monotonicity monotonicity = a.function_base->getMonotonicityForRange(*out_type, Field(), Field());
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
if (!monotonicity.is_always_monotonic)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
// Apply the next transformation step
|
|
|
|
DataTypePtr new_type;
|
2018-09-21 14:15:21 +00:00
|
|
|
applyFunction(a.function_base, out_type, out_value, new_type, out_value);
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
if (!new_type)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
out_type.swap(new_type);
|
|
|
|
expr_name = a.result_name;
|
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
// Transformation results in a key expression, accept
|
2018-04-20 00:20:36 +00:00
|
|
|
auto it = key_columns.find(expr_name);
|
|
|
|
if (key_columns.end() != it)
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
{
|
2018-04-20 00:27:25 +00:00
|
|
|
out_key_column_num = it->second;
|
|
|
|
out_key_column_type = sample_block.getByName(it->first).type;
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
found_transformation = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return found_transformation;
|
|
|
|
}
|
|
|
|
|
2018-07-02 18:57:14 +00:00
|
|
|
bool KeyCondition::tryPrepareSetIndex(
|
2019-01-22 12:33:56 +00:00
|
|
|
const ASTs & args,
|
2018-02-02 13:19:40 +00:00
|
|
|
const Context & context,
|
|
|
|
RPNElement & out,
|
2018-04-20 00:27:25 +00:00
|
|
|
size_t & out_key_column_num)
|
2018-02-08 14:15:21 +00:00
|
|
|
{
|
2019-01-22 12:33:56 +00:00
|
|
|
const ASTPtr & left_arg = args[0];
|
2018-07-02 18:57:14 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
out_key_column_num = 0;
|
|
|
|
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> indexes_mapping;
|
2019-01-22 12:33:56 +00:00
|
|
|
DataTypes data_types;
|
2018-04-18 19:38:40 +00:00
|
|
|
|
2019-01-22 12:33:56 +00:00
|
|
|
auto get_key_tuple_position_mapping = [&](const ASTPtr & node, size_t tuple_index)
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2019-01-22 12:33:56 +00:00
|
|
|
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))
|
2018-04-18 19:38:40 +00:00
|
|
|
{
|
2019-01-22 12:33:56 +00:00
|
|
|
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;
|
2018-04-18 19:38:40 +00:00
|
|
|
}
|
2019-01-22 12:33:56 +00:00
|
|
|
};
|
2018-04-18 19:38:40 +00:00
|
|
|
|
2019-08-19 16:51:50 +00:00
|
|
|
size_t left_args_count = 1;
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * left_arg_tuple = left_arg->as<ASTFunction>();
|
2019-01-22 12:33:56 +00:00
|
|
|
if (left_arg_tuple && left_arg_tuple->name == "tuple")
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2019-01-22 12:33:56 +00:00
|
|
|
const auto & tuple_elements = left_arg_tuple->arguments->children;
|
2019-08-19 16:51:50 +00:00
|
|
|
left_args_count = tuple_elements.size();
|
|
|
|
for (size_t i = 0; i < left_args_count; ++i)
|
2019-01-22 12:33:56 +00:00
|
|
|
get_key_tuple_position_mapping(tuple_elements[i], i);
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
2019-01-22 12:33:56 +00:00
|
|
|
else
|
|
|
|
get_key_tuple_position_mapping(left_arg, 0);
|
2018-02-02 13:19:40 +00:00
|
|
|
|
|
|
|
if (indexes_mapping.empty())
|
|
|
|
return false;
|
|
|
|
|
2019-01-22 12:33:56 +00:00
|
|
|
const ASTPtr & right_arg = args[1];
|
|
|
|
|
|
|
|
PreparedSetKey set_key;
|
2019-03-11 13:22:51 +00:00
|
|
|
if (right_arg->as<ASTSubquery>() || right_arg->as<ASTIdentifier>())
|
2019-01-22 12:33:56 +00:00
|
|
|
set_key = PreparedSetKey::forSubquery(*right_arg);
|
|
|
|
else
|
|
|
|
set_key = PreparedSetKey::forLiteral(*right_arg, data_types);
|
|
|
|
|
|
|
|
auto set_it = prepared_sets.find(set_key);
|
|
|
|
if (set_it == prepared_sets.end())
|
|
|
|
return false;
|
|
|
|
|
|
|
|
const SetPtr & prepared_set = set_it->second;
|
|
|
|
|
|
|
|
/// 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, removeLowCardinality(data_types[i]));
|
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
out.set_index = std::make_shared<MergeTreeSetIndex>(prepared_set->getSetElements(), std::move(indexes_mapping));
|
2018-02-02 13:19:40 +00:00
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
|
2018-04-18 19:38:40 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
|
2017-04-01 07:20:54 +00:00
|
|
|
const ASTPtr & node,
|
|
|
|
const Context & context,
|
2018-04-20 00:27:25 +00:00
|
|
|
size_t & out_key_column_num,
|
|
|
|
DataTypePtr & out_key_res_column_type,
|
2018-06-04 17:01:00 +00:00
|
|
|
MonotonicFunctionsChain & out_functions_chain)
|
2012-12-05 12:44:55 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::vector<const ASTFunction *> chain_not_tested_for_monotonicity;
|
2018-04-20 00:27:25 +00:00
|
|
|
DataTypePtr key_column_type;
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity))
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
|
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
auto func_builder = FunctionFactory::instance().tryGet((*it)->name, context);
|
2018-04-20 00:27:25 +00:00
|
|
|
ColumnsWithTypeAndName arguments{{ nullptr, key_column_type, "" }};
|
2018-02-02 08:33:36 +00:00
|
|
|
auto func = func_builder->build(arguments);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!func || !func->hasInformationAboutMonotonicity())
|
|
|
|
return false;
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
key_column_type = func->getReturnType();
|
2017-04-01 07:20:54 +00:00
|
|
|
out_functions_chain.push_back(func);
|
|
|
|
}
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
out_key_res_column_type = key_column_type;
|
2016-11-23 19:58:33 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2015-11-29 08:06:29 +00:00
|
|
|
}
|
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
2017-04-01 07:20:54 +00:00
|
|
|
const ASTPtr & node,
|
2018-04-20 00:27:25 +00:00
|
|
|
size_t & out_key_column_num,
|
|
|
|
DataTypePtr & out_key_column_type,
|
2017-04-01 07:20:54 +00:00
|
|
|
std::vector<const ASTFunction *> & out_functions_chain)
|
2015-11-29 08:06:29 +00:00
|
|
|
{
|
2018-04-20 00:27:25 +00:00
|
|
|
/** By itself, the key column can be a functional expression. for example, `intHash32(UserID)`.
|
2017-04-01 07:20:54 +00:00
|
|
|
* Therefore, use the full name of the expression for search.
|
|
|
|
*/
|
2018-04-20 00:20:36 +00:00
|
|
|
const auto & sample_block = key_expr->getSampleBlock();
|
2017-04-01 07:20:54 +00:00
|
|
|
String name = node->getColumnName();
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
auto it = key_columns.find(name);
|
|
|
|
if (key_columns.end() != it)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-20 00:27:25 +00:00
|
|
|
out_key_column_num = it->second;
|
|
|
|
out_key_column_type = sample_block.getByName(it->first).type;
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * func = node->as<ASTFunction>())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
const auto & args = func->arguments->children;
|
|
|
|
if (args.size() != 1)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
out_functions_chain.push_back(func);
|
|
|
|
|
2020-03-09 01:50:33 +00:00
|
|
|
return isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
2015-11-29 08:06:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-11-24 12:26:47 +00:00
|
|
|
static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const ASTPtr & node)
|
2016-11-21 12:58:42 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (desired_type->equals(*src_type))
|
|
|
|
return;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
/// NOTE: We don't need accurate info about src_type at this moment
|
|
|
|
src_value = convertFieldToType(src_value, *desired_type);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2018-04-20 00:27:25 +00:00
|
|
|
throw Exception("Key expression contains comparison between inconvertible types: " +
|
2017-04-01 07:20:54 +00:00
|
|
|
desired_type->getName() + " and " + src_type->getName() +
|
2018-02-26 03:37:08 +00:00
|
|
|
" inside " + queryToString(node),
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::BAD_TYPE_OF_FIELD);
|
|
|
|
}
|
2016-11-21 12:58:42 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-02-20 09:53:02 +00:00
|
|
|
bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
|
2015-11-29 08:06:29 +00:00
|
|
|
{
|
2018-04-20 00:27:25 +00:00
|
|
|
/** Functions < > = != <= >= in `notIn`, 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-04-01 07:20:54 +00:00
|
|
|
* or constant expression - number.
|
|
|
|
*/
|
|
|
|
Field const_value;
|
|
|
|
DataTypePtr const_type;
|
2019-03-11 13:22:51 +00:00
|
|
|
if (const auto * func = node->as<ASTFunction>())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-03-15 16:14:13 +00:00
|
|
|
const ASTs & args = func->arguments->children;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
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)
|
2018-06-04 17:01:00 +00:00
|
|
|
MonotonicFunctionsChain chain;
|
2019-07-10 14:53:57 +00:00
|
|
|
std::string func_name = func->name;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-07-25 15:56:20 +00:00
|
|
|
if (atom_map.find(func_name) == std::end(atom_map))
|
|
|
|
return false;
|
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
if (args.size() == 1)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-07-10 14:53:57 +00:00
|
|
|
if (!(isKeyPossiblyWrappedByMonotonicFunctions(args[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);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2019-07-10 14:53:57 +00:00
|
|
|
else if (args.size() == 2)
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
{
|
2019-07-10 14:53:57 +00:00
|
|
|
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
|
|
|
|
bool is_set_const = false;
|
|
|
|
bool is_constant_transformed = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
if (functionIsInOrGlobalInOperator(func_name)
|
|
|
|
&& tryPrepareSetIndex(args, context, out, key_column_num))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
is_set_const = true;
|
|
|
|
}
|
|
|
|
else if (getConstant(args[1], block_with_constants, const_value, const_type)
|
|
|
|
&& isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
}
|
|
|
|
else if (getConstant(args[1], block_with_constants, const_value, const_type)
|
|
|
|
&& canConstantBeWrappedByMonotonicFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
is_constant_transformed = true;
|
|
|
|
}
|
|
|
|
else if (getConstant(args[0], block_with_constants, const_value, const_type)
|
|
|
|
&& isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
|
|
|
|
{
|
|
|
|
key_arg_pos = 1;
|
|
|
|
}
|
|
|
|
else if (getConstant(args[0], block_with_constants, const_value, const_type)
|
|
|
|
&& canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
|
|
|
|
{
|
|
|
|
key_arg_pos = 1;
|
|
|
|
is_constant_transformed = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
2019-01-18 16:30:35 +00:00
|
|
|
|
2019-07-10 14:53:57 +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);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
/// 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";
|
|
|
|
}
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
/// Replace <const> <sign> <data> on to <data> <-sign> <const>
|
|
|
|
if (key_arg_pos == 1)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-07-10 14:53:57 +00:00
|
|
|
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";
|
2020-02-20 09:53:02 +00:00
|
|
|
else if (func_name == "in" || func_name == "notIn" ||
|
|
|
|
func_name == "like" || func_name == "notLike" ||
|
|
|
|
func_name == "startsWith")
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
|
|
|
/// "const IN data_column" doesn't make sense (unlike "data_column IN const")
|
|
|
|
return false;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
bool cast_not_needed =
|
|
|
|
is_set_const /// Set args are already casted inside Set::createFromAST
|
|
|
|
|| (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast.
|
|
|
|
|
|
|
|
if (!cast_not_needed)
|
|
|
|
castValueToType(key_expr_type, const_value, const_type, node);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
const auto atom_it = atom_map.find(func_name);
|
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
out.key_column = key_column_num;
|
|
|
|
out.monotonic_functions_chain = std::move(chain);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-01-22 12:33:56 +00:00
|
|
|
return atom_it->second(out, const_value);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else if (getConstant(node, 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
|
|
|
|
|| const_value.getType() == Field::Types::Int64
|
|
|
|
|| const_value.getType() == Field::Types::Float64)
|
|
|
|
{
|
|
|
|
/// Zero in all types is represented in memory the same way as in UInt64.
|
|
|
|
out.function = const_value.get<UInt64>()
|
|
|
|
? RPNElement::ALWAYS_TRUE
|
|
|
|
: RPNElement::ALWAYS_FALSE;
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false;
|
2012-12-05 12:44:55 +00:00
|
|
|
}
|
|
|
|
|
2020-02-20 09:53:02 +00:00
|
|
|
bool KeyCondition::tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out)
|
2019-03-08 08:29:48 +00:00
|
|
|
{
|
|
|
|
/// Functions AND, OR, NOT.
|
2019-03-08 09:45:34 +00:00
|
|
|
const ASTs & args = func->arguments->children;
|
2019-03-08 08:29:48 +00:00
|
|
|
|
|
|
|
if (func->name == "not")
|
|
|
|
{
|
|
|
|
if (args.size() != 1)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
out.function = RPNElement::FUNCTION_NOT;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-03-07 19:56:38 +00:00
|
|
|
if (func->name == "and")
|
2019-03-08 08:29:48 +00:00
|
|
|
out.function = RPNElement::FUNCTION_AND;
|
|
|
|
else if (func->name == "or")
|
|
|
|
out.function = RPNElement::FUNCTION_OR;
|
|
|
|
else
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
String KeyCondition::toString() const
|
2012-12-05 12:44:55 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
String res;
|
|
|
|
for (size_t i = 0; i < rpn.size(); ++i)
|
|
|
|
{
|
|
|
|
if (i)
|
|
|
|
res += ", ";
|
|
|
|
res += rpn[i].toString();
|
|
|
|
}
|
|
|
|
return res;
|
2012-12-05 12:44:55 +00:00
|
|
|
}
|
|
|
|
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
/** 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.
|
2016-03-27 11:37:25 +00:00
|
|
|
*
|
2018-04-20 00:27:25 +00:00
|
|
|
* The key is the tuple.
|
|
|
|
* The data is sorted by key in the sense of lexicographic order over tuples.
|
2016-03-27 11:37:25 +00:00
|
|
|
*
|
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 ],
|
2018-04-20 00:27:25 +00:00
|
|
|
* 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.
|
2016-03-27 11:37:25 +00:00
|
|
|
*
|
2017-03-12 19:18:07 +00:00
|
|
|
* Or, the last mark specifies the range open on the right: [ a b c .. + inf )
|
2016-03-27 11:37:25 +00:00
|
|
|
*
|
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.
|
2016-03-27 11:37:25 +00:00
|
|
|
*
|
2020-03-10 14:56:55 +00:00
|
|
|
* Hyperrectangles (you can also find the term "rail")
|
2017-03-12 19:18:07 +00:00
|
|
|
* will be the subrange of an n-dimensional space that is a direct product of one-dimensional ranges.
|
|
|
|
* In this case, the one-dimensional range can be: a period, a segment, an interval, a half-interval, unlimited on the left, unlimited on the right ...
|
2016-03-27 11:37:25 +00:00
|
|
|
*
|
2020-03-10 14:56:55 +00:00
|
|
|
* The range of tuples can always be represented as a combination of hyperrectangles.
|
|
|
|
* For example, the range [ x1 y1 .. x2 y2 ] given x1 != x2 is equal to the union of the following three hyperrectangles:
|
2016-03-27 11:37:25 +00:00
|
|
|
* [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:
|
2016-03-27 11:37:25 +00:00
|
|
|
* [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.
|
2016-03-27 11:37:25 +00:00
|
|
|
*
|
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.
|
2016-03-27 11:37:25 +00:00
|
|
|
*/
|
|
|
|
|
|
|
|
template <typename F>
|
2020-03-10 14:56:55 +00:00
|
|
|
static BoolMask forAnyHyperrectangle(
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t key_size,
|
|
|
|
const Field * key_left,
|
|
|
|
const Field * key_right,
|
|
|
|
bool left_bounded,
|
|
|
|
bool right_bounded,
|
2020-03-10 14:56:55 +00:00
|
|
|
std::vector<Range> & hyperrectangle,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t prefix_size,
|
2020-01-29 21:40:22 +00:00
|
|
|
BoolMask initial_mask,
|
2017-04-01 07:20:54 +00:00
|
|
|
F && callback)
|
2012-12-05 12:44:55 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!left_bounded && !right_bounded)
|
2020-03-10 14:56:55 +00:00
|
|
|
return callback(hyperrectangle);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (left_bounded && right_bounded)
|
|
|
|
{
|
|
|
|
/// Let's go through the matching elements of the key.
|
|
|
|
while (prefix_size < key_size)
|
|
|
|
{
|
|
|
|
if (key_left[prefix_size] == key_right[prefix_size])
|
|
|
|
{
|
|
|
|
/// Point ranges.
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range(key_left[prefix_size]);
|
2017-04-01 07:20:54 +00:00
|
|
|
++prefix_size;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (prefix_size == key_size)
|
2020-03-10 14:56:55 +00:00
|
|
|
return callback(hyperrectangle);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (prefix_size + 1 == key_size)
|
|
|
|
{
|
|
|
|
if (left_bounded && right_bounded)
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range(key_left[prefix_size], true, key_right[prefix_size], true);
|
2017-04-01 07:20:54 +00:00
|
|
|
else if (left_bounded)
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range::createLeftBounded(key_left[prefix_size], true);
|
2017-04-01 07:20:54 +00:00
|
|
|
else if (right_bounded)
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range::createRightBounded(key_right[prefix_size], true);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-10 14:56:55 +00:00
|
|
|
return callback(hyperrectangle);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// (x1 .. x2) x (-inf .. +inf)
|
|
|
|
|
|
|
|
if (left_bounded && right_bounded)
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range(key_left[prefix_size], false, key_right[prefix_size], false);
|
2017-04-01 07:20:54 +00:00
|
|
|
else if (left_bounded)
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range::createLeftBounded(key_left[prefix_size], false);
|
2017-04-01 07:20:54 +00:00
|
|
|
else if (right_bounded)
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range::createRightBounded(key_right[prefix_size], false);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t i = prefix_size + 1; i < key_size; ++i)
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[i] = Range();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-29 21:40:22 +00:00
|
|
|
|
|
|
|
BoolMask result = initial_mask;
|
2020-03-10 14:56:55 +00:00
|
|
|
result = result | callback(hyperrectangle);
|
2020-01-29 21:40:22 +00:00
|
|
|
|
|
|
|
/// 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())
|
2020-01-29 21:40:22 +00:00
|
|
|
return result;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// [x1] x [y1 .. +inf)
|
|
|
|
|
|
|
|
if (left_bounded)
|
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range(key_left[prefix_size]);
|
|
|
|
result = result | forAnyHyperrectangle(key_size, key_left, key_right, true, false, hyperrectangle, prefix_size + 1, initial_mask, callback);
|
2020-01-29 23:36:39 +00:00
|
|
|
if (result.isComplete())
|
2020-01-29 21:40:22 +00:00
|
|
|
return result;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// [x2] x (-inf .. y2]
|
|
|
|
|
|
|
|
if (right_bounded)
|
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[prefix_size] = Range(key_right[prefix_size]);
|
|
|
|
result = result | forAnyHyperrectangle(key_size, key_left, key_right, false, true, hyperrectangle, prefix_size + 1, initial_mask, callback);
|
2020-01-29 23:36:39 +00:00
|
|
|
if (result.isComplete())
|
2020-01-29 21:40:22 +00:00
|
|
|
return result;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-01-29 21:40:22 +00:00
|
|
|
return result;
|
2016-03-27 11:37:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-01-29 21:40:22 +00:00
|
|
|
BoolMask KeyCondition::checkInRange(
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t used_key_size,
|
2018-04-20 00:27:25 +00:00
|
|
|
const Field * left_key,
|
|
|
|
const Field * right_key,
|
2017-04-01 07:20:54 +00:00
|
|
|
const DataTypes & data_types,
|
2020-01-29 21:40:22 +00:00
|
|
|
bool right_bounded,
|
|
|
|
BoolMask initial_mask) const
|
2016-03-27 11:37:25 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::vector<Range> key_ranges(used_key_size, Range());
|
|
|
|
|
2017-05-25 01:12:41 +00:00
|
|
|
/* std::cerr << "Checking for: [";
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i != used_key_size; ++i)
|
2018-04-20 00:27:25 +00:00
|
|
|
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), left_key[i]);
|
2017-04-01 07:20:54 +00:00
|
|
|
std::cerr << " ... ";
|
|
|
|
|
|
|
|
if (right_bounded)
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i != used_key_size; ++i)
|
2018-04-20 00:27:25 +00:00
|
|
|
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), right_key[i]);
|
2017-04-01 07:20:54 +00:00
|
|
|
std::cerr << "]\n";
|
|
|
|
}
|
|
|
|
else
|
|
|
|
std::cerr << "+inf)\n";*/
|
|
|
|
|
2020-03-10 14:56:55 +00:00
|
|
|
return forAnyHyperrectangle(used_key_size, left_key, right_key, true, right_bounded, key_ranges, 0, initial_mask,
|
|
|
|
[&] (const std::vector<Range> & key_ranges_hyperrectangle)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
auto res = checkInHyperrectangle(key_ranges_hyperrectangle, data_types);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-10 14:56:55 +00:00
|
|
|
/* std::cerr << "Hyperrectangle: ";
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0, size = key_ranges.size(); i != size; ++i)
|
|
|
|
std::cerr << (i != 0 ? " x " : "") << key_ranges[i].toString();
|
|
|
|
std::cerr << ": " << res << "\n";*/
|
|
|
|
|
|
|
|
return res;
|
|
|
|
});
|
2016-03-27 11:37:25 +00:00
|
|
|
}
|
|
|
|
|
2020-01-29 21:40:22 +00:00
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange(
|
2018-02-02 13:19:40 +00:00
|
|
|
Range key_range,
|
2018-06-04 17:01:00 +00:00
|
|
|
MonotonicFunctionsChain & functions,
|
2020-01-29 21:40:22 +00:00
|
|
|
DataTypePtr current_type)
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
|
|
|
for (auto & func : functions)
|
|
|
|
{
|
|
|
|
/// We check the monotonicity of each function on a specific range.
|
|
|
|
IFunction::Monotonicity monotonicity = func->getMonotonicityForRange(
|
|
|
|
*current_type.get(), key_range.left, key_range.right);
|
|
|
|
|
|
|
|
if (!monotonicity.is_monotonic)
|
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Apply the function.
|
|
|
|
DataTypePtr new_type;
|
|
|
|
if (!key_range.left.isNull())
|
|
|
|
applyFunction(func, current_type, key_range.left, new_type, key_range.left);
|
|
|
|
if (!key_range.right.isNull())
|
|
|
|
applyFunction(func, current_type, key_range.right, new_type, key_range.right);
|
|
|
|
|
|
|
|
if (!new_type)
|
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
|
|
|
current_type.swap(new_type);
|
|
|
|
|
|
|
|
if (!monotonicity.is_positive)
|
|
|
|
key_range.swapLeftAndRight();
|
|
|
|
}
|
|
|
|
return key_range;
|
|
|
|
}
|
2016-03-27 11:37:25 +00:00
|
|
|
|
2020-03-10 14:56:55 +00:00
|
|
|
BoolMask KeyCondition::checkInHyperrectangle(
|
|
|
|
const std::vector<Range> & hyperrectangle,
|
2020-01-29 21:40:22 +00:00
|
|
|
const DataTypes & data_types) const
|
2016-03-27 11:37:25 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::vector<BoolMask> rpn_stack;
|
2020-03-09 01:50:33 +00:00
|
|
|
for (const auto & element : rpn)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (element.function == RPNElement::FUNCTION_UNKNOWN)
|
|
|
|
{
|
|
|
|
rpn_stack.emplace_back(true, true);
|
|
|
|
}
|
|
|
|
else if (element.function == RPNElement::FUNCTION_IN_RANGE
|
2018-02-02 13:19:40 +00:00
|
|
|
|| element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
const Range * key_range = &hyperrectangle[element.key_column];
|
2017-04-01 07:20:54 +00:00
|
|
|
|
PKCondition: infer index use with pk subexpression
By default only constraints explicitly matching
primary key expression (or expression wrapped in
a monotonic function) are eligible for part and
range selection. So for example, if index is:
(toStartOfHour(dt), UserID)
Then a query such as this resorts to full scan:
SELECT count() FROM t WHERE dt = now()
Intuitively, only parts with toStartOfHour(now())
could be selected, but it is less trivial to prove.
The primary key currently can be wrapped in a chain
of monotonic functions, so following would work:
toStartOfHour(dt) = toStartOfHour(now()) AND dt = now()
It must be however explicitly stated, if we wanted
to infer that we’d have to know the inverse function,
and prove that the inverse function is monotonic
on given interval. This is not practical as
there is no inverse function that for example undos
rounding, it isn’t strictly monotonic.
There are however functions that don’t transform
output range and preserve monotonicity on the
complete input range, such as rounding or casts
to a same or wider numeric type. This eliminates
the need to find inverse function, as no check for monotonicity over arbitrary interval is needed,
and thus makes this optimisation possible.
2017-07-06 05:39:05 +00:00
|
|
|
/// The case when the column is wrapped in a chain of possibly monotonic functions.
|
2018-02-02 13:19:40 +00:00
|
|
|
Range transformed_range;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!element.monotonic_functions_chain.empty())
|
|
|
|
{
|
2018-02-02 13:19:40 +00:00
|
|
|
std::optional<Range> new_range = applyMonotonicFunctionsChainToRange(
|
|
|
|
*key_range,
|
|
|
|
element.monotonic_functions_chain,
|
|
|
|
data_types[element.key_column]
|
|
|
|
);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
if (!new_range)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
rpn_stack.emplace_back(true, true);
|
|
|
|
continue;
|
|
|
|
}
|
2018-02-02 13:19:40 +00:00
|
|
|
transformed_range = *new_range;
|
|
|
|
key_range = &transformed_range;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
bool intersects = element.range.intersectsRange(*key_range);
|
|
|
|
bool contains = element.range.containsRange(*key_range);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
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_IN_SET
|
|
|
|
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
|
|
|
|
{
|
2019-01-22 12:33:56 +00:00
|
|
|
if (!element.set_index)
|
2018-04-18 20:27:27 +00:00
|
|
|
throw Exception("Set for IN is not created yet", ErrorCodes::LOGICAL_ERROR);
|
2019-01-22 12:33:56 +00:00
|
|
|
|
2020-03-10 14:56:55 +00:00
|
|
|
rpn_stack.emplace_back(element.set_index->checkInRange(hyperrectangle, data_types));
|
2019-01-22 12:33:56 +00:00
|
|
|
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
|
|
|
|
rpn_stack.back() = !rpn_stack.back();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else if (element.function == RPNElement::FUNCTION_NOT)
|
|
|
|
{
|
2019-12-18 19:00:09 +00:00
|
|
|
assert(!rpn_stack.empty());
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
rpn_stack.back() = !rpn_stack.back();
|
|
|
|
}
|
|
|
|
else if (element.function == RPNElement::FUNCTION_AND)
|
|
|
|
{
|
2019-12-18 19:00:09 +00:00
|
|
|
assert(!rpn_stack.empty());
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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)
|
|
|
|
{
|
2019-12-18 19:00:09 +00:00
|
|
|
assert(!rpn_stack.empty());
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
auto arg1 = rpn_stack.back();
|
|
|
|
rpn_stack.pop_back();
|
|
|
|
auto arg2 = rpn_stack.back();
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
else
|
2018-04-20 00:20:36 +00:00
|
|
|
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (rpn_stack.size() != 1)
|
2020-01-29 21:40:22 +00:00
|
|
|
throw Exception("Unexpected stack size in KeyCondition::checkInRange", ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-29 21:40:22 +00:00
|
|
|
return rpn_stack[0];
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
BoolMask KeyCondition::checkInRange(
|
|
|
|
size_t used_key_size,
|
|
|
|
const Field * left_key,
|
|
|
|
const Field * right_key,
|
|
|
|
const DataTypes & data_types,
|
|
|
|
BoolMask initial_mask) const
|
|
|
|
{
|
|
|
|
return checkInRange(used_key_size, left_key, right_key, data_types, true, initial_mask);
|
2012-12-05 12:44:55 +00:00
|
|
|
}
|
|
|
|
|
2016-03-27 11:37:25 +00:00
|
|
|
|
2019-09-24 01:29:26 +00:00
|
|
|
bool KeyCondition::mayBeTrueInRange(
|
2020-01-29 21:40:22 +00:00
|
|
|
size_t used_key_size,
|
|
|
|
const Field * left_key,
|
|
|
|
const Field * right_key,
|
|
|
|
const DataTypes & data_types) const
|
2012-12-06 09:45:09 +00:00
|
|
|
{
|
2020-01-29 21:40:22 +00:00
|
|
|
return checkInRange(used_key_size, left_key, right_key, data_types, true, BoolMask::consider_only_can_be_true).can_be_true;
|
2012-12-06 09:45:09 +00:00
|
|
|
}
|
|
|
|
|
2020-01-29 21:40:22 +00:00
|
|
|
|
|
|
|
BoolMask KeyCondition::checkAfter(
|
|
|
|
size_t used_key_size,
|
|
|
|
const Field * left_key,
|
|
|
|
const DataTypes & data_types,
|
|
|
|
BoolMask initial_mask) const
|
|
|
|
{
|
|
|
|
return checkInRange(used_key_size, left_key, nullptr, data_types, false, initial_mask);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-09-24 01:29:26 +00:00
|
|
|
bool KeyCondition::mayBeTrueAfter(
|
2020-01-29 21:40:22 +00:00
|
|
|
size_t used_key_size,
|
|
|
|
const Field * left_key,
|
|
|
|
const DataTypes & data_types) const
|
2012-12-06 09:45:09 +00:00
|
|
|
{
|
2020-01-29 21:40:22 +00:00
|
|
|
return checkInRange(used_key_size, left_key, nullptr, data_types, false, BoolMask::consider_only_can_be_true).can_be_true;
|
2012-12-06 09:45:09 +00:00
|
|
|
}
|
|
|
|
|
2016-03-27 11:37:25 +00:00
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
String KeyCondition::RPNElement::toString() const
|
2014-04-01 10:25:56 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
auto print_wrapped_column = [this](std::ostringstream & ss)
|
|
|
|
{
|
|
|
|
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
|
|
|
|
ss << (*it)->getName() << "(";
|
|
|
|
|
|
|
|
ss << "column " << key_column;
|
|
|
|
|
|
|
|
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
|
|
|
|
ss << ")";
|
|
|
|
};
|
|
|
|
|
|
|
|
std::ostringstream ss;
|
|
|
|
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:
|
|
|
|
{
|
|
|
|
ss << "(";
|
|
|
|
print_wrapped_column(ss);
|
2018-04-20 01:14:04 +00:00
|
|
|
ss << (function == FUNCTION_IN_SET ? " in " : " notIn ");
|
2018-04-20 17:34:09 +00:00
|
|
|
if (!set_index)
|
|
|
|
ss << "unknown size set";
|
|
|
|
else
|
|
|
|
ss << set_index->size() << "-element set";
|
2017-04-01 07:20:54 +00:00
|
|
|
ss << ")";
|
|
|
|
return ss.str();
|
|
|
|
}
|
|
|
|
case FUNCTION_IN_RANGE:
|
|
|
|
case FUNCTION_NOT_IN_RANGE:
|
|
|
|
{
|
|
|
|
ss << "(";
|
|
|
|
print_wrapped_column(ss);
|
|
|
|
ss << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString();
|
|
|
|
ss << ")";
|
|
|
|
return ss.str();
|
|
|
|
}
|
|
|
|
case ALWAYS_FALSE:
|
|
|
|
return "false";
|
|
|
|
case ALWAYS_TRUE:
|
|
|
|
return "true";
|
|
|
|
}
|
2019-01-05 03:33:22 +00:00
|
|
|
|
|
|
|
__builtin_unreachable();
|
2014-04-01 10:25:56 +00:00
|
|
|
}
|
2015-03-27 03:06:06 +00:00
|
|
|
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
bool KeyCondition::alwaysUnknownOrTrue() const
|
2015-03-27 03:06:06 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::vector<UInt8> rpn_stack;
|
|
|
|
|
|
|
|
for (const auto & element : rpn)
|
|
|
|
{
|
|
|
|
if (element.function == RPNElement::FUNCTION_UNKNOWN
|
|
|
|
|| 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::ALWAYS_FALSE)
|
|
|
|
{
|
|
|
|
rpn_stack.push_back(false);
|
|
|
|
}
|
|
|
|
else if (element.function == RPNElement::FUNCTION_NOT)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
else if (element.function == RPNElement::FUNCTION_AND)
|
|
|
|
{
|
2019-12-18 19:00:09 +00:00
|
|
|
assert(!rpn_stack.empty());
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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)
|
|
|
|
{
|
2019-12-18 19:00:09 +00:00
|
|
|
assert(!rpn_stack.empty());
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
auto arg1 = rpn_stack.back();
|
|
|
|
rpn_stack.pop_back();
|
|
|
|
auto arg2 = rpn_stack.back();
|
|
|
|
rpn_stack.back() = arg1 | arg2;
|
|
|
|
}
|
|
|
|
else
|
2018-04-20 00:20:36 +00:00
|
|
|
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-12-18 19:00:09 +00:00
|
|
|
if (rpn_stack.size() != 1)
|
2019-12-18 19:04:33 +00:00
|
|
|
throw Exception("Unexpected stack size in KeyCondition::alwaysUnknownOrTrue", ErrorCodes::LOGICAL_ERROR);
|
2019-12-18 19:00:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return rpn_stack[0];
|
2015-03-27 03:06:06 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
size_t KeyCondition::getMaxKeyColumn() const
|
2016-03-27 11:37:25 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
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_IN_SET
|
|
|
|
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
|
|
|
|
{
|
|
|
|
if (element.key_column > res)
|
|
|
|
res = element.key_column;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return res;
|
2016-03-27 11:37:25 +00:00
|
|
|
}
|
|
|
|
|
2012-12-05 12:44:55 +00:00
|
|
|
}
|