ClickHouse/dbms/src/Interpreters/Set.h

166 lines
5.5 KiB
C++
Raw Normal View History

2012-08-22 20:29:01 +00:00
#pragma once
#include <shared_mutex>
#include <Columns/ColumnArray.h>
#include <DataStreams/IBlockInputStream.h>
#include <Interpreters/Limits.h>
#include <Interpreters/SetVariants.h>
#include <Parsers/IAST.h>
#include <Storages/MergeTree/BoolMask.h>
2012-08-22 20:29:01 +00:00
#include <common/logger_useful.h>
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
namespace DB
{
2017-01-14 09:11:11 +00:00
struct Range;
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
2017-03-03 21:15:46 +00:00
/** Data structure for implementation of IN expression.
2012-08-22 20:29:01 +00:00
*/
2012-08-23 22:40:51 +00:00
class Set
2012-08-22 20:29:01 +00:00
{
2012-08-23 22:40:51 +00:00
public:
Set(const Limits & limits) :
log(&Logger::get("Set")),
max_rows(limits.max_rows_in_set),
max_bytes(limits.max_bytes_in_set),
overflow_mode(limits.set_overflow_mode)
{
}
bool empty() const { return data.empty(); }
2012-08-23 20:22:44 +00:00
/** Create a Set from expression (specified literally in the query).
* 'types' - types of what are on the left hand side of IN.
* 'node' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
* 'create_ordered_set' - if true, create ordered vector of elements. For primary key to work.
*/
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set);
// Returns false, if some limit was exceeded and no need to insert more data.
bool insertFromBlock(const Block & block, bool create_ordered_set = false);
/** For columns of 'block', check belonging of corresponding rows to the set.
* Return UInt8 column with the result.
*/
ColumnPtr execute(const Block & block, bool negative) const;
2014-03-26 10:56:21 +00:00
std::string describe() const;
2014-04-01 10:09:22 +00:00
/// Check, if the Set could possibly contain elements for specified range.
BoolMask mayBeTrueInRange(const Range & range) const;
size_t getTotalRowCount() const { return data.getTotalRowCount(); }
size_t getTotalByteCount() const { return data.getTotalByteCount(); }
2012-08-23 20:22:44 +00:00
private:
Sizes key_sizes;
SetVariants data;
/** How IN works with Nullable types.
*
* For simplicity reasons, all NULL values and any tuples with at least one NULL element are ignored in the Set.
* And for left hand side values, that are NULLs or contain any NULLs, we return 0 (means that element is not in Set).
*
* If we want more standard compliant behaviour, we must return NULL
* if lhs is NULL and set is not empty or if lhs is not in set, but set contains at least one NULL.
* It is more complicated with tuples.
* For example,
* (1, NULL, 2) IN ((1, NULL, 3)) must return 0,
* but (1, NULL, 2) IN ((1, 1111, 2)) must return NULL.
*
* We have not implemented such sophisticated behaviour.
*/
2017-06-02 21:37:28 +00:00
/** The data types from which the set was created.
* When checking for belonging to a set, the types of columns to be checked must match with them.
*/
DataTypes data_types;
Logger * log;
2017-06-02 21:37:28 +00:00
/// Limitations on the maximum size of the set
size_t max_rows;
size_t max_bytes;
OverflowMode overflow_mode;
2017-06-02 21:37:28 +00:00
/// If there is an array on the left side of IN. We check that at least one element of the array presents in the set.
void executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const;
2017-06-02 21:37:28 +00:00
/// If in the left part columns contains the same types as the elements of the set.
void executeOrdinary(
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
const PaddedPODArray<UInt8> * null_map) const;
2017-06-02 21:37:28 +00:00
/// Check whether the permissible sizes of keys set reached
bool checkSetSizeLimits() const;
2017-06-02 21:37:28 +00:00
/// Vector of ordered elements of `Set`.
/// It is necessary for the index to work on the primary key in the IN statement.
using OrderedSetElements = std::vector<Field>;
using OrderedSetElementsPtr = std::unique_ptr<OrderedSetElements>;
OrderedSetElementsPtr ordered_set_elements;
2017-06-02 21:37:28 +00:00
/** Protects work with the set in the functions `insertFromBlock` and `execute`.
* These functions can be called simultaneously from different threads only when using StorageSet,
* and StorageSet calls only these two functions.
* Therefore, the rest of the functions for working with set are not protected.
*/
mutable std::shared_mutex rwlock;
template <typename Method>
void insertFromBlockImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
size_t rows,
SetVariants & variants,
ConstNullMapPtr null_map);
template <typename Method, bool has_null_map>
void insertFromBlockImplCase(
Method & method,
const ConstColumnPlainPtrs & key_columns,
size_t rows,
SetVariants & variants,
ConstNullMapPtr null_map);
template <typename Method>
void executeImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows,
ConstNullMapPtr null_map) const;
template <typename Method, bool has_null_map>
void executeImplCase(
Method & method,
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows,
ConstNullMapPtr null_map) const;
template <typename Method>
void executeArrayImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
const ColumnArray::Offsets_t & offsets,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const;
2012-08-22 20:29:01 +00:00
};
using SetPtr = std::shared_ptr<Set>;
using ConstSetPtr = std::shared_ptr<const Set>;
using Sets = std::vector<SetPtr>;
2012-08-22 20:29:01 +00:00
}