2012-08-22 20:29:01 +00:00
|
|
|
#pragma once
|
|
|
|
|
2018-03-11 00:15:26 +00:00
|
|
|
#include <Core/Block.h>
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <QueryPipeline/SizeLimits.h>
|
2018-03-11 00:15:26 +00:00
|
|
|
#include <DataTypes/IDataType.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/SetVariants.h>
|
2023-06-22 14:23:04 +00:00
|
|
|
#include <Interpreters/SetKeys.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/IAST.h>
|
|
|
|
#include <Storages/MergeTree/BoolMask.h>
|
2012-08-22 20:29:01 +00:00
|
|
|
|
2023-01-12 15:51:04 +00:00
|
|
|
#include <Common/SharedMutex.h>
|
2023-10-23 11:31:44 +00:00
|
|
|
#include <Interpreters/castColumn.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
|
|
|
|
2020-05-20 20:16:32 +00:00
|
|
|
class Context;
|
2018-02-08 17:46:22 +00:00
|
|
|
class IFunctionBase;
|
2022-11-30 18:48:09 +00:00
|
|
|
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2022-07-19 17:43:18 +00:00
|
|
|
class Chunk;
|
2018-06-30 21:35:01 +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:
|
2018-07-02 18:57:14 +00:00
|
|
|
/// 'fill_set_elements': in addition to hash table
|
|
|
|
/// (that is useful only for checking that some value is in the set and may not store the original values),
|
|
|
|
/// store all set elements in explicit form.
|
|
|
|
/// This is needed for subsequent use for index.
|
2023-06-22 14:23:04 +00:00
|
|
|
Set(const SizeLimits & limits_, size_t max_elements_to_fill_, bool transform_null_in_)
|
2024-01-23 17:04:50 +00:00
|
|
|
: log(getLogger("Set")),
|
2023-10-23 11:31:44 +00:00
|
|
|
limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_),
|
|
|
|
cast_cache(std::make_unique<InternalCastFunctionCache>())
|
|
|
|
{}
|
2014-07-06 19:48:39 +00:00
|
|
|
|
2018-04-19 21:34:04 +00:00
|
|
|
/** Set can be created either from AST or from a stream of data (subquery result).
|
|
|
|
*/
|
|
|
|
|
|
|
|
/** Create a Set from stream.
|
|
|
|
* Call setHeader, then call insertFromBlock for each block.
|
2018-01-21 07:30:07 +00:00
|
|
|
*/
|
2021-08-18 10:30:02 +00:00
|
|
|
void setHeader(const ColumnsWithTypeAndName & header);
|
2018-04-19 21:34:04 +00:00
|
|
|
|
|
|
|
/// Returns false, if some limit was exceeded and no need to insert more data.
|
2023-05-25 13:33:52 +00:00
|
|
|
bool insertFromColumns(const Columns & columns);
|
2021-08-18 10:30:02 +00:00
|
|
|
bool insertFromBlock(const ColumnsWithTypeAndName & columns);
|
2022-07-19 17:43:18 +00:00
|
|
|
|
2023-06-22 14:23:04 +00:00
|
|
|
void fillSetElements();
|
2023-05-25 13:33:52 +00:00
|
|
|
bool insertFromColumns(const Columns & columns, SetKeyColumns & holder);
|
|
|
|
void appendSetElements(SetKeyColumns & holder);
|
|
|
|
|
2019-11-01 10:58:29 +00:00
|
|
|
/// Call after all blocks were inserted. To get the information that set is already created.
|
2023-04-11 21:19:44 +00:00
|
|
|
void finishInsert() { is_created = true; }
|
2019-11-01 10:58:29 +00:00
|
|
|
|
2022-07-19 17:43:18 +00:00
|
|
|
/// finishInsert and isCreated are thread-safe
|
|
|
|
bool isCreated() const { return is_created.load(); }
|
2014-03-04 11:26:55 +00:00
|
|
|
|
2023-04-05 11:05:36 +00:00
|
|
|
void checkIsCreated() const;
|
2023-04-04 10:01:01 +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
|
|
|
*/
|
2021-08-18 10:30:02 +00:00
|
|
|
ColumnPtr execute(const ColumnsWithTypeAndName & columns, bool negative) const;
|
2014-03-26 10:56:21 +00:00
|
|
|
|
2023-08-03 19:18:59 +00:00
|
|
|
bool hasNull() const;
|
|
|
|
|
2020-12-23 02:10:37 +00:00
|
|
|
bool empty() const;
|
|
|
|
size_t getTotalRowCount() const;
|
|
|
|
size_t getTotalByteCount() const;
|
2018-04-18 19:38:40 +00:00
|
|
|
|
|
|
|
const DataTypes & getDataTypes() const { return data_types; }
|
2019-10-31 16:14:06 +00:00
|
|
|
const DataTypes & getElementsTypes() const { return set_elements_types; }
|
2018-04-18 19:38:40 +00:00
|
|
|
|
2023-05-25 13:33:52 +00:00
|
|
|
bool hasExplicitSetElements() const { return fill_set_elements || (!set_elements.empty() && set_elements.front()->size() == data.getTotalRowCount()); }
|
2023-12-06 12:53:14 +00:00
|
|
|
bool hasSetElements() const { return !set_elements.empty(); }
|
2023-04-05 11:05:36 +00:00
|
|
|
Columns getSetElements() const { checkIsCreated(); return { set_elements.begin(), set_elements.end() }; }
|
2014-06-18 20:08:31 +00:00
|
|
|
|
2019-08-19 16:51:50 +00:00
|
|
|
void checkColumnsNumber(size_t num_key_columns) const;
|
2020-07-01 14:05:54 +00:00
|
|
|
bool areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const;
|
2019-08-19 16:51:50 +00:00
|
|
|
void checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const;
|
|
|
|
|
2023-06-20 14:53:13 +00:00
|
|
|
static DataTypes getElementTypes(DataTypes types, bool transform_null_in);
|
2023-06-01 21:15:15 +00:00
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
private:
|
2018-06-04 15:45:08 +00:00
|
|
|
size_t keys_size = 0;
|
2013-03-25 13:02:12 +00:00
|
|
|
Sizes key_sizes;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
SetVariants data;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-28 03:00:33 +00:00
|
|
|
/** 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-04-01 07:20:54 +00:00
|
|
|
|
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.
|
2012-08-23 22:40:51 +00:00
|
|
|
*/
|
|
|
|
DataTypes data_types;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-10-31 16:14:06 +00:00
|
|
|
/// Types for set_elements.
|
|
|
|
DataTypes set_elements_types;
|
|
|
|
|
2024-01-23 17:04:50 +00:00
|
|
|
LoggerPtr log;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Limitations on the maximum size of the set
|
2018-03-11 00:15:26 +00:00
|
|
|
SizeLimits limits;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-07-02 18:57:14 +00:00
|
|
|
/// Do we need to additionally store all elements of the set in explicit form for subsequent use for index.
|
2023-06-22 14:23:04 +00:00
|
|
|
bool fill_set_elements = false;
|
2023-06-01 21:15:15 +00:00
|
|
|
size_t max_elements_to_fill;
|
2018-07-02 18:57:14 +00:00
|
|
|
|
2020-11-05 18:07:44 +00:00
|
|
|
/// If true, insert NULL values to set.
|
2020-04-09 08:27:55 +00:00
|
|
|
bool transform_null_in;
|
2020-04-06 13:30:16 +00:00
|
|
|
|
2019-11-01 10:58:29 +00:00
|
|
|
/// Check if set contains all the data.
|
2022-07-19 17:43:18 +00:00
|
|
|
std::atomic<bool> is_created = false;
|
2019-11-01 10:58:29 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// If in the left part columns contains the same types as the elements of the set.
|
2017-03-28 03:00:33 +00:00
|
|
|
void executeOrdinary(
|
2017-12-13 01:27:53 +00:00
|
|
|
const ColumnRawPtrs & key_columns,
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnUInt8::Container & vec_res,
|
2017-03-28 03:00:33 +00:00
|
|
|
bool negative,
|
|
|
|
const PaddedPODArray<UInt8> * null_map) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
/// Collected elements of `Set`.
|
2017-06-02 21:37:28 +00:00
|
|
|
/// It is necessary for the index to work on the primary key in the IN statement.
|
2019-03-29 14:17:29 +00:00
|
|
|
std::vector<IColumn::WrappedPtr> set_elements;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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,
|
2015-01-27 00:52:03 +00:00
|
|
|
*/
|
2023-01-12 15:51:04 +00:00
|
|
|
mutable SharedMutex rwlock;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-10-23 11:31:44 +00:00
|
|
|
/// A cache for cast functions (if any) to avoid rebuilding cast functions
|
|
|
|
/// for every call to `execute`
|
|
|
|
mutable std::unique_ptr<InternalCastFunctionCache> cast_cache;
|
|
|
|
|
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,
|
2017-12-13 01:27:53 +00:00
|
|
|
const ColumnRawPtrs & key_columns,
|
2015-03-02 01:11:37 +00:00
|
|
|
size_t rows,
|
2017-03-28 03:00:33 +00:00
|
|
|
SetVariants & variants,
|
2018-06-30 21:35:01 +00:00
|
|
|
ConstNullMapPtr null_map,
|
|
|
|
ColumnUInt8::Container * out_filter);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
template <typename Method, bool has_null_map, bool build_filter>
|
2017-03-28 03:00:33 +00:00
|
|
|
void insertFromBlockImplCase(
|
|
|
|
Method & method,
|
2017-12-13 01:27:53 +00:00
|
|
|
const ColumnRawPtrs & key_columns,
|
2017-03-28 03:00:33 +00:00
|
|
|
size_t rows,
|
|
|
|
SetVariants & variants,
|
2018-06-30 21:35:01 +00:00
|
|
|
ConstNullMapPtr null_map,
|
|
|
|
ColumnUInt8::Container * out_filter);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
template <typename Method>
|
|
|
|
void executeImpl(
|
|
|
|
Method & method,
|
2017-12-13 01:27:53 +00:00
|
|
|
const ColumnRawPtrs & key_columns,
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnUInt8::Container & vec_res,
|
2015-03-02 01:11:37 +00:00
|
|
|
bool negative,
|
2017-03-28 03:00:33 +00:00
|
|
|
size_t rows,
|
|
|
|
ConstNullMapPtr null_map) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-28 03:00:33 +00:00
|
|
|
template <typename Method, bool has_null_map>
|
|
|
|
void executeImplCase(
|
|
|
|
Method & method,
|
2017-12-13 01:27:53 +00:00
|
|
|
const ColumnRawPtrs & key_columns,
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnUInt8::Container & vec_res,
|
2017-03-28 03:00:33 +00:00
|
|
|
bool negative,
|
|
|
|
size_t rows,
|
|
|
|
ConstNullMapPtr null_map) 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
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
class IFunction;
|
|
|
|
using FunctionPtr = std::shared_ptr<IFunction>;
|
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
/** Class that represents single value with possible infinities.
|
|
|
|
* Single field is stored in column for more optimal inplace comparisons with other regular columns.
|
|
|
|
* Extracting fields from columns and further their comparison is suboptimal and requires extra copying.
|
|
|
|
*/
|
2021-01-02 09:47:38 +00:00
|
|
|
struct FieldValue
|
2020-02-26 14:21:56 +00:00
|
|
|
{
|
2023-10-24 19:08:17 +00:00
|
|
|
explicit FieldValue(MutableColumnPtr && column_) : column(std::move(column_)) {}
|
2020-02-26 14:21:56 +00:00
|
|
|
void update(const Field & x);
|
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
bool isNormal() const { return !value.isPositiveInfinity() && !value.isNegativeInfinity(); }
|
|
|
|
bool isPositiveInfinity() const { return value.isPositiveInfinity(); }
|
|
|
|
bool isNegativeInfinity() const { return value.isNegativeInfinity(); }
|
2020-02-26 14:21:56 +00:00
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
Field value; // Null, -Inf, +Inf
|
2020-02-26 14:21:56 +00:00
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
// If value is Null, uses the actual value in column
|
2020-02-26 14:21:56 +00:00
|
|
|
MutableColumnPtr column;
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2020-01-29 21:40:22 +00:00
|
|
|
/// Class for checkInRange function.
|
2018-02-08 15:31:37 +00:00
|
|
|
class MergeTreeSetIndex
|
|
|
|
{
|
2018-02-02 13:19:40 +00:00
|
|
|
public:
|
2018-02-08 14:15:21 +00:00
|
|
|
/** Mapping for tuple positions from Set::set_elements to
|
2018-07-02 18:57:14 +00:00
|
|
|
* position of pk index and functions chain applied to this column.
|
2018-02-08 14:15:21 +00:00
|
|
|
*/
|
2018-04-20 00:27:25 +00:00
|
|
|
struct KeyTuplePositionMapping
|
2018-02-08 15:31:37 +00:00
|
|
|
{
|
2018-02-02 13:19:40 +00:00
|
|
|
size_t tuple_index;
|
2018-04-20 00:20:36 +00:00
|
|
|
size_t key_index;
|
2018-02-08 17:46:22 +00:00
|
|
|
std::vector<FunctionBasePtr> functions;
|
2018-02-02 13:19:40 +00:00
|
|
|
};
|
|
|
|
|
2021-08-29 08:49:30 +00:00
|
|
|
MergeTreeSetIndex(const Columns & set_elements, std::vector<KeyTuplePositionMapping> && indexes_mapping_);
|
2018-02-02 13:19:40 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
size_t size() const { return ordered_set.at(0)->size(); }
|
2018-04-20 01:14:04 +00:00
|
|
|
|
2020-04-02 17:27:07 +00:00
|
|
|
bool hasMonotonicFunctionsChain() const;
|
|
|
|
|
2022-03-09 07:48:42 +00:00
|
|
|
BoolMask checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types, bool single_point = false) const;
|
2018-04-20 01:14:04 +00:00
|
|
|
|
2023-12-01 18:59:25 +00:00
|
|
|
const Columns & getOrderedSet() const { return ordered_set; }
|
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
private:
|
2021-08-29 08:49:30 +00:00
|
|
|
// If all arguments in tuple are key columns, we can optimize NOT IN when there is only one element.
|
|
|
|
bool has_all_keys;
|
2018-06-30 21:35:01 +00:00
|
|
|
Columns ordered_set;
|
2018-04-20 00:27:25 +00:00
|
|
|
std::vector<KeyTuplePositionMapping> indexes_mapping;
|
2020-02-26 14:21:56 +00:00
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
using FieldValues = std::vector<FieldValue>;
|
2018-02-02 13:19:40 +00:00
|
|
|
};
|
|
|
|
|
2019-03-08 03:23:34 +00:00
|
|
|
}
|