2018-06-30 21:35:01 +00:00
|
|
|
#include <optional>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Field.h>
|
2017-11-24 13:55:31 +00:00
|
|
|
#include <Common/FieldVisitors.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Row.h>
|
2012-08-23 20:22:44 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <Columns/ColumnTuple.h>
|
2012-08-23 20:22:44 +00:00
|
|
|
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
|
2019-01-23 14:48:50 +00:00
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
2012-08-24 19:42:03 +00:00
|
|
|
|
2017-12-08 00:50:25 +00:00
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2017-12-09 07:32:32 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2016-02-13 06:37:19 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
2012-08-23 23:49:28 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Set.h>
|
|
|
|
#include <Interpreters/convertFieldToType.h>
|
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Interpreters/NullableUtils.h>
|
2018-06-30 21:35:01 +00:00
|
|
|
#include <Interpreters/sortBlock.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2012-08-23 20:22:44 +00:00
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
#include <Storages/MergeTree/KeyCondition.h>
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
#include <ext/range.h>
|
2018-09-27 15:55:22 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2016-01-12 02:21:15 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int SET_SIZE_LIMIT_EXCEEDED;
|
|
|
|
extern const int TYPE_MISMATCH;
|
|
|
|
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
2016-01-12 02:21:15 +00:00
|
|
|
}
|
|
|
|
|
2017-03-28 03:00:33 +00:00
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
template <typename Method>
|
2015-03-02 05:41:21 +00:00
|
|
|
void NO_INLINE Set::insertFromBlockImpl(
|
2017-04-01 07:20:54 +00:00
|
|
|
Method & method,
|
2017-12-13 01:27:53 +00:00
|
|
|
const ColumnRawPtrs & key_columns,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t rows,
|
|
|
|
SetVariants & variants,
|
2018-06-30 21:35:01 +00:00
|
|
|
ConstNullMapPtr null_map,
|
|
|
|
ColumnUInt8::Container * out_filter)
|
2017-03-28 03:00:33 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (null_map)
|
2018-06-30 21:35:01 +00:00
|
|
|
{
|
|
|
|
if (out_filter)
|
|
|
|
insertFromBlockImplCase<Method, true, true>(method, key_columns, rows, variants, null_map, out_filter);
|
|
|
|
else
|
|
|
|
insertFromBlockImplCase<Method, true, false>(method, key_columns, rows, variants, null_map, out_filter);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2018-06-30 21:35:01 +00:00
|
|
|
{
|
|
|
|
if (out_filter)
|
|
|
|
insertFromBlockImplCase<Method, false, true>(method, key_columns, rows, variants, null_map, out_filter);
|
|
|
|
else
|
|
|
|
insertFromBlockImplCase<Method, false, false>(method, key_columns, rows, variants, null_map, out_filter);
|
|
|
|
}
|
2017-03-28 03:00:33 +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 NO_INLINE Set::insertFromBlockImplCase(
|
2017-04-01 07:20:54 +00:00
|
|
|
Method & method,
|
2017-12-13 01:27:53 +00:00
|
|
|
const ColumnRawPtrs & key_columns,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t rows,
|
|
|
|
SetVariants & variants,
|
2019-02-04 16:48:43 +00:00
|
|
|
[[maybe_unused]] ConstNullMapPtr null_map,
|
|
|
|
[[maybe_unused]] ColumnUInt8::Container * out_filter)
|
2015-03-02 01:11:37 +00:00
|
|
|
{
|
2019-02-04 14:36:15 +00:00
|
|
|
typename Method::State state(key_columns, key_sizes, nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// For all rows
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
{
|
2019-02-04 16:48:43 +00:00
|
|
|
if constexpr (has_null_map)
|
2019-12-17 10:20:04 +00:00
|
|
|
{
|
2019-02-04 16:48:43 +00:00
|
|
|
if ((*null_map)[i])
|
2019-12-17 10:20:04 +00:00
|
|
|
{
|
|
|
|
if constexpr (build_filter)
|
|
|
|
{
|
|
|
|
(*out_filter)[i] = false;
|
|
|
|
}
|
2019-02-04 16:48:43 +00:00
|
|
|
continue;
|
2019-12-17 10:20:04 +00:00
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-04 16:48:43 +00:00
|
|
|
[[maybe_unused]] auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool);
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2019-02-04 16:48:43 +00:00
|
|
|
if constexpr (build_filter)
|
2019-02-04 14:36:15 +00:00
|
|
|
(*out_filter)[i] = emplace_result.isInserted();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-03-09 00:28:05 +00:00
|
|
|
void Set::setHeader(const Block & header)
|
2012-08-23 20:22:44 +00:00
|
|
|
{
|
2017-12-15 19:17:15 +00:00
|
|
|
std::unique_lock lock(rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-19 21:34:04 +00:00
|
|
|
if (!empty())
|
|
|
|
return;
|
|
|
|
|
2020-03-09 00:28:05 +00:00
|
|
|
keys_size = header.columns();
|
2017-12-13 01:27:53 +00:00
|
|
|
ColumnRawPtrs key_columns;
|
2017-04-01 07:20:54 +00:00
|
|
|
key_columns.reserve(keys_size);
|
2018-04-19 21:34:04 +00:00
|
|
|
data_types.reserve(keys_size);
|
2019-10-31 16:14:06 +00:00
|
|
|
set_elements_types.reserve(keys_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-19 21:34:04 +00:00
|
|
|
/// The constant columns to the right of IN are not supported directly. For this, they first materialize.
|
|
|
|
Columns materialized_columns;
|
|
|
|
|
|
|
|
/// Remember the columns we will work with
|
|
|
|
for (size_t i = 0; i < keys_size; ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-09 00:28:05 +00:00
|
|
|
materialized_columns.emplace_back(header.safeGetByPosition(i).column->convertToFullColumnIfConst());
|
2018-12-21 16:00:07 +00:00
|
|
|
key_columns.emplace_back(materialized_columns.back().get());
|
2020-03-09 00:28:05 +00:00
|
|
|
data_types.emplace_back(header.safeGetByPosition(i).type);
|
|
|
|
set_elements_types.emplace_back(header.safeGetByPosition(i).type);
|
2018-04-19 21:34:04 +00:00
|
|
|
|
2018-09-20 09:40:10 +00:00
|
|
|
/// Convert low cardinality column to full.
|
2020-04-22 06:01:33 +00:00
|
|
|
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(data_types.back().get()))
|
2018-09-20 09:40:10 +00:00
|
|
|
{
|
|
|
|
data_types.back() = low_cardinality_type->getDictionaryType();
|
2018-09-27 15:55:22 +00:00
|
|
|
materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality());
|
2018-09-20 09:40:10 +00:00
|
|
|
key_columns.back() = materialized_columns.back().get();
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-04-19 21:34:04 +00:00
|
|
|
/// We will insert to the Set only keys, where all components are not NULL.
|
|
|
|
ConstNullMapPtr null_map{};
|
2020-11-05 18:07:44 +00:00
|
|
|
ColumnPtr null_map_holder;
|
|
|
|
if (!transform_null_in)
|
|
|
|
extractNestedColumnsAndNullMap(key_columns, null_map);
|
2018-04-19 21:34:04 +00:00
|
|
|
|
2018-07-02 18:57:14 +00:00
|
|
|
if (fill_set_elements)
|
|
|
|
{
|
|
|
|
/// Create empty columns with set values in advance.
|
|
|
|
/// It is needed because set may be empty, so method 'insertFromBlock' will be never called.
|
|
|
|
set_elements.reserve(keys_size);
|
2019-10-31 16:14:06 +00:00
|
|
|
for (const auto & type : set_elements_types)
|
|
|
|
set_elements.emplace_back(type->createColumn());
|
2018-07-02 18:57:14 +00:00
|
|
|
}
|
|
|
|
|
2018-04-19 21:34:04 +00:00
|
|
|
/// Choose data structure to use for the set.
|
|
|
|
data.init(data.chooseMethod(key_columns, key_sizes));
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-07-02 18:57:14 +00:00
|
|
|
bool Set::insertFromBlock(const Block & block)
|
2018-04-19 21:34:04 +00:00
|
|
|
{
|
|
|
|
std::unique_lock lock(rwlock);
|
|
|
|
|
|
|
|
if (empty())
|
|
|
|
throw Exception("Method Set::setHeader must be called before Set::insertFromBlock", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
ColumnRawPtrs key_columns;
|
|
|
|
key_columns.reserve(keys_size);
|
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// The constant columns to the right of IN are not supported directly. For this, they first materialize.
|
2017-04-01 07:20:54 +00:00
|
|
|
Columns materialized_columns;
|
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// Remember the columns we will work with
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < keys_size; ++i)
|
|
|
|
{
|
2018-12-21 16:00:07 +00:00
|
|
|
materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality());
|
|
|
|
key_columns.emplace_back(materialized_columns.back().get());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
size_t rows = block.rows();
|
|
|
|
|
|
|
|
/// We will insert to the Set only keys, where all components are not NULL.
|
|
|
|
ConstNullMapPtr null_map{};
|
2020-11-05 18:07:44 +00:00
|
|
|
ColumnPtr null_map_holder;
|
|
|
|
if (!transform_null_in)
|
|
|
|
null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
/// Filter to extract distinct values from the block.
|
|
|
|
ColumnUInt8::MutablePtr filter;
|
|
|
|
if (fill_set_elements)
|
|
|
|
filter = ColumnUInt8::create(block.rows());
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
switch (data.type)
|
|
|
|
{
|
|
|
|
case SetVariants::Type::EMPTY:
|
|
|
|
break;
|
2015-03-02 05:41:21 +00:00
|
|
|
#define M(NAME) \
|
2017-04-01 07:20:54 +00:00
|
|
|
case SetVariants::Type::NAME: \
|
2018-06-30 21:35:01 +00:00
|
|
|
insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, filter ? &filter->getData() : nullptr); \
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
APPLY_FOR_SET_VARIANTS(M)
|
2015-03-02 05:41:21 +00:00
|
|
|
#undef M
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2012-08-23 23:49:28 +00:00
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
if (fill_set_elements)
|
2018-01-21 07:30:07 +00:00
|
|
|
{
|
2018-06-30 21:35:01 +00:00
|
|
|
for (size_t i = 0; i < keys_size; ++i)
|
|
|
|
{
|
|
|
|
auto filtered_column = block.getByPosition(i).column->filter(filter->getData(), rows);
|
2018-07-02 18:57:14 +00:00
|
|
|
if (set_elements[i]->empty())
|
2018-06-30 21:56:45 +00:00
|
|
|
set_elements[i] = filtered_column;
|
|
|
|
else
|
2019-03-29 14:17:29 +00:00
|
|
|
set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size());
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
2018-01-21 07:30:07 +00:00
|
|
|
}
|
2014-03-04 14:19:32 +00:00
|
|
|
|
2018-03-11 00:15:26 +00:00
|
|
|
return limits.check(getTotalRowCount(), getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|
|
|
|
|
2018-04-19 21:34:04 +00:00
|
|
|
|
2015-10-08 03:41:11 +00:00
|
|
|
ColumnPtr Set::execute(const Block & block, bool negative) const
|
2012-08-23 20:22:44 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t num_key_columns = block.columns();
|
|
|
|
|
|
|
|
if (0 == num_key_columns)
|
|
|
|
throw Exception("Logical error: no columns passed to Set::execute method.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2017-12-14 01:43:19 +00:00
|
|
|
auto res = ColumnUInt8::create();
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnUInt8::Container & vec_res = res->getData();
|
2017-04-01 07:20:54 +00:00
|
|
|
vec_res.resize(block.safeGetByPosition(0).column->size());
|
|
|
|
|
2018-09-02 03:47:14 +00:00
|
|
|
if (vec_res.empty())
|
|
|
|
return res;
|
|
|
|
|
2017-12-15 19:17:15 +00:00
|
|
|
std::shared_lock lock(rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// If the set is empty.
|
|
|
|
if (data_types.empty())
|
|
|
|
{
|
|
|
|
if (negative)
|
2018-09-02 03:00:04 +00:00
|
|
|
memset(vec_res.data(), 1, vec_res.size());
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2018-09-02 03:00:04 +00:00
|
|
|
memset(vec_res.data(), 0, vec_res.size());
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-08-19 16:51:50 +00:00
|
|
|
checkColumnsNumber(num_key_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-05 20:52:01 +00:00
|
|
|
/// Remember the columns we will work with. Also check that the data types are correct.
|
|
|
|
ColumnRawPtrs key_columns;
|
|
|
|
key_columns.reserve(num_key_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-05 20:52:01 +00:00
|
|
|
/// The constant columns to the left of IN are not supported directly. For this, they first materialize.
|
|
|
|
Columns materialized_columns;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-05 20:52:01 +00:00
|
|
|
for (size_t i = 0; i < num_key_columns; ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-08-19 16:51:50 +00:00
|
|
|
checkTypesEqual(i, block.safeGetByPosition(i).type);
|
2018-12-21 16:00:07 +00:00
|
|
|
materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst());
|
|
|
|
key_columns.emplace_back() = materialized_columns.back().get();
|
2018-04-05 20:52:01 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-05 20:52:01 +00:00
|
|
|
/// We will check existence in Set only for keys, where all components are not NULL.
|
|
|
|
ConstNullMapPtr null_map{};
|
2020-11-05 18:07:44 +00:00
|
|
|
ColumnPtr null_map_holder;
|
|
|
|
if (!transform_null_in)
|
|
|
|
null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-05 20:52:01 +00:00
|
|
|
executeOrdinary(key_columns, vec_res, negative, null_map);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res;
|
2013-03-19 12:25:59 +00:00
|
|
|
}
|
2012-08-23 22:40:51 +00:00
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
|
|
|
|
template <typename Method>
|
2015-03-02 01:39:42 +00:00
|
|
|
void NO_INLINE Set::executeImpl(
|
2017-04-01 07:20:54 +00:00
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
bool negative,
|
|
|
|
size_t rows,
|
|
|
|
ConstNullMapPtr null_map) const
|
2017-03-28 03:00:33 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (null_map)
|
|
|
|
executeImplCase<Method, true>(method, key_columns, vec_res, negative, rows, null_map);
|
|
|
|
else
|
|
|
|
executeImplCase<Method, false>(method, key_columns, vec_res, negative, rows, null_map);
|
2017-03-28 03:00:33 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Method, bool has_null_map>
|
|
|
|
void NO_INLINE Set::executeImplCase(
|
2017-04-01 07:20:54 +00:00
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
bool negative,
|
|
|
|
size_t rows,
|
|
|
|
ConstNullMapPtr null_map) const
|
2013-03-19 12:25:59 +00:00
|
|
|
{
|
2019-02-04 14:36:15 +00:00
|
|
|
Arena pool;
|
|
|
|
typename Method::State state(key_columns, key_sizes, nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-04 19:40:06 +00:00
|
|
|
/// NOTE Optimization is not used for consecutive identical strings.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// For all rows
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
{
|
|
|
|
if (has_null_map && (*null_map)[i])
|
2020-04-06 13:30:16 +00:00
|
|
|
{
|
2020-11-05 18:07:44 +00:00
|
|
|
vec_res[i] = negative;
|
2020-04-06 13:30:16 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
|
|
|
{
|
2019-02-04 14:36:15 +00:00
|
|
|
auto find_result = state.findKey(method.data, i, pool);
|
|
|
|
vec_res[i] = negative ^ find_result.isFound();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2015-03-02 01:11:37 +00:00
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
|
2017-03-28 03:00:33 +00:00
|
|
|
void Set::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-04-01 07:20:54 +00:00
|
|
|
bool negative,
|
|
|
|
ConstNullMapPtr null_map) const
|
2015-03-02 01:11:37 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t rows = key_columns[0]->size();
|
2015-03-02 01:11:37 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
switch (data.type)
|
|
|
|
{
|
|
|
|
case SetVariants::Type::EMPTY:
|
|
|
|
break;
|
2015-03-02 05:41:21 +00:00
|
|
|
#define M(NAME) \
|
2017-04-01 07:20:54 +00:00
|
|
|
case SetVariants::Type::NAME: \
|
|
|
|
executeImpl(*data.NAME, key_columns, vec_res, negative, rows, null_map); \
|
|
|
|
break;
|
|
|
|
APPLY_FOR_SET_VARIANTS(M)
|
2015-03-02 01:11:37 +00:00
|
|
|
#undef M
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|
|
|
|
|
2019-08-19 16:51:50 +00:00
|
|
|
void Set::checkColumnsNumber(size_t num_key_columns) const
|
|
|
|
{
|
|
|
|
if (data_types.size() != num_key_columns)
|
|
|
|
{
|
|
|
|
std::stringstream message;
|
2020-11-07 00:14:53 +00:00
|
|
|
message.exceptions(std::ios::failbit);
|
2019-08-19 16:51:50 +00:00
|
|
|
message << "Number of columns in section IN doesn't match. "
|
|
|
|
<< num_key_columns << " at left, " << data_types.size() << " at right.";
|
|
|
|
throw Exception(message.str(), ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-07-01 14:05:54 +00:00
|
|
|
bool Set::areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const
|
|
|
|
{
|
|
|
|
return removeNullable(recursiveRemoveLowCardinality(data_types[set_type_idx]))->equals(*removeNullable(recursiveRemoveLowCardinality(other_type)));
|
|
|
|
}
|
|
|
|
|
2019-08-19 16:51:50 +00:00
|
|
|
void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const
|
|
|
|
{
|
2020-07-01 14:05:54 +00:00
|
|
|
if (!this->areTypesEqual(set_type_idx, other_type))
|
2019-08-19 16:51:50 +00:00
|
|
|
throw Exception("Types of column " + toString(set_type_idx + 1) + " in section IN don't match: "
|
2019-11-20 00:57:06 +00:00
|
|
|
+ other_type->getName() + " on the left, "
|
|
|
|
+ data_types[set_type_idx]->getName() + " on the right", ErrorCodes::TYPE_MISMATCH);
|
2019-08-19 16:51:50 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<KeyTuplePositionMapping> && index_mapping_)
|
|
|
|
: indexes_mapping(std::move(index_mapping_))
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2018-02-08 15:31:37 +00:00
|
|
|
std::sort(indexes_mapping.begin(), indexes_mapping.end(),
|
2018-04-20 00:27:25 +00:00
|
|
|
[](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r)
|
2018-02-08 15:31:37 +00:00
|
|
|
{
|
2018-04-20 00:20:36 +00:00
|
|
|
return std::forward_as_tuple(l.key_index, l.tuple_index) < std::forward_as_tuple(r.key_index, r.tuple_index);
|
2018-02-08 15:31:37 +00:00
|
|
|
});
|
|
|
|
|
2018-04-18 18:44:02 +00:00
|
|
|
indexes_mapping.erase(std::unique(
|
2018-02-02 13:19:40 +00:00
|
|
|
indexes_mapping.begin(), indexes_mapping.end(),
|
2018-04-20 00:27:25 +00:00
|
|
|
[](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r)
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2018-04-20 00:20:36 +00:00
|
|
|
return l.key_index == r.key_index;
|
2018-04-18 18:44:02 +00:00
|
|
|
}), indexes_mapping.end());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
size_t tuple_size = indexes_mapping.size();
|
|
|
|
ordered_set.resize(tuple_size);
|
2020-02-26 14:21:56 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
ordered_set[i] = set_elements[indexes_mapping[i].tuple_index];
|
2018-04-18 18:44:02 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
Block block_to_sort;
|
|
|
|
SortDescription sort_description;
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
{
|
|
|
|
block_to_sort.insert({ ordered_set[i], nullptr, "" });
|
|
|
|
sort_description.emplace_back(i, 1, 1);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
sortBlock(block_to_sort, sort_description);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
ordered_set[i] = block_to_sort.getByPosition(i).column;
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-05 20:52:01 +00:00
|
|
|
|
2018-02-08 14:15:21 +00:00
|
|
|
/** Return the BoolMask where:
|
|
|
|
* 1: the intersection of the set and the range is non-empty
|
|
|
|
* 2: the range contains elements not in the set
|
|
|
|
*/
|
2020-07-21 11:13:35 +00:00
|
|
|
BoolMask MergeTreeSetIndex::checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types) const
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2018-06-30 21:35:01 +00:00
|
|
|
size_t tuple_size = indexes_mapping.size();
|
|
|
|
|
2020-07-21 11:15:40 +00:00
|
|
|
ColumnsWithInfinity left_point;
|
|
|
|
ColumnsWithInfinity right_point;
|
|
|
|
left_point.reserve(tuple_size);
|
|
|
|
right_point.reserve(tuple_size);
|
2020-07-21 11:13:35 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
{
|
|
|
|
left_point.emplace_back(ordered_set[i]->cloneEmpty());
|
|
|
|
right_point.emplace_back(ordered_set[i]->cloneEmpty());
|
|
|
|
}
|
|
|
|
|
2018-02-08 14:15:21 +00:00
|
|
|
bool invert_left_infinities = false;
|
|
|
|
bool invert_right_infinities = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-20 00:20:36 +00:00
|
|
|
std::optional<Range> new_range = KeyCondition::applyMonotonicFunctionsChainToRange(
|
|
|
|
key_ranges[indexes_mapping[i].key_index],
|
2018-02-08 15:31:37 +00:00
|
|
|
indexes_mapping[i].functions,
|
2018-04-20 00:20:36 +00:00
|
|
|
data_types[indexes_mapping[i].key_index]);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
if (!new_range)
|
|
|
|
return {true, true};
|
2014-04-01 10:09:22 +00:00
|
|
|
|
2018-02-08 14:15:21 +00:00
|
|
|
/** A range that ends in (x, y, ..., +inf) exclusive is the same as a range
|
|
|
|
* that ends in (x, y, ..., -inf) inclusive and vice versa for the left bound.
|
|
|
|
*/
|
2018-02-02 13:19:40 +00:00
|
|
|
if (new_range->left_bounded)
|
|
|
|
{
|
2018-02-08 14:15:21 +00:00
|
|
|
if (!new_range->left_included)
|
|
|
|
invert_left_infinities = true;
|
2018-02-08 15:31:37 +00:00
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
left_point[i].update(new_range->left);
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-02-08 14:15:21 +00:00
|
|
|
if (invert_left_infinities)
|
2020-02-26 14:21:56 +00:00
|
|
|
left_point[i].update(ValueWithInfinity::PLUS_INFINITY);
|
2018-02-08 15:31:37 +00:00
|
|
|
else
|
2020-02-26 14:21:56 +00:00
|
|
|
left_point[i].update(ValueWithInfinity::MINUS_INFINITY);
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
2015-10-12 07:05:54 +00:00
|
|
|
|
2018-02-02 13:19:40 +00:00
|
|
|
if (new_range->right_bounded)
|
|
|
|
{
|
2018-02-08 14:15:21 +00:00
|
|
|
if (!new_range->right_included)
|
|
|
|
invert_right_infinities = true;
|
2018-02-08 15:31:37 +00:00
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
right_point[i].update(new_range->right);
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-02-08 14:15:21 +00:00
|
|
|
if (invert_right_infinities)
|
2020-02-26 14:21:56 +00:00
|
|
|
right_point[i].update(ValueWithInfinity::MINUS_INFINITY);
|
2018-02-08 15:31:37 +00:00
|
|
|
else
|
2020-02-26 14:21:56 +00:00
|
|
|
right_point[i].update(ValueWithInfinity::PLUS_INFINITY);
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
auto compare = [](const IColumn & lhs, const ValueWithInfinity & rhs, size_t row)
|
2018-06-30 21:35:01 +00:00
|
|
|
{
|
2020-02-26 14:21:56 +00:00
|
|
|
auto type = rhs.getType();
|
2020-02-27 11:23:55 +00:00
|
|
|
/// Return inverted infinity sign, because in 'lhs' all values are finite.
|
2020-02-26 14:21:56 +00:00
|
|
|
if (type != ValueWithInfinity::NORMAL)
|
2020-02-27 11:23:55 +00:00
|
|
|
return -static_cast<int>(type);
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
return lhs.compareAt(row, 0, rhs.getColumnIfFinite(), 1);
|
2018-06-30 21:35:01 +00:00
|
|
|
};
|
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
auto less = [this, &compare, tuple_size](size_t row, const auto & point)
|
2020-02-21 14:33:51 +00:00
|
|
|
{
|
2020-02-26 14:21:56 +00:00
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
{
|
|
|
|
int res = compare(*ordered_set[i], point[i], row);
|
|
|
|
if (res)
|
|
|
|
return res < 0;
|
|
|
|
}
|
|
|
|
return false;
|
2020-02-21 14:33:51 +00:00
|
|
|
};
|
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
auto equals = [this, &compare, tuple_size](size_t row, const auto & point)
|
2020-02-21 14:33:51 +00:00
|
|
|
{
|
2020-02-26 14:21:56 +00:00
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
if (compare(*ordered_set[i], point[i], row) != 0)
|
|
|
|
return false;
|
|
|
|
return true;
|
2020-02-21 14:33:51 +00:00
|
|
|
};
|
|
|
|
|
2020-03-10 14:56:55 +00:00
|
|
|
/** Because each hyperrectangle maps to a contiguous sequence of elements
|
2020-08-08 01:01:47 +00:00
|
|
|
* laid out in the lexicographically increasing order, the set intersects the range
|
2020-02-21 14:33:51 +00:00
|
|
|
* if and only if either bound coincides with an element or at least one element
|
|
|
|
* is between the lower bounds
|
|
|
|
*/
|
2020-02-26 14:21:56 +00:00
|
|
|
auto indices = ext::range(0, size());
|
|
|
|
auto left_lower = std::lower_bound(indices.begin(), indices.end(), left_point, less);
|
|
|
|
auto right_lower = std::lower_bound(indices.begin(), indices.end(), right_point, less);
|
2018-06-30 21:35:01 +00:00
|
|
|
|
|
|
|
return
|
|
|
|
{
|
|
|
|
left_lower != right_lower
|
2020-02-26 14:21:56 +00:00
|
|
|
|| (left_lower != indices.end() && equals(*left_lower, left_point))
|
|
|
|
|| (right_lower != indices.end() && equals(*right_lower, right_point)),
|
2018-06-30 21:35:01 +00:00
|
|
|
true
|
|
|
|
};
|
2015-10-12 07:05:54 +00:00
|
|
|
}
|
|
|
|
|
2020-04-02 17:27:07 +00:00
|
|
|
bool MergeTreeSetIndex::hasMonotonicFunctionsChain() const
|
|
|
|
{
|
|
|
|
for (const auto & mapping : indexes_mapping)
|
|
|
|
if (!mapping.functions.empty())
|
|
|
|
return true;
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
void ValueWithInfinity::update(const Field & x)
|
|
|
|
{
|
|
|
|
/// Keep at most one element in column.
|
|
|
|
if (!column->empty())
|
|
|
|
column->popBack(1);
|
|
|
|
column->insert(x);
|
|
|
|
type = NORMAL;
|
|
|
|
}
|
|
|
|
|
|
|
|
const IColumn & ValueWithInfinity::getColumnIfFinite() const
|
|
|
|
{
|
2020-04-02 17:27:07 +00:00
|
|
|
#ifndef NDEBUG
|
2020-02-26 14:21:56 +00:00
|
|
|
if (type != NORMAL)
|
2020-02-27 11:23:55 +00:00
|
|
|
throw Exception("Trying to get column of infinite type", ErrorCodes::LOGICAL_ERROR);
|
2020-04-02 17:27:07 +00:00
|
|
|
#endif
|
|
|
|
|
2020-02-26 14:21:56 +00:00
|
|
|
return *column;
|
|
|
|
}
|
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|