#pragma once #include #include #include #include #include #include #include #include namespace DB { struct Range; class FieldWithInfinity; using SetElements = std::vector>; using SetElementsPtr = std::unique_ptr; /** Data structure for implementation of IN expression. */ class Set { 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), set_elements(std::make_unique()) { } bool empty() const { return data.empty(); } /** 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). * 'fill_set_elements' - if true, fill vector of elements. For primary key to work. */ void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool fill_set_elements); /** Returns false, if some limit was exceeded and no need to insert more data. */ bool insertFromBlock(const Block & block, bool fill_set_elements); /** 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; size_t getTotalRowCount() const { return data.getTotalRowCount(); } size_t getTotalByteCount() const { return data.getTotalByteCount(); } SetElements & getSetElements() { return *set_elements.get(); } 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. */ /** 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; /// Limitations on the maximum size of the set size_t max_rows; size_t max_bytes; OverflowMode overflow_mode; /// 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 & vec_res, bool negative) const; /// If in the left part columns contains the same types as the elements of the set. void executeOrdinary( const ColumnRawPtrs & key_columns, ColumnUInt8::Container & vec_res, bool negative, const PaddedPODArray * null_map) const; /// Check whether the permissible sizes of keys set reached bool checkSetSizeLimits() const; /// Vector of elements of `Set`. /// It is necessary for the index to work on the primary key in the IN statement. SetElementsPtr set_elements; /** 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 void insertFromBlockImpl( Method & method, const ColumnRawPtrs & key_columns, size_t rows, SetVariants & variants, ConstNullMapPtr null_map); template void insertFromBlockImplCase( Method & method, const ColumnRawPtrs & key_columns, size_t rows, SetVariants & variants, ConstNullMapPtr null_map); template void executeImpl( Method & method, const ColumnRawPtrs & key_columns, ColumnUInt8::Container & vec_res, bool negative, size_t rows, ConstNullMapPtr null_map) const; template void executeImplCase( Method & method, const ColumnRawPtrs & key_columns, ColumnUInt8::Container & vec_res, bool negative, size_t rows, ConstNullMapPtr null_map) const; template void executeArrayImpl( Method & method, const ColumnRawPtrs & key_columns, const ColumnArray::Offsets & offsets, ColumnUInt8::Container & vec_res, bool negative, size_t rows) const; }; using SetPtr = std::shared_ptr; using ConstSetPtr = std::shared_ptr; using Sets = std::vector; class IFunction; using FunctionPtr = std::shared_ptr; /// Class for mayBeTrueInRange function. class MergeTreeSetIndex { public: /** Mapping for tuple positions from Set::set_elements to * position of pk index and data type of this pk column * and functions chain applied to this column. */ struct PKTuplePositionMapping { size_t tuple_index; size_t pk_index; std::vector functions; DataTypePtr data_type; bool operator< (const PKTuplePositionMapping & other) const; }; MergeTreeSetIndex(const SetElements & set_elements, std::vector && indexes_mapping_); BoolMask mayBeTrueInRange(const std::vector & key_ranges); private: using OrderedTuples = std::vector>; OrderedTuples ordered_set; std::vector indexes_mapping; }; }