2012-12-05 12:44:55 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <sstream>
|
2018-02-08 15:25:46 +00:00
|
|
|
#include <optional>
|
2012-12-05 12:44:55 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Context.h>
|
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
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2017-07-14 00:33:37 +00:00
|
|
|
#include <Interpreters/Set.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/SortDescription.h>
|
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
2017-07-15 03:48:36 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2012-12-05 12:44:55 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2013-12-12 00:50:54 +00:00
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
class IFunction;
|
2018-02-02 08:33:36 +00:00
|
|
|
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-08-29 18:51:48 +00:00
|
|
|
/** Range with open or closed ends; possibly unbounded.
|
2013-12-12 00:50:54 +00:00
|
|
|
*/
|
2012-12-05 12:44:55 +00:00
|
|
|
struct Range
|
|
|
|
{
|
2013-12-12 00:50:54 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
static bool equals(const Field & lhs, const Field & rhs);
|
|
|
|
static bool less(const Field & lhs, const Field & rhs);
|
2015-03-27 03:06:06 +00:00
|
|
|
|
2013-12-12 00:50:54 +00:00
|
|
|
public:
|
2017-04-16 15:00:33 +00:00
|
|
|
Field left; /// the left border, if any
|
|
|
|
Field right; /// the right border, if any
|
2017-08-29 18:51:48 +00:00
|
|
|
bool left_bounded = false; /// bounded at the left
|
|
|
|
bool right_bounded = false; /// bounded at the right
|
2017-04-16 15:00:33 +00:00
|
|
|
bool left_included = false; /// includes the left border, if any
|
|
|
|
bool right_included = false; /// includes the right border, if any
|
|
|
|
|
2017-08-29 18:51:48 +00:00
|
|
|
/// The whole unversum.
|
2017-04-01 07:20:54 +00:00
|
|
|
Range() {}
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// One point.
|
2017-04-01 07:20:54 +00:00
|
|
|
Range(const Field & point)
|
|
|
|
: left(point), right(point), left_bounded(true), right_bounded(true), left_included(true), right_included(true) {}
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// A bounded two-sided range.
|
2017-04-01 07:20:54 +00:00
|
|
|
Range(const Field & left_, bool left_included_, const Field & right_, bool right_included_)
|
|
|
|
: left(left_), right(right_),
|
|
|
|
left_bounded(true), right_bounded(true),
|
|
|
|
left_included(left_included_), right_included(right_included_)
|
|
|
|
{
|
|
|
|
shrinkToIncludedIfPossible();
|
|
|
|
}
|
|
|
|
|
|
|
|
static Range createRightBounded(const Field & right_point, bool right_included)
|
|
|
|
{
|
|
|
|
Range r;
|
|
|
|
r.right = right_point;
|
|
|
|
r.right_bounded = true;
|
|
|
|
r.right_included = right_included;
|
|
|
|
r.shrinkToIncludedIfPossible();
|
|
|
|
return r;
|
|
|
|
}
|
|
|
|
|
|
|
|
static Range createLeftBounded(const Field & left_point, bool left_included)
|
|
|
|
{
|
|
|
|
Range r;
|
|
|
|
r.left = left_point;
|
|
|
|
r.left_bounded = true;
|
|
|
|
r.left_included = left_included;
|
|
|
|
r.shrinkToIncludedIfPossible();
|
|
|
|
return r;
|
|
|
|
}
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Optimize the range. If it has an open boundary and the Field type is "loose"
|
|
|
|
* - then convert it to closed, narrowing by one.
|
|
|
|
* That is, for example, turn (0,2) into [1].
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void shrinkToIncludedIfPossible()
|
|
|
|
{
|
|
|
|
if (left_bounded && !left_included)
|
|
|
|
{
|
|
|
|
if (left.getType() == Field::Types::UInt64 && left.get<UInt64>() != std::numeric_limits<UInt64>::max())
|
|
|
|
{
|
|
|
|
++left.get<UInt64 &>();
|
|
|
|
left_included = true;
|
|
|
|
}
|
|
|
|
if (left.getType() == Field::Types::Int64 && left.get<Int64>() != std::numeric_limits<Int64>::max())
|
|
|
|
{
|
|
|
|
++left.get<Int64 &>();
|
|
|
|
left_included = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (right_bounded && !right_included)
|
|
|
|
{
|
|
|
|
if (right.getType() == Field::Types::UInt64 && right.get<UInt64>() != std::numeric_limits<UInt64>::min())
|
|
|
|
{
|
|
|
|
--right.get<UInt64 &>();
|
|
|
|
right_included = true;
|
|
|
|
}
|
|
|
|
if (right.getType() == Field::Types::Int64 && right.get<Int64>() != std::numeric_limits<Int64>::min())
|
|
|
|
{
|
|
|
|
--right.get<Int64 &>();
|
|
|
|
right_included = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
bool empty() const
|
|
|
|
{
|
|
|
|
return left_bounded && right_bounded
|
|
|
|
&& (less(right, left)
|
|
|
|
|| ((!left_included || !right_included) && !less(left, right)));
|
|
|
|
}
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// x contained in the range
|
2017-04-01 07:20:54 +00:00
|
|
|
bool contains(const Field & x) const
|
|
|
|
{
|
|
|
|
return !leftThan(x) && !rightThan(x);
|
|
|
|
}
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// x is to the left
|
2017-04-01 07:20:54 +00:00
|
|
|
bool rightThan(const Field & x) const
|
|
|
|
{
|
|
|
|
return (left_bounded
|
|
|
|
? !(less(left, x) || (left_included && equals(x, left)))
|
|
|
|
: false);
|
|
|
|
}
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// x is to the right
|
2017-04-01 07:20:54 +00:00
|
|
|
bool leftThan(const Field & x) const
|
|
|
|
{
|
|
|
|
return (right_bounded
|
|
|
|
? !(less(x, right) || (right_included && equals(x, right)))
|
|
|
|
: false);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool intersectsRange(const Range & r) const
|
|
|
|
{
|
2017-04-16 15:00:33 +00:00
|
|
|
/// r to the left of me.
|
2017-04-01 07:20:54 +00:00
|
|
|
if (r.right_bounded
|
|
|
|
&& left_bounded
|
|
|
|
&& (less(r.right, left)
|
|
|
|
|| ((!left_included || !r.right_included)
|
|
|
|
&& equals(r.right, left))))
|
|
|
|
return false;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// r to the right of me.
|
2017-04-01 07:20:54 +00:00
|
|
|
if (r.left_bounded
|
|
|
|
&& right_bounded
|
2017-08-29 18:51:48 +00:00
|
|
|
&& (less(right, r.left) /// ...} {...
|
2017-04-16 15:00:33 +00:00
|
|
|
|| ((!right_included || !r.left_included) /// ...) [... or ...] (...
|
2017-04-01 07:20:54 +00:00
|
|
|
&& equals(r.left, right))))
|
|
|
|
return false;
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool containsRange(const Range & r) const
|
|
|
|
{
|
2017-04-16 15:00:33 +00:00
|
|
|
/// r starts to the left of me.
|
2017-04-01 07:20:54 +00:00
|
|
|
if (left_bounded
|
|
|
|
&& (!r.left_bounded
|
|
|
|
|| less(r.left, left)
|
|
|
|
|| (r.left_included
|
|
|
|
&& !left_included
|
|
|
|
&& equals(r.left, left))))
|
|
|
|
return false;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// r ends right of me.
|
2017-04-01 07:20:54 +00:00
|
|
|
if (right_bounded
|
|
|
|
&& (!r.right_bounded
|
|
|
|
|| less(right, r.right)
|
|
|
|
|| (r.right_included
|
|
|
|
&& !right_included
|
|
|
|
&& equals(r.right, right))))
|
|
|
|
return false;
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
void swapLeftAndRight()
|
|
|
|
{
|
|
|
|
std::swap(left, right);
|
|
|
|
std::swap(left_bounded, right_bounded);
|
|
|
|
std::swap(left_included, right_included);
|
|
|
|
}
|
|
|
|
|
|
|
|
String toString() const;
|
2012-12-05 12:44:55 +00:00
|
|
|
};
|
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2018-02-08 14:15:21 +00:00
|
|
|
/// Class that extends arbitrary objects with infinities, like +-inf for floats
|
2018-02-08 15:25:46 +00:00
|
|
|
class FieldWithInfinity
|
|
|
|
{
|
2018-02-08 14:15:21 +00:00
|
|
|
public:
|
2018-02-08 15:31:37 +00:00
|
|
|
enum Type
|
|
|
|
{
|
2018-02-08 14:15:21 +00:00
|
|
|
MINUS_INFINITY = -1,
|
|
|
|
NORMAL = 0,
|
|
|
|
PLUS_INFINITY = 1
|
|
|
|
};
|
|
|
|
|
|
|
|
explicit FieldWithInfinity(const Field & field_);
|
|
|
|
FieldWithInfinity(Field && field_);
|
|
|
|
|
|
|
|
static FieldWithInfinity getMinusInfinity();
|
|
|
|
static FieldWithInfinity getPlusinfinity();
|
|
|
|
|
|
|
|
bool operator<(const FieldWithInfinity & other) const;
|
|
|
|
bool operator==(const FieldWithInfinity & other) const;
|
2018-02-08 15:25:46 +00:00
|
|
|
|
2018-02-08 14:15:21 +00:00
|
|
|
private:
|
|
|
|
Field field;
|
|
|
|
Type type;
|
|
|
|
|
|
|
|
FieldWithInfinity(const Type type_);
|
|
|
|
};
|
2016-03-27 11:37:25 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Condition on the index.
|
2015-03-27 03:06:06 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* Consists of the conditions for the key belonging to all possible ranges or sets,
|
2017-08-29 19:02:07 +00:00
|
|
|
* as well as logical operators AND/OR/NOT above these conditions.
|
2015-03-27 03:06:06 +00:00
|
|
|
*
|
2017-04-16 15:00:33 +00:00
|
|
|
* Constructs a reverse polish notation from these conditions
|
2017-08-29 19:02:07 +00:00
|
|
|
* and can calculate (interpret) its satisfiability over key ranges.
|
2015-03-27 03:06:06 +00:00
|
|
|
*/
|
2018-04-20 00:20:36 +00:00
|
|
|
class KeyCondition
|
2012-12-05 12:44:55 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-08-29 19:02:07 +00:00
|
|
|
/// Does not take into account the SAMPLE section. all_columns - the set of all columns of the table.
|
2018-04-20 00:20:36 +00:00
|
|
|
KeyCondition(
|
2017-07-15 03:48:36 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2017-07-14 00:33:37 +00:00
|
|
|
const Context & context,
|
2018-06-30 21:35:01 +00:00
|
|
|
const Names & key_column_names,
|
2018-04-20 00:20:36 +00:00
|
|
|
const ExpressionActionsPtr & key_expr);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Whether the condition is feasible in the key range.
|
2018-04-20 00:27:25 +00:00
|
|
|
/// left_key and right_key must contain all fields in the sort_descr in the appropriate order.
|
|
|
|
/// data_types - the types of the key columns.
|
|
|
|
bool mayBeTrueInRange(size_t used_key_size, const Field * left_key, const Field * right_key, const DataTypes & data_types) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-04 16:20:02 +00:00
|
|
|
/// Whether the condition is feasible in the direct product of single column ranges specified by `parallelogram`.
|
|
|
|
bool mayBeTrueInParallelogram(const std::vector<Range> & parallelogram, const DataTypes & data_types) const;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Is the condition valid in a semi-infinite (not limited to the right) key range.
|
2018-04-20 00:27:25 +00:00
|
|
|
/// left_key must contain all the fields in the sort_descr in the appropriate order.
|
|
|
|
bool mayBeTrueAfter(size_t used_key_size, const Field * left_key, const DataTypes & data_types) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Checks that the index can not be used.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool alwaysUnknownOrTrue() const;
|
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
/// Get the maximum number of the key element used in the condition.
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t getMaxKeyColumn() const;
|
|
|
|
|
2018-12-11 17:43:12 +00:00
|
|
|
/// Impose an additional condition: the value in the column `column` must be in the range `range`.
|
2018-04-20 00:27:25 +00:00
|
|
|
/// Returns whether there is such a column in the key.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool addCondition(const String & column, const Range & range);
|
|
|
|
|
|
|
|
String toString() const;
|
|
|
|
|
|
|
|
|
2018-06-04 17:01:00 +00:00
|
|
|
/** A chain of possibly monotone functions.
|
|
|
|
* If the key column is wrapped in functions that can be monotonous in some value ranges
|
|
|
|
* (for example: -toFloat64(toDayOfWeek(date))), then here the functions will be located: toDayOfWeek, toFloat64, negate.
|
|
|
|
*/
|
|
|
|
using MonotonicFunctionsChain = std::vector<FunctionBasePtr>;
|
|
|
|
|
2019-02-20 16:24:46 +00:00
|
|
|
/** Computes value of constant expression and its data type.
|
|
|
|
* Returns false, if expression isn't constant.
|
|
|
|
*/
|
|
|
|
static bool getConstant(
|
|
|
|
const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type);
|
2018-06-04 17:01:00 +00:00
|
|
|
|
|
|
|
static Block getBlockWithConstants(
|
2018-11-08 17:28:52 +00:00
|
|
|
const ASTPtr & query, const SyntaxAnalyzerResultPtr & syntax_analyzer_result, const Context & context);
|
2018-06-04 17:01:00 +00:00
|
|
|
|
|
|
|
static std::optional<Range> applyMonotonicFunctionsChainToRange(
|
|
|
|
Range key_range,
|
|
|
|
MonotonicFunctionsChain & functions,
|
|
|
|
DataTypePtr current_type);
|
|
|
|
|
|
|
|
private:
|
2017-04-16 15:00:33 +00:00
|
|
|
/// The expression is stored as Reverse Polish Notation.
|
2017-04-01 07:20:54 +00:00
|
|
|
struct RPNElement
|
|
|
|
{
|
|
|
|
enum Function
|
|
|
|
{
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Atoms of a Boolean expression.
|
2017-04-01 07:20:54 +00:00
|
|
|
FUNCTION_IN_RANGE,
|
|
|
|
FUNCTION_NOT_IN_RANGE,
|
|
|
|
FUNCTION_IN_SET,
|
|
|
|
FUNCTION_NOT_IN_SET,
|
2017-04-16 15:00:33 +00:00
|
|
|
FUNCTION_UNKNOWN, /// Can take any value.
|
|
|
|
/// Operators of the logical expression.
|
2017-04-01 07:20:54 +00:00
|
|
|
FUNCTION_NOT,
|
|
|
|
FUNCTION_AND,
|
|
|
|
FUNCTION_OR,
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Constants
|
2017-04-01 07:20:54 +00:00
|
|
|
ALWAYS_FALSE,
|
|
|
|
ALWAYS_TRUE,
|
|
|
|
};
|
|
|
|
|
|
|
|
RPNElement() {}
|
|
|
|
RPNElement(Function function_) : function(function_) {}
|
|
|
|
RPNElement(Function function_, size_t key_column_) : function(function_), key_column(key_column_) {}
|
|
|
|
RPNElement(Function function_, size_t key_column_, const Range & range_)
|
|
|
|
: function(function_), range(range_), key_column(key_column_) {}
|
|
|
|
|
|
|
|
String toString() const;
|
|
|
|
|
|
|
|
Function function = FUNCTION_UNKNOWN;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// For FUNCTION_IN_RANGE and FUNCTION_NOT_IN_RANGE.
|
2017-04-01 07:20:54 +00:00
|
|
|
Range range;
|
2018-06-04 17:01:00 +00:00
|
|
|
size_t key_column = 0;
|
2017-04-16 15:00:33 +00:00
|
|
|
/// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET
|
2018-02-02 13:19:40 +00:00
|
|
|
using MergeTreeSetIndexPtr = std::shared_ptr<MergeTreeSetIndex>;
|
|
|
|
MergeTreeSetIndexPtr set_index;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
mutable MonotonicFunctionsChain monotonic_functions_chain; /// The function execution does not violate the constancy.
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
2018-06-04 17:01:00 +00:00
|
|
|
using RPN = std::vector<RPNElement>;
|
|
|
|
using ColumnIndices = std::map<String, size_t>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-01-22 12:33:56 +00:00
|
|
|
using AtomMap = std::unordered_map<std::string, bool(*)(RPNElement & out, const Field & value)>;
|
2018-02-02 13:19:40 +00:00
|
|
|
|
2018-06-04 17:01:00 +00:00
|
|
|
public:
|
|
|
|
static const AtomMap atom_map;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
bool mayBeTrueInRange(
|
|
|
|
size_t used_key_size,
|
2018-04-20 00:27:25 +00:00
|
|
|
const Field * left_key,
|
|
|
|
const Field * right_key,
|
2017-04-01 07:20:54 +00:00
|
|
|
const DataTypes & data_types,
|
|
|
|
bool right_bounded) const;
|
|
|
|
|
2019-03-08 08:29:48 +00:00
|
|
|
void traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants);
|
2017-05-25 01:12:41 +00:00
|
|
|
bool atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
|
2019-03-08 08:29:48 +00:00
|
|
|
bool operatorFromAST(const ASTFunction * func, RPNElement & out);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
/** Is node the key column
|
|
|
|
* or expression in which column of key is wrapped by chain of functions,
|
2019-01-22 19:56:53 +00:00
|
|
|
* that can be monotonic on certain ranges?
|
2018-04-20 00:27:25 +00:00
|
|
|
* If these conditions are true, then returns number of column in key, type of resulting expression
|
2017-04-01 07:20:54 +00:00
|
|
|
* and fills chain of possibly-monotonic functions.
|
|
|
|
*/
|
2018-04-20 00:27:25 +00:00
|
|
|
bool 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);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-20 00:27:25 +00:00
|
|
|
bool 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);
|
|
|
|
|
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 canConstantBeWrappedByMonotonicFunctions(
|
|
|
|
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);
|
|
|
|
|
2018-07-02 18:57:14 +00:00
|
|
|
/// If it's possible to make an RPNElement
|
|
|
|
/// that will filter values (possibly tuples) by the content of 'prepared_set',
|
|
|
|
/// do it and return true.
|
|
|
|
bool 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-02 13:19:40 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
RPN rpn;
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
ColumnIndices key_columns;
|
|
|
|
ExpressionActionsPtr key_expr;
|
2017-07-15 03:48:36 +00:00
|
|
|
PreparedSets prepared_sets;
|
2012-12-05 12:44:55 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|