2012-08-22 20:29:01 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
|
#include <Poco/RWLock.h>
|
2016-12-21 12:10:31 +00:00
|
|
|
|
#include <DB/Columns/ColumnArray.h>
|
|
|
|
|
#include <DB/Columns/ColumnConst.h>
|
2012-08-23 23:49:28 +00:00
|
|
|
|
#include <DB/DataStreams/IBlockInputStream.h>
|
2013-06-20 12:12:27 +00:00
|
|
|
|
#include <DB/Interpreters/Limits.h>
|
2016-12-21 12:10:31 +00:00
|
|
|
|
#include <DB/Interpreters/SetVariants.h>
|
|
|
|
|
#include <DB/Parsers/IAST.h>
|
2014-03-26 18:19:25 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/BoolMask.h>
|
2012-08-22 20:29:01 +00:00
|
|
|
|
|
2016-12-21 12:10:31 +00:00
|
|
|
|
#include <common/logger_useful.h>
|
2015-03-02 01:11:37 +00:00
|
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
|
|
2016-12-21 12:10:31 +00:00
|
|
|
|
namespace DB
|
2015-03-02 01:11:37 +00:00
|
|
|
|
{
|
|
|
|
|
|
2017-01-14 09:11:11 +00:00
|
|
|
|
struct Range;
|
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:
|
2014-07-06 19:48:39 +00:00
|
|
|
|
Set(const Limits & limits) :
|
2014-05-15 10:24:03 +00:00
|
|
|
|
log(&Logger::get("Set")),
|
2013-06-20 12:12:27 +00:00
|
|
|
|
max_rows(limits.max_rows_in_set),
|
|
|
|
|
max_bytes(limits.max_bytes_in_set),
|
|
|
|
|
overflow_mode(limits.set_overflow_mode)
|
|
|
|
|
{
|
|
|
|
|
}
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
|
bool empty() const { return data.empty(); }
|
2012-08-23 20:22:44 +00:00
|
|
|
|
|
2017-03-03 21:15:46 +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.
|
2012-08-24 19:42:03 +00:00
|
|
|
|
*/
|
2016-11-24 12:26:47 +00:00
|
|
|
|
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set);
|
2014-03-04 11:26:55 +00:00
|
|
|
|
|
2017-03-03 21:15:46 +00:00
|
|
|
|
// Returns false, if some limit was exceeded and no need to insert more data.
|
2015-01-27 00:52:03 +00:00
|
|
|
|
bool insertFromBlock(const Block & block, bool create_ordered_set = false);
|
2014-03-04 11:26:55 +00:00
|
|
|
|
|
2017-03-03 21:15:46 +00:00
|
|
|
|
/** For columns of 'block', check belonging of corresponding rows to the set.
|
|
|
|
|
* Return UInt8 column with the result.
|
2012-08-23 20:22:44 +00:00
|
|
|
|
*/
|
2015-10-08 03:41:11 +00:00
|
|
|
|
ColumnPtr execute(const Block & block, bool negative) const;
|
2014-03-26 10:56:21 +00:00
|
|
|
|
|
2015-10-12 07:05:54 +00:00
|
|
|
|
std::string describe() const;
|
2014-04-01 10:09:22 +00:00
|
|
|
|
|
2017-03-03 21:15:46 +00:00
|
|
|
|
/// Check, if the Set could possibly contain elements for specified range.
|
2015-03-27 03:37:46 +00:00
|
|
|
|
BoolMask mayBeTrueInRange(const Range & range) const;
|
2014-06-18 20:08:31 +00:00
|
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
|
size_t getTotalRowCount() const { return data.getTotalRowCount(); }
|
|
|
|
|
size_t getTotalByteCount() const { return data.getTotalByteCount(); }
|
2014-06-18 20:08:31 +00:00
|
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
|
private:
|
2013-03-25 13:02:12 +00:00
|
|
|
|
Sizes key_sizes;
|
2012-08-23 22:40:51 +00:00
|
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
|
SetVariants data;
|
|
|
|
|
|
2012-08-23 22:40:51 +00:00
|
|
|
|
/** Типы данных, из которых было создано множество.
|
|
|
|
|
* При проверке на принадлежность множеству, типы проверяемых столбцов должны с ними совпадать.
|
|
|
|
|
*/
|
|
|
|
|
DataTypes data_types;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
|
Logger * log;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2013-06-20 12:12:27 +00:00
|
|
|
|
/// Ограничения на максимальный размер множества
|
|
|
|
|
size_t max_rows;
|
|
|
|
|
size_t max_bytes;
|
2014-02-17 23:56:45 +00:00
|
|
|
|
OverflowMode overflow_mode;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2013-03-19 12:25:59 +00:00
|
|
|
|
/// Если в левой части IN стоит массив. Проверяем, что хоть один элемент массива лежит в множестве.
|
2013-03-25 13:02:12 +00:00
|
|
|
|
void executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2013-03-19 12:25:59 +00:00
|
|
|
|
/// Если в левой части набор столбцов тех же типов, что элементы множества.
|
|
|
|
|
void executeOrdinary(const ConstColumnPlainPtrs & key_columns, ColumnUInt8::Container_t & vec_res, bool negative) const;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2013-06-20 12:12:27 +00:00
|
|
|
|
/// Проверить не превышены ли допустимые размеры множества ключей
|
|
|
|
|
bool checkSetSizeLimits() const;
|
2014-05-15 10:24:03 +00:00
|
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
|
/// Вектор упорядоченных элементов Set.
|
|
|
|
|
/// Нужен для работы индекса по первичному ключу в операторе IN.
|
2016-05-28 10:35:44 +00:00
|
|
|
|
using OrderedSetElements = std::vector<Field>;
|
|
|
|
|
using OrderedSetElementsPtr = std::unique_ptr<OrderedSetElements>;
|
2014-04-08 12:54:32 +00:00
|
|
|
|
OrderedSetElementsPtr ordered_set_elements;
|
2015-01-27 00:52:03 +00:00
|
|
|
|
|
|
|
|
|
/** Защищает работу с множеством в функциях insertFromBlock и execute.
|
|
|
|
|
* Эти функции могут вызываться одновременно из разных потоков только при использовании StorageSet,
|
|
|
|
|
* и StorageSet вызывает только эти две функции.
|
|
|
|
|
* Поэтому остальные функции по работе с множеством, не защинены.
|
|
|
|
|
*/
|
|
|
|
|
mutable Poco::RWLock rwlock;
|
2015-03-02 01:11:37 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Method>
|
2015-03-02 05:41:21 +00:00
|
|
|
|
void insertFromBlockImpl(
|
2015-03-02 01:11:37 +00:00
|
|
|
|
Method & method,
|
|
|
|
|
const ConstColumnPlainPtrs & key_columns,
|
|
|
|
|
size_t rows,
|
|
|
|
|
SetVariants & variants);
|
|
|
|
|
|
|
|
|
|
template <typename Method>
|
|
|
|
|
void executeImpl(
|
|
|
|
|
Method & method,
|
|
|
|
|
const ConstColumnPlainPtrs & key_columns,
|
|
|
|
|
ColumnUInt8::Container_t & vec_res,
|
|
|
|
|
bool negative,
|
2015-03-03 20:00:39 +00:00
|
|
|
|
size_t rows) const;
|
2015-03-02 01:11:37 +00:00
|
|
|
|
|
|
|
|
|
template <typename Method>
|
|
|
|
|
void executeArrayImpl(
|
|
|
|
|
Method & method,
|
|
|
|
|
const ConstColumnPlainPtrs & key_columns,
|
|
|
|
|
const ColumnArray::Offsets_t & offsets,
|
|
|
|
|
ColumnUInt8::Container_t & vec_res,
|
|
|
|
|
bool negative,
|
2015-03-03 20:00:39 +00:00
|
|
|
|
size_t rows) const;
|
2012-08-22 20:29:01 +00:00
|
|
|
|
};
|
|
|
|
|
|
2016-05-28 10:35:44 +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
|
|
|
|
|
|
|
|
|
}
|