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>
|
2020-09-05 14:12:47 +00:00
|
|
|
#include <DataTypes/FieldToDataType.h>
|
2020-07-22 17:13:05 +00:00
|
|
|
#include <Interpreters/TreeRewriter.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2020-11-06 03:50:58 +00:00
|
|
|
#include <Interpreters/castColumn.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>
|
2020-06-14 18:42:10 +00:00
|
|
|
#include <Common/FieldVisitorsAccurateComparison.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>
|
2020-11-09 19:07:38 +00:00
|
|
|
#include <IO/WriteBufferFromString.h>
|
|
|
|
#include <IO/Operators.h>
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2019-12-18 19:00:09 +00:00
|
|
|
#include <cassert>
|
2020-11-09 11:58:11 +00:00
|
|
|
#include <stack>
|
2019-12-18 19:00:09 +00:00
|
|
|
|
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
|
|
|
|
{
|
2020-11-09 19:07:38 +00:00
|
|
|
WriteBufferFromOwnString 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",
|
2020-05-29 21:32:35 +00:00
|
|
|
[] (RPNElement & out, const Field & value)
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
2020-05-29 21:32:35 +00:00
|
|
|
if (value.getType() != Field::Types::String)
|
|
|
|
return false;
|
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
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)
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
2020-05-29 21:32:35 +00:00
|
|
|
if (value.getType() != Field::Types::String)
|
|
|
|
return false;
|
|
|
|
|
2019-07-10 14:53:57 +00:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2020-04-22 06:22:14 +00:00
|
|
|
auto cloned_node = node->clone();
|
2020-02-20 09:53:02 +00:00
|
|
|
|
|
|
|
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
|
|
|
|
|
|
|
|
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(
|
2020-07-22 17:13:05 +00:00
|
|
|
const ASTPtr & query, const TreeRewriterResultPtr & 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,
|
2020-09-21 10:13:01 +00:00
|
|
|
const ExpressionActionsPtr & key_expr_,
|
|
|
|
bool single_point_,
|
|
|
|
bool strict_)
|
|
|
|
: key_expr(key_expr_), prepared_sets(query_info.sets), 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)
|
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
|
|
|
|
2020-04-02 17:27:07 +00:00
|
|
|
static Field applyFunctionForField(
|
2018-02-06 19:34:53 +00:00
|
|
|
const FunctionBasePtr & func,
|
2020-04-02 17:27:07 +00:00
|
|
|
const DataTypePtr & arg_type,
|
|
|
|
const Field & arg_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
|
|
|
{
|
2020-10-09 07:41:28 +00:00
|
|
|
ColumnsWithTypeAndName columns
|
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-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
|
|
|
};
|
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
auto col = func->execute(columns, func->getResultType(), 1);
|
|
|
|
return (*col)[0];
|
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
|
|
|
}
|
|
|
|
|
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(
|
2020-08-02 20:55:20 +00:00
|
|
|
const FunctionOverloadResolverPtr & func,
|
|
|
|
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" }};
|
2020-08-02 20:55:20 +00:00
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
FunctionBasePtr func_base = func->build(arguments);
|
2020-08-02 20:55:20 +00:00
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
DataTypePtr return_type = func_base->getResultType();
|
2020-08-02 21:01:39 +00:00
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
auto col = func_base->execute(arguments, return_type, 1);
|
2020-08-02 21:01:39 +00:00
|
|
|
|
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)
|
2020-04-02 17:27:07 +00:00
|
|
|
{
|
|
|
|
/// Fallback for fields without block reference.
|
|
|
|
if (field.isExplicit())
|
|
|
|
return applyFunctionForField(func, current_type, field);
|
|
|
|
|
|
|
|
String result_name = "_" + func->getName() + "_" + toString(field.column_idx);
|
2020-10-09 07:41:28 +00:00
|
|
|
const auto & columns = field.columns;
|
|
|
|
size_t result_idx = columns->size();
|
|
|
|
|
|
|
|
for (size_t i = 0; i < result_idx; ++i)
|
2020-04-02 17:27:07 +00:00
|
|
|
{
|
2020-10-09 07:41:28 +00:00
|
|
|
if ((*columns)[i].name == result_name)
|
|
|
|
result_idx = i;
|
|
|
|
}
|
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
ColumnsWithTypeAndName args{(*columns)[field.column_idx]};
|
2020-10-09 07:41:28 +00:00
|
|
|
if (result_idx == columns->size())
|
|
|
|
{
|
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());
|
2020-04-02 17:27:07 +00:00
|
|
|
}
|
|
|
|
|
2020-10-09 07:41:28 +00:00
|
|
|
return {field.columns, field.row_idx, result_idx};
|
2020-04-02 17:27:07 +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
|
|
|
|
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)
|
|
|
|
{
|
2020-09-21 10:13:01 +00:00
|
|
|
/// 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');
|
|
|
|
///
|
|
|
|
/// toDate(DateTime) is always monotonic, but we cannot relaxing the predicates to be
|
|
|
|
/// >= toDate(toDateTime('2020-09-01 10:00:00')), which returns 3 instead of the right count: 2.
|
|
|
|
if (strict)
|
|
|
|
return false;
|
|
|
|
|
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
|
|
|
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;
|
|
|
|
|
2020-11-06 11:18:42 +00:00
|
|
|
/// TODO Nullable index is not yet landed.
|
|
|
|
if (out_value.isNull())
|
|
|
|
return false;
|
|
|
|
|
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
|
|
|
bool found_transformation = false;
|
2020-11-06 11:18:42 +00:00
|
|
|
auto input_column = sample_block.getByName(expr_name);
|
|
|
|
auto const_column = out_type->createColumnConst(1, out_value);
|
|
|
|
out_value = (*castColumn({const_column, out_type, "c"}, input_column.type))[0];
|
|
|
|
out_type = input_column.type;
|
2020-11-03 11:28:28 +00:00
|
|
|
for (const auto & action : 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.
|
|
|
|
*/
|
2020-11-03 11:28:28 +00:00
|
|
|
const auto & children = action.node->children;
|
2020-11-10 14:54:59 +00:00
|
|
|
if (action.node->type == ActionsDAG::ActionType::FUNCTION
|
2020-11-03 11:28:28 +00:00
|
|
|
&& children.size() == 1
|
|
|
|
&& children[0]->result_name == expr_name)
|
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
|
|
|
{
|
2020-11-03 11:28:28 +00:00
|
|
|
if (!action.node->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;
|
|
|
|
|
2020-08-02 20:55:20 +00:00
|
|
|
/// Range is irrelevant in this case.
|
2020-11-03 11:28:28 +00:00
|
|
|
IFunction::Monotonicity monotonicity = action.node->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;
|
|
|
|
|
2020-08-02 20:55:20 +00:00
|
|
|
/// Apply the next transformation step.
|
2020-08-02 21:01:39 +00:00
|
|
|
std::tie(out_value, out_type) = applyFunctionForFieldOfUnknownType(
|
2020-11-03 11:28:28 +00:00
|
|
|
action.node->function_builder,
|
2020-08-02 20:55:20 +00:00
|
|
|
out_type, out_value);
|
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
expr_name = action.node->result_name;
|
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
|
|
|
|
2020-08-02 20:55:20 +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;
|
|
|
|
}
|
|
|
|
|
2020-11-06 03:50:58 +00:00
|
|
|
/// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)`
|
|
|
|
bool KeyCondition::canConstantBeWrappedByFunctions(
|
2020-11-09 11:58:11 +00:00
|
|
|
const ASTPtr & ast, 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
|
|
|
{
|
2020-11-06 11:18:42 +00:00
|
|
|
if (strict)
|
|
|
|
return false;
|
|
|
|
|
2020-11-09 11:58:11 +00:00
|
|
|
String expr_name = ast->getColumnName();
|
2020-11-06 03:50:58 +00:00
|
|
|
const auto & sample_block = key_expr->getSampleBlock();
|
|
|
|
if (!sample_block.has(expr_name))
|
|
|
|
return false;
|
|
|
|
|
2020-11-06 11:18:42 +00:00
|
|
|
/// TODO Nullable index is not yet landed.
|
|
|
|
if (out_value.isNull())
|
|
|
|
return false;
|
|
|
|
|
2020-11-09 11:58:11 +00:00
|
|
|
for (const auto & node : key_expr->getNodes())
|
2020-11-06 03:50:58 +00:00
|
|
|
{
|
2020-11-09 11:58:11 +00:00
|
|
|
auto it = key_columns.find(node.result_name);
|
|
|
|
if (it != key_columns.end())
|
2020-11-06 03:50:58 +00:00
|
|
|
{
|
2020-11-09 11:58:11 +00:00
|
|
|
std::stack<const ActionsDAG::Node *> chain;
|
|
|
|
|
|
|
|
const auto * cur_node = &node;
|
|
|
|
bool is_valid_chain = true;
|
|
|
|
|
|
|
|
while (is_valid_chain)
|
2020-11-06 03:50:58 +00:00
|
|
|
{
|
2020-11-09 11:58:11 +00:00
|
|
|
if (cur_node->result_name == expr_name)
|
|
|
|
break;
|
|
|
|
|
|
|
|
chain.push(cur_node);
|
|
|
|
|
2020-11-10 14:54:59 +00:00
|
|
|
if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2)
|
2020-11-09 11:58:11 +00:00
|
|
|
{
|
|
|
|
if (!cur_node->function_base->isDeterministic())
|
|
|
|
is_valid_chain = false;
|
|
|
|
|
|
|
|
const ActionsDAG::Node * next_node = nullptr;
|
|
|
|
for (const auto * arg : cur_node->children)
|
|
|
|
{
|
|
|
|
if (arg->column && isColumnConst(*arg->column))
|
|
|
|
continue;
|
|
|
|
|
|
|
|
if (next_node)
|
|
|
|
is_valid_chain = false;
|
|
|
|
|
|
|
|
next_node = arg;
|
|
|
|
}
|
|
|
|
|
2020-11-09 16:35:43 +00:00
|
|
|
if (!next_node)
|
|
|
|
is_valid_chain = false;
|
|
|
|
|
2020-11-09 11:58:11 +00:00
|
|
|
cur_node = next_node;
|
|
|
|
}
|
2020-11-10 14:54:59 +00:00
|
|
|
else if (cur_node->type == ActionsDAG::ActionType::ALIAS)
|
2020-11-09 11:58:11 +00:00
|
|
|
cur_node = cur_node->children.front();
|
|
|
|
else
|
|
|
|
is_valid_chain = false;
|
2020-11-06 03:50:58 +00:00
|
|
|
}
|
2020-11-09 11:58:11 +00:00
|
|
|
|
|
|
|
if (is_valid_chain)
|
2020-11-06 03:50:58 +00:00
|
|
|
{
|
|
|
|
{
|
2020-11-09 11:58:11 +00:00
|
|
|
auto input_column = sample_block.getByName(expr_name);
|
|
|
|
auto const_column = out_type->createColumnConst(1, out_value);
|
|
|
|
out_value = (*castColumn({const_column, out_type, "c"}, input_column.type))[0];
|
|
|
|
out_type = input_column.type;
|
2020-11-06 03:50:58 +00:00
|
|
|
}
|
2020-11-09 11:58:11 +00:00
|
|
|
|
|
|
|
while (!chain.empty())
|
2020-11-06 03:50:58 +00:00
|
|
|
{
|
2020-11-09 11:58:11 +00:00
|
|
|
const auto * func = chain.top();
|
|
|
|
chain.pop();
|
|
|
|
|
2020-11-10 14:54:59 +00:00
|
|
|
if (func->type != ActionsDAG::ActionType::FUNCTION)
|
2020-11-09 11:58:11 +00:00
|
|
|
continue;
|
|
|
|
|
|
|
|
if (func->children.size() == 1)
|
|
|
|
{
|
|
|
|
std::tie(out_value, out_type) = applyFunctionForFieldOfUnknownType(func->function_builder, out_type, out_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(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType(
|
|
|
|
func->function_builder, left_arg_type, left_arg_value, out_type, out_value);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto right_arg_type = right->result_type;
|
|
|
|
auto right_arg_value = (*right->column)[0];
|
|
|
|
std::tie(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType(
|
|
|
|
func->function_builder, out_type, out_value, right_arg_type, right_arg_value);
|
|
|
|
}
|
|
|
|
}
|
2020-11-06 03:50:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
out_key_column_num = it->second;
|
|
|
|
out_key_column_type = sample_block.getByName(it->first).type;
|
2020-11-09 11:58:11 +00:00
|
|
|
return true;
|
2020-11-06 03:50:58 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-09 11:58:11 +00:00
|
|
|
return false;
|
2020-11-06 03:50:58 +00:00
|
|
|
}
|
|
|
|
|
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];
|
|
|
|
|
2020-06-30 15:33:16 +00:00
|
|
|
SetPtr prepared_set;
|
2019-03-11 13:22:51 +00:00
|
|
|
if (right_arg->as<ASTSubquery>() || right_arg->as<ASTIdentifier>())
|
2020-06-30 15:33:16 +00:00
|
|
|
{
|
2020-07-01 14:05:54 +00:00
|
|
|
auto set_it = prepared_sets.find(PreparedSetKey::forSubquery(*right_arg));
|
2020-06-30 15:33:16 +00:00
|
|
|
if (set_it == prepared_sets.end())
|
|
|
|
return false;
|
|
|
|
|
|
|
|
prepared_set = set_it->second;
|
|
|
|
}
|
2019-01-22 12:33:56 +00:00
|
|
|
else
|
2020-06-30 15:33:16 +00:00
|
|
|
{
|
2020-07-01 14:05:54 +00:00
|
|
|
/// 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 set_ast_hash = right_arg->getTreeHash();
|
2020-06-30 15:33:16 +00:00
|
|
|
auto set_it = std::find_if(
|
2020-07-01 14:05:54 +00:00
|
|
|
prepared_sets.begin(), prepared_sets.end(),
|
|
|
|
[&](const auto & candidate_entry)
|
2020-06-30 15:33:16 +00:00
|
|
|
{
|
2020-07-01 14:05:54 +00:00
|
|
|
if (candidate_entry.first.ast_hash != set_ast_hash)
|
|
|
|
return false;
|
2019-01-22 12:33:56 +00:00
|
|
|
|
2020-07-01 14:05:54 +00:00
|
|
|
for (size_t i = 0; i < indexes_mapping.size(); ++i)
|
|
|
|
if (!candidate_entry.second->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i]))
|
|
|
|
return false;
|
|
|
|
|
|
|
|
return true;
|
2020-06-30 15:33:16 +00:00
|
|
|
});
|
|
|
|
if (set_it == prepared_sets.end())
|
|
|
|
return false;
|
2019-01-22 12:33:56 +00:00
|
|
|
|
2020-06-30 15:33:16 +00:00
|
|
|
prepared_set = set_it->second;
|
|
|
|
}
|
2019-01-22 12:33:56 +00:00
|
|
|
|
|
|
|
/// 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)
|
2020-07-01 14:05:54 +00:00
|
|
|
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));
|
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)
|
|
|
|
{
|
2020-09-05 14:12:47 +00:00
|
|
|
const auto & args = (*it)->arguments->children;
|
2018-02-02 08:33:36 +00:00
|
|
|
auto func_builder = FunctionFactory::instance().tryGet((*it)->name, context);
|
2020-09-05 14:12:47 +00:00
|
|
|
ColumnsWithTypeAndName arguments;
|
|
|
|
if (args.size() == 2)
|
|
|
|
{
|
|
|
|
if (const auto * arg_left = args[0]->as<ASTLiteral>())
|
|
|
|
{
|
|
|
|
auto left_arg_type = applyVisitor(FieldToDataType(), arg_left->value);
|
|
|
|
arguments.push_back({ left_arg_type->createColumnConst(0, arg_left->value), left_arg_type, "" });
|
|
|
|
arguments.push_back({ nullptr, key_column_type, "" });
|
|
|
|
}
|
|
|
|
else if (const auto * arg_right = args[1]->as<ASTLiteral>())
|
|
|
|
{
|
|
|
|
arguments.push_back({ nullptr, key_column_type, "" });
|
|
|
|
auto right_arg_type = applyVisitor(FieldToDataType(), arg_right->value);
|
|
|
|
arguments.push_back({ right_arg_type->createColumnConst(0, arg_right->value), right_arg_type, "" });
|
|
|
|
}
|
|
|
|
}
|
|
|
|
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()))
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
2015-11-29 08:06:29 +00:00
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
key_column_type = func->getResultType();
|
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;
|
2020-09-13 13:19:02 +00:00
|
|
|
if (args.size() > 2 || args.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
return false;
|
|
|
|
|
|
|
|
out_functions_chain.push_back(func);
|
2020-09-05 14:12:47 +00:00
|
|
|
bool ret = false;
|
|
|
|
if (args.size() == 2)
|
|
|
|
{
|
|
|
|
if (args[0]->as<ASTLiteral>())
|
|
|
|
{
|
|
|
|
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[1], out_key_column_num, out_key_column_type, out_functions_chain);
|
|
|
|
}
|
|
|
|
else if (args[1]->as<ASTLiteral>())
|
|
|
|
{
|
|
|
|
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain);
|
|
|
|
}
|
|
|
|
return ret;
|
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
|
|
|
|
{
|
2020-05-10 23:17:54 +00:00
|
|
|
src_value = convertFieldToType(src_value, *desired_type, src_type.get());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
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
|
|
|
|
2020-11-06 11:18:42 +00:00
|
|
|
if (functionIsInOrGlobalInOperator(func_name))
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
2020-11-06 11:18:42 +00:00
|
|
|
if (tryPrepareSetIndex(args, context, out, key_column_num))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
is_set_const = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
2019-07-10 14:53:57 +00:00
|
|
|
}
|
2020-11-06 03:50:58 +00:00
|
|
|
else if (getConstant(args[1], block_with_constants, const_value, const_type))
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
2020-11-06 03:50:58 +00:00
|
|
|
if (isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
}
|
|
|
|
else if (canConstantBeWrappedByMonotonicFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
is_constant_transformed = true;
|
|
|
|
}
|
|
|
|
else if (
|
|
|
|
single_point && func_name == "equals"
|
|
|
|
&& canConstantBeWrappedByFunctions(args[0], key_column_num, key_expr_type, const_value, const_type))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
is_constant_transformed = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
2019-07-10 14:53:57 +00:00
|
|
|
}
|
2020-11-06 03:50:58 +00:00
|
|
|
else if (getConstant(args[0], block_with_constants, const_value, const_type))
|
2019-07-10 14:53:57 +00:00
|
|
|
{
|
2020-11-06 03:50:58 +00:00
|
|
|
if (isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
|
|
|
|
{
|
|
|
|
key_arg_pos = 1;
|
|
|
|
}
|
|
|
|
else if (canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
|
|
|
|
{
|
|
|
|
key_arg_pos = 1;
|
|
|
|
is_constant_transformed = true;
|
|
|
|
}
|
|
|
|
else if (
|
|
|
|
single_point && func_name == "equals"
|
|
|
|
&& canConstantBeWrappedByFunctions(args[1], key_column_num, key_expr_type, const_value, const_type))
|
|
|
|
{
|
|
|
|
key_arg_pos = 0;
|
|
|
|
is_constant_transformed = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
2019-07-10 14:53:57 +00:00
|
|
|
}
|
|
|
|
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" ||
|
2020-07-05 15:57:59 +00:00
|
|
|
func_name == "ilike" || func_name == "notIlike" ||
|
2020-02-20 09:53:02 +00:00
|
|
|
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 =
|
2020-06-14 00:16:01 +00:00
|
|
|
is_set_const /// Set args are already casted inside Set::createFromAST
|
|
|
|
|| (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast.
|
2019-07-10 14:53:57 +00:00
|
|
|
|
|
|
|
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
|
|
|
}
|
2020-07-10 06:30:49 +00:00
|
|
|
else if (getConstant(node, block_with_constants, const_value, const_type))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-07-10 06:30:49 +00:00
|
|
|
/// For cases where it says, for example, `WHERE 0 AND something`
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-07-10 06:30:49 +00:00
|
|
|
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)
|
|
|
|
{
|
|
|
|
out.function = const_value.safeGet<Float64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
|
2017-04-01 07:20:54 +00:00
|
|
|
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,
|
2020-04-02 17:27:07 +00:00
|
|
|
const FieldRef * key_left,
|
|
|
|
const FieldRef * key_right,
|
2017-04-01 07:20:54 +00:00
|
|
|
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,
|
2020-04-02 17:27:07 +00:00
|
|
|
const FieldRef * left_key,
|
|
|
|
const FieldRef * 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();
|
2020-07-29 16:09:38 +00:00
|
|
|
std::cerr << ": " << res.can_be_true << "\n";*/
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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,
|
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)
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2020-07-21 11:02:58 +00:00
|
|
|
for (const auto & func : functions)
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
|
|
|
/// 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);
|
2018-02-02 13:19:40 +00:00
|
|
|
|
|
|
|
if (!monotonicity.is_monotonic)
|
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2020-07-29 16:09:38 +00:00
|
|
|
/// 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.
|
2018-02-02 13:19:40 +00:00
|
|
|
if (!key_range.left.isNull())
|
2020-07-29 16:09:38 +00:00
|
|
|
{
|
2020-04-02 17:27:07 +00:00
|
|
|
key_range.left = applyFunction(func, current_type, key_range.left);
|
2020-07-29 16:09:38 +00:00
|
|
|
key_range.left_included = true;
|
|
|
|
}
|
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
if (!key_range.right.isNull())
|
2020-07-29 16:09:38 +00:00
|
|
|
{
|
2020-04-02 17:27:07 +00:00
|
|
|
key_range.right = applyFunction(func, current_type, key_range.right);
|
2020-07-29 16:09:38 +00:00
|
|
|
key_range.right_included = true;
|
|
|
|
}
|
2018-02-02 13:19:40 +00:00
|
|
|
|
2020-10-19 18:37:44 +00:00
|
|
|
current_type = func->getResultType();
|
2018-02-02 13:19:40 +00:00
|
|
|
|
|
|
|
if (!monotonicity.is_positive)
|
|
|
|
key_range.swapLeftAndRight();
|
|
|
|
}
|
|
|
|
return key_range;
|
|
|
|
}
|
2016-03-27 11:37:25 +00:00
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
2020-07-12 01:59:23 +00:00
|
|
|
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,
|
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,
|
2020-09-21 10:13:01 +00:00
|
|
|
data_types[element.key_column],
|
|
|
|
single_point
|
2018-02-02 13:19:40 +00:00
|
|
|
);
|
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,
|
2020-04-02 17:27:07 +00:00
|
|
|
const FieldRef * left_key,
|
|
|
|
const FieldRef * right_key,
|
2020-01-29 21:40:22 +00:00
|
|
|
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,
|
2020-04-02 17:27:07 +00:00
|
|
|
const FieldRef * left_key,
|
|
|
|
const FieldRef * right_key,
|
2020-01-29 21:40:22 +00:00
|
|
|
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,
|
2020-04-02 17:27:07 +00:00
|
|
|
const FieldRef * left_key,
|
2020-01-29 21:40:22 +00:00
|
|
|
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,
|
2020-04-02 17:27:07 +00:00
|
|
|
const FieldRef * left_key,
|
2020-01-29 21:40:22 +00:00
|
|
|
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
|
|
|
{
|
2020-11-09 19:07:38 +00:00
|
|
|
auto print_wrapped_column = [this](WriteBuffer & buf)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
|
2020-11-09 19:07:38 +00:00
|
|
|
buf << (*it)->getName() << "(";
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-09 19:07:38 +00:00
|
|
|
buf << "column " << key_column;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
|
2020-11-09 19:07:38 +00:00
|
|
|
buf << ")";
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
2020-11-09 19:07:38 +00:00
|
|
|
WriteBufferFromOwnString buf;
|
2017-04-01 07:20:54 +00:00
|
|
|
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();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
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();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
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
|
|
|
}
|
|
|
|
|
2020-04-02 17:27:07 +00:00
|
|
|
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
|
|
|
}
|