ClickHouse/src/Storages/MergeTree/KeyCondition.h

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

487 lines
17 KiB
C++
Raw Normal View History

2012-12-05 12:44:55 +00:00
#pragma once
2018-02-08 15:25:46 +00:00
#include <optional>
2012-12-05 12:44:55 +00:00
#include <Interpreters/Set.h>
#include <Core/SortDescription.h>
#include <Parsers/ASTExpressionList.h>
#include <Storages/SelectQueryInfo.h>
2012-12-05 12:44:55 +00:00
namespace DB
{
class ASTFunction;
class Context;
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
class IFunction;
2018-02-02 08:33:36 +00:00
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
2021-06-22 10:28:56 +00:00
struct ActionDAGNodes;
/** A field, that can be stored in two representations:
* - A standalone field.
2020-04-08 02:56:25 +00:00
* - A field with reference to its position in a block.
* It's needed for execution of functions on ranges during
* index analysis. If function was executed once for field,
2020-04-08 02:56:25 +00:00
* its result would be cached for whole block for which field's reference points to.
*/
struct FieldRef : public Field
{
FieldRef() = default;
/// Create as explicit field without block.
template <typename T>
FieldRef(T && value) : Field(std::forward<T>(value)) {} /// NOLINT
/// Create as reference to field in block.
FieldRef(ColumnsWithTypeAndName * columns_, size_t row_idx_, size_t column_idx_)
: Field((*(*columns_)[column_idx_].column)[row_idx_]),
columns(columns_), row_idx(row_idx_), column_idx(column_idx_) {}
bool isExplicit() const { return columns == nullptr; }
ColumnsWithTypeAndName * columns = nullptr;
2020-04-06 13:35:11 +00:00
size_t row_idx = 0;
size_t column_idx = 0;
};
2017-08-29 18:51:48 +00:00
/** Range with open or closed ends; possibly unbounded.
*/
2012-12-05 12:44:55 +00:00
struct Range
{
private:
static bool equals(const Field & lhs, const Field & rhs);
static bool less(const Field & lhs, const Field & rhs);
public:
FieldRef left = NEGATIVE_INFINITY; /// the left border
FieldRef right = POSITIVE_INFINITY; /// the right border
bool left_included = false; /// includes the left border
bool right_included = false; /// includes the right border
/// The whole universe (not null).
Range() {} /// NOLINT
2017-04-16 15:00:33 +00:00
/// One point.
Range(const FieldRef & point) /// NOLINT
: left(point), right(point), left_included(true), right_included(true) {}
2017-04-16 15:00:33 +00:00
/// A bounded two-sided range.
Range(const FieldRef & left_, bool left_included_, const FieldRef & right_, bool right_included_)
: left(left_)
, right(right_)
, left_included(left_included_)
, right_included(right_included_)
{
shrinkToIncludedIfPossible();
}
static Range createRightBounded(const FieldRef & right_point, bool right_included)
2012-12-05 12:44:55 +00:00
{
Range r;
r.right = right_point;
r.right_included = right_included;
r.shrinkToIncludedIfPossible();
// Special case for [-Inf, -Inf]
if (r.right.isNegativeInfinity() && right_included)
r.left_included = true;
2012-12-05 12:44:55 +00:00
return r;
}
static Range createLeftBounded(const FieldRef & left_point, bool left_included)
2012-12-05 12:44:55 +00:00
{
Range r;
r.left = left_point;
2012-12-05 12:44:55 +00:00
r.left_included = left_included;
r.shrinkToIncludedIfPossible();
// Special case for [+Inf, +Inf]
if (r.left.isPositiveInfinity() && left_included)
r.right_included = true;
2012-12-05 12:44:55 +00:00
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].
*/
void shrinkToIncludedIfPossible()
2012-12-05 12:44:55 +00:00
{
if (left.isExplicit() && !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.isExplicit() && !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;
}
}
2012-12-05 12:44:55 +00:00
}
bool empty() const { return less(right, left) || ((!left_included || !right_included) && !less(left, right)); }
2017-04-16 15:00:33 +00:00
/// x contained in the range
bool contains(const FieldRef & x) const
2012-12-05 12:44:55 +00:00
{
return !leftThan(x) && !rightThan(x);
}
2017-04-16 15:00:33 +00:00
/// x is to the left
bool rightThan(const FieldRef & x) const
2012-12-05 12:44:55 +00:00
{
return less(left, x) || (left_included && equals(x, left));
2012-12-05 12:44:55 +00:00
}
2017-04-16 15:00:33 +00:00
/// x is to the right
bool leftThan(const FieldRef & x) const
2012-12-05 12:44:55 +00:00
{
return less(x, right) || (right_included && equals(x, right));
2012-12-05 12:44:55 +00:00
}
bool intersectsRange(const Range & r) const
2012-12-05 12:44:55 +00:00
{
2017-04-16 15:00:33 +00:00
/// r to the left of me.
if (less(r.right, left) || ((!left_included || !r.right_included) && equals(r.right, left)))
2012-12-05 12:44:55 +00:00
return false;
2017-04-16 15:00:33 +00:00
/// r to the right of me.
if (less(right, r.left) || ((!right_included || !r.left_included) && equals(r.left, right)))
2012-12-05 12:44:55 +00:00
return false;
2012-12-05 12:44:55 +00:00
return true;
}
bool containsRange(const Range & r) const
2012-12-05 12:44:55 +00:00
{
2017-04-16 15:00:33 +00:00
/// r starts to the left of me.
if (less(r.left, left) || (r.left_included && !left_included && equals(r.left, left)))
2012-12-05 12:44:55 +00:00
return false;
2017-04-16 15:00:33 +00:00
/// r ends right of me.
if (less(right, r.right) || (r.right_included && !right_included && equals(r.right, right)))
2012-12-05 12:44:55 +00:00
return false;
2012-12-05 12:44:55 +00:00
return true;
}
void invert()
{
std::swap(left, right);
if (left.isPositiveInfinity())
left = NEGATIVE_INFINITY;
if (right.isNegativeInfinity())
right = POSITIVE_INFINITY;
std::swap(left_included, right_included);
}
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
String toString() const;
2012-12-05 12:44:55 +00:00
};
2017-04-16 15:00:33 +00:00
/** Condition on the index.
*
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.
*
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.
*/
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.
KeyCondition(
2021-06-21 16:17:05 +00:00
const ASTPtr & query,
2022-07-28 09:40:09 +00:00
const ASTs & additional_filter_asts,
2021-06-21 16:17:05 +00:00
TreeRewriterResultPtr syntax_analyzer_result,
PreparedSetsPtr prepared_sets_,
2021-06-21 16:17:05 +00:00
ContextPtr context,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr,
bool single_point_ = false,
bool strict_ = false);
2022-08-10 13:43:55 +00:00
KeyCondition(
const SelectQueryInfo & query_info,
ContextPtr context,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr_,
bool single_point_ = false,
bool strict_ = false)
2022-07-28 09:40:09 +00:00
: KeyCondition(
query_info.query,
query_info.filter_asts,
query_info.syntax_analyzer_result,
query_info.prepared_sets,
context,
key_column_names,
key_expr_,
single_point_,
strict_)
{
}
2022-08-10 13:43:55 +00:00
2021-06-21 16:17:05 +00:00
KeyCondition(
2021-06-22 10:28:56 +00:00
ActionDAGNodes dag_nodes,
2021-06-22 14:45:22 +00:00
TreeRewriterResultPtr syntax_analyzer_result,
PreparedSetsPtr prepared_sets_,
ContextPtr context,
2018-06-30 21:35:01 +00:00
const Names & key_column_names,
2020-09-21 10:13:01 +00:00
const ExpressionActionsPtr & key_expr,
bool single_point_ = false,
bool strict_ = false);
2020-03-10 14:56:55 +00:00
/// Whether the condition and its negation are feasible in the direct product of single column ranges specified by `hyperrectangle`.
BoolMask checkInHyperrectangle(
const std::vector<Range> & hyperrectangle,
const DataTypes & data_types) const;
/// Whether the condition and its negation are (independently) feasible in the key range.
/// 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.
/// Argument initial_mask is used for early exiting the implementation when we do not care about
/// one of the resulting mask components (see BoolMask::consider_only_can_be_XXX).
BoolMask checkInRange(
size_t used_key_size,
const FieldRef * left_keys,
const FieldRef * right_keys,
const DataTypes & data_types,
BoolMask initial_mask = BoolMask(false, false)) const;
/// Same as checkInRange, but calculate only may_be_true component of a result.
/// This is more efficient than checkInRange(...).can_be_true.
bool mayBeTrueInRange(
size_t used_key_size,
const FieldRef * left_keys,
const FieldRef * right_keys,
const DataTypes & data_types) const;
/// Checks that the index can not be used
/// FUNCTION_UNKNOWN will be AND'ed (if any).
bool alwaysUnknownOrTrue() const;
/// Checks that the index can not be used
/// Does not allow any FUNCTION_UNKNOWN (will instantly return true).
bool anyUnknownOrAlwaysTrue() const;
bool alwaysFalse() const;
/// Get the maximum number of the key element used in the condition.
size_t getMaxKeyColumn() const;
bool hasMonotonicFunctionsChain() const;
/// 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 key.
bool addCondition(const String & column, const Range & range);
2015-03-27 03:37:46 +00:00
String toString() const;
2021-04-15 17:30:04 +00:00
/// Condition description for EXPLAIN query.
struct Description
{
2021-04-16 09:42:23 +00:00
/// Which columns from PK were used, in PK order.
2021-04-15 17:30:04 +00:00
std::vector<std::string> used_keys;
2021-04-16 09:42:23 +00:00
/// Condition which was applied, mostly human-readable.
2021-04-15 17:30:04 +00:00
std::string condition;
};
Description getDescription() const;
/** 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);
static Block getBlockWithConstants(
const ASTPtr & query, const TreeRewriterResultPtr & syntax_analyzer_result, ContextPtr context);
static std::optional<Range> applyMonotonicFunctionsChainToRange(
Range key_range,
2020-07-21 11:02:58 +00:00
const MonotonicFunctionsChain & functions,
2020-09-21 10:13:01 +00:00
DataTypePtr current_type,
bool single_point = false);
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
bool matchesExactContinuousRange() const;
private:
2017-04-16 15:00:33 +00:00
/// The expression is stored as Reverse Polish Notation.
2012-12-05 12:44:55 +00:00
struct RPNElement
{
enum Function
{
2017-04-16 15:00:33 +00:00
/// Atoms of a Boolean expression.
2012-12-05 12:44:55 +00:00
FUNCTION_IN_RANGE,
FUNCTION_NOT_IN_RANGE,
2014-03-20 12:25:26 +00:00
FUNCTION_IN_SET,
2014-03-26 10:56:21 +00:00
FUNCTION_NOT_IN_SET,
FUNCTION_IS_NULL,
FUNCTION_IS_NOT_NULL,
2017-04-16 15:00:33 +00:00
FUNCTION_UNKNOWN, /// Can take any value.
/// Operators of the logical expression.
2012-12-05 12:44:55 +00:00
FUNCTION_NOT,
FUNCTION_AND,
FUNCTION_OR,
2017-04-16 15:00:33 +00:00
/// Constants
ALWAYS_FALSE,
ALWAYS_TRUE,
2012-12-05 12:44:55 +00:00
};
RPNElement() = default;
RPNElement(Function function_) : function(function_) {} /// NOLINT
2012-12-05 12:44:55 +00:00
RPNElement(Function function_, size_t key_column_) : function(function_), key_column(key_column_) {}
RPNElement(Function function_, size_t key_column_, const Range & range_)
2014-04-22 22:52:00 +00:00
: function(function_), range(range_), key_column(key_column_) {}
2015-03-27 03:37:46 +00:00
String toString() const;
String toString(std::string_view column_name, bool print_constants) const;
Function function = FUNCTION_UNKNOWN;
2017-04-16 15:00:33 +00:00
/// For FUNCTION_IN_RANGE and FUNCTION_NOT_IN_RANGE.
2012-12-05 12:44:55 +00:00
Range range;
size_t key_column = 0;
2017-04-16 15:00:33 +00:00
/// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET
using MergeTreeSetIndexPtr = std::shared_ptr<const MergeTreeSetIndex>;
MergeTreeSetIndexPtr set_index;
2020-07-21 11:02:58 +00:00
MonotonicFunctionsChain monotonic_functions_chain;
2012-12-05 12:44:55 +00:00
};
using RPN = std::vector<RPNElement>;
using ColumnIndices = std::map<String, size_t>;
using AtomMap = std::unordered_map<std::string, bool(*)(RPNElement & out, const Field & value)>;
public:
static const AtomMap atom_map;
2021-06-02 16:56:24 +00:00
class Tree;
class FunctionTree;
2018-06-30 21:35:01 +00:00
private:
BoolMask checkInRange(
size_t used_key_size,
const FieldRef * left_key,
const FieldRef * right_key,
const DataTypes & data_types,
bool right_bounded,
BoolMask initial_mask) const;
2021-06-02 16:56:24 +00:00
void traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants);
bool tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out);
static bool tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out);
/** 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?
* If these conditions are true, then returns number of column in key, type of resulting expression
* and fills chain of possibly-monotonic functions.
*/
bool isKeyPossiblyWrappedByMonotonicFunctions(
2021-06-02 16:56:24 +00:00
const Tree & node,
ContextPtr context,
size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type,
MonotonicFunctionsChain & out_functions_chain);
bool isKeyPossiblyWrappedByMonotonicFunctionsImpl(
2021-06-02 16:56:24 +00:00
const Tree & node,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
2021-06-02 16:56:24 +00:00
std::vector<FunctionTree> & out_functions_chain);
2021-08-28 15:33:18 +00:00
bool transformConstantWithValidFunctions(
const String & expr_name,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
DataTypePtr & out_type,
std::function<bool(IFunctionBase &, const IDataType &)> always_monotonic) const;
bool canConstantBeWrappedByMonotonicFunctions(
2021-06-21 16:17:05 +00:00
const Tree & node,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
DataTypePtr & out_type);
2020-11-06 03:50:58 +00:00
bool canConstantBeWrappedByFunctions(
2021-06-21 16:17:05 +00:00
const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type);
2020-11-06 03:50:58 +00:00
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(
2021-06-03 12:26:02 +00:00
const FunctionTree & func,
ContextPtr context,
RPNElement & out,
size_t & out_key_column_num);
/// Checks that the index can not be used.
///
/// If unknown_any is false (used by alwaysUnknownOrTrue()), then FUNCTION_UNKNOWN can be AND'ed,
/// otherwise (anyUnknownOrAlwaysTrue()) first FUNCTION_UNKNOWN will return true (index cannot be used).
///
/// Consider the following example:
///
/// CREATE TABLE test(p DateTime, k int) ENGINE MergeTree PARTITION BY toDate(p) ORDER BY k;
/// INSERT INTO test 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 test WHERE toDate(p) >= '2020-09-01' AND p <= '2020-09-01 00:00:00'
/// In this case rpn will be (FUNCTION_IN_RANGE, FUNCTION_UNKNOWN (due to strict), FUNCTION_AND)
/// and for optimize_trivial_count_query we cannot use index if there is at least one FUNCTION_UNKNOWN.
/// since there is no post processing and return count() based on only the first predicate is wrong.
///
/// - SELECT * FROM test WHERE toDate(p) >= '2020-09-01' AND p <= '2020-09-01 00:00:00'
/// In this case will be (FUNCTION_IN_RANGE, FUNCTION_IN_RANGE (due to non-strict), FUNCTION_AND)
/// so it will prune everything out and nothing will be read.
///
/// - SELECT * FROM test WHERE toDate(p) >= '2020-09-01' AND toUnixTimestamp(p)%5==0
/// In this case will be (FUNCTION_IN_RANGE, FUNCTION_UNKNOWN, FUNCTION_AND)
/// and all, two, partitions will be scanned, but due to filtering later none of rows will be matched.
bool unknownOrAlwaysTrue(bool unknown_any) const;
2012-12-05 12:44:55 +00:00
RPN rpn;
ColumnIndices key_columns;
2021-06-15 13:47:37 +00:00
/// Expression which is used for key condition.
const ExpressionActionsPtr key_expr;
/// All intermediate columns are used to calculate key_expr.
const NameSet key_subexpr_names;
NameSet array_joined_columns;
2022-07-18 15:53:30 +00:00
PreparedSetsPtr prepared_sets;
2020-09-21 10:13:01 +00:00
// If true, always allow key_expr to be wrapped by function
bool single_point;
// If true, do not use always_monotonic information to transform constants
bool strict;
2012-12-05 12:44:55 +00:00
};
2021-12-08 02:40:59 +00:00
String extractFixedPrefixFromLikePattern(const String & like_pattern);
2012-12-05 12:44:55 +00:00
}