2012-12-05 12:44:55 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <sstream>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Core/SortDescription.h>
|
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTLiteral.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;
|
|
|
|
using FunctionPtr = std::shared_ptr<IFunction>;
|
|
|
|
|
2013-12-12 00:50:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Range with open or closed ends; Perhaps unlimited.
|
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
|
|
|
|
bool left_bounded = false; /// limited to the left
|
|
|
|
bool right_bounded = false; /// limited to the right
|
|
|
|
bool left_included = false; /// includes the left border, if any
|
|
|
|
bool right_included = false; /// includes the right border, if any
|
|
|
|
|
|
|
|
/// The whole set.
|
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
|
|
|
|
&& (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
|
|
|
};
|
|
|
|
|
2016-03-27 11:37:25 +00:00
|
|
|
|
2014-05-12 00:45:50 +00:00
|
|
|
class ASTSet;
|
2015-03-27 03:06:06 +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,
|
|
|
|
* as well as logical links 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
|
|
|
|
* and can calculate (interpret) its feasibility over key ranges.
|
2015-03-27 03:06:06 +00:00
|
|
|
*/
|
2012-12-10 10:23:10 +00:00
|
|
|
class PKCondition
|
2012-12-05 12:44:55 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Does not include the SAMPLE section. all_columns - the set of all columns of the table.
|
2017-05-25 01:12:41 +00:00
|
|
|
PKCondition(const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Block & pk_sample_block);
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Whether the condition is feasible in the key range.
|
|
|
|
/// left_pk and right_pk must contain all fields in the sort_descr in the appropriate order.
|
|
|
|
/// data_types - the types of the primary key columns.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool mayBeTrueInRange(size_t used_key_size, const Field * left_pk, const Field * right_pk, 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.
|
|
|
|
/// left_pk must contain all the fields in the sort_descr in the appropriate order.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool mayBeTrueAfter(size_t used_key_size, const Field * left_pk, const DataTypes & data_types) const;
|
|
|
|
|
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;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Get the maximum number of the primary key element used in the condition.
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t getMaxKeyColumn() const;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/// Impose an additional condition: the value in the column column must be in the `range` range.
|
|
|
|
/// Returns whether there is such a column in the primary key.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool addCondition(const String & column, const Range & range);
|
|
|
|
|
|
|
|
String toString() const;
|
|
|
|
|
|
|
|
|
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;
|
|
|
|
size_t key_column;
|
2017-04-16 15:00:33 +00:00
|
|
|
/// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTPtr in_function;
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** A chain of possibly monotone functions.
|
|
|
|
* If the primary 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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
using MonotonicFunctionsChain = std::vector<FunctionPtr>;
|
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
|
|
|
};
|
|
|
|
|
|
|
|
static Block getBlockWithConstants(
|
|
|
|
const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns);
|
|
|
|
|
2017-05-25 01:12:41 +00:00
|
|
|
using AtomMap = std::unordered_map<std::string, bool(*)(RPNElement & out, const Field & value, const ASTPtr & node)>;
|
2017-04-01 07:20:54 +00:00
|
|
|
static const AtomMap atom_map;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
using RPN = std::vector<RPNElement>;
|
|
|
|
using ColumnIndices = std::map<String, size_t>;
|
|
|
|
|
|
|
|
bool mayBeTrueInRange(
|
|
|
|
size_t used_key_size,
|
|
|
|
const Field * left_pk,
|
|
|
|
const Field * right_pk,
|
|
|
|
const DataTypes & data_types,
|
|
|
|
bool right_bounded) const;
|
|
|
|
|
|
|
|
bool mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, const DataTypes & data_types) const;
|
|
|
|
|
2017-05-25 01:12:41 +00:00
|
|
|
void traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants);
|
|
|
|
bool atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
|
2017-04-01 07:20:54 +00:00
|
|
|
bool operatorFromAST(const ASTFunction * func, RPNElement & out);
|
|
|
|
|
|
|
|
/** Is node the primary key column
|
|
|
|
* or expression in which column of primary key is wrapped by chain of functions,
|
|
|
|
* that can be monotomic on certain ranges?
|
|
|
|
* If these conditions are true, then returns number of column in primary key, type of resulting expression
|
|
|
|
* and fills chain of possibly-monotonic functions.
|
|
|
|
*/
|
|
|
|
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
|
|
|
|
const ASTPtr & node,
|
|
|
|
const Context & context,
|
|
|
|
size_t & out_primary_key_column_num,
|
|
|
|
DataTypePtr & out_primary_key_res_column_type,
|
|
|
|
RPNElement::MonotonicFunctionsChain & out_functions_chain);
|
|
|
|
|
|
|
|
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
|
|
|
const ASTPtr & node,
|
|
|
|
size_t & out_primary_key_column_num,
|
|
|
|
DataTypePtr & out_primary_key_column_type,
|
|
|
|
std::vector<const ASTFunction *> & out_functions_chain);
|
|
|
|
|
|
|
|
RPN rpn;
|
|
|
|
|
|
|
|
SortDescription sort_descr;
|
|
|
|
ColumnIndices pk_columns;
|
|
|
|
const Block & pk_sample_block;
|
2012-12-05 12:44:55 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|