2018-06-30 21:35:01 +00:00
|
|
|
#include <optional>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Field.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>
|
|
|
|
|
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-12-05 13:18:56 +00:00
|
|
|
#include <Interpreters/castColumn.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2012-08-23 20:22:44 +00:00
|
|
|
|
2022-07-19 17:43:18 +00:00
|
|
|
#include <Processors/Chunk.h>
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
#include <Storages/MergeTree/KeyCondition.h>
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/range.h>
|
2022-01-30 19:49:48 +00:00
|
|
|
#include <base/sort.h>
|
2018-09-27 15:55:22 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2022-01-30 19:49:48 +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
|
|
|
|
{
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
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(
|
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-03-28 03:00:33 +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-03-28 03:00:33 +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(
|
|
|
|
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,
|
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
|
|
|
|
2017-03-03 21:15:46 +00:00
|
|
|
/// For all rows
|
2015-03-02 05:41:21 +00:00
|
|
|
for (size_t i = 0; i < rows; ++i)
|
2015-03-02 01:11:37 +00:00
|
|
|
{
|
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();
|
2015-03-02 01:11:37 +00:00
|
|
|
}
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-08-18 10:30:02 +00:00
|
|
|
void Set::setHeader(const ColumnsWithTypeAndName & header)
|
2012-08-23 20:22:44 +00:00
|
|
|
{
|
2022-06-28 19:19:06 +00:00
|
|
|
std::lock_guard lock(rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-12-29 18:16:24 +00:00
|
|
|
if (!data.empty())
|
2018-04-19 21:34:04 +00:00
|
|
|
return;
|
|
|
|
|
2021-08-18 10:30:02 +00:00
|
|
|
keys_size = header.size();
|
2017-12-13 01:27:53 +00:00
|
|
|
ColumnRawPtrs key_columns;
|
2016-07-10 11:49:33 +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)
|
2016-07-10 18:53:27 +00:00
|
|
|
{
|
2021-08-18 10:30:02 +00:00
|
|
|
materialized_columns.emplace_back(header.at(i).column->convertToFullColumnIfConst());
|
2018-12-21 16:00:07 +00:00
|
|
|
key_columns.emplace_back(materialized_columns.back().get());
|
2021-08-18 10:30:02 +00:00
|
|
|
data_types.emplace_back(header.at(i).type);
|
|
|
|
set_elements_types.emplace_back(header.at(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();
|
2022-10-22 16:17:05 +00:00
|
|
|
set_elements_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();
|
|
|
|
}
|
2016-07-10 18:53:27 +00:00
|
|
|
}
|
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)
|
2021-01-31 18:49:27 +00:00
|
|
|
{
|
|
|
|
/// We convert nullable columns to non nullable we also need to update nullable types
|
|
|
|
for (size_t i = 0; i < set_elements_types.size(); ++i)
|
|
|
|
{
|
|
|
|
data_types[i] = removeNullable(data_types[i]);
|
|
|
|
set_elements_types[i] = removeNullable(set_elements_types[i]);
|
|
|
|
}
|
|
|
|
|
2020-11-05 18:07:44 +00:00
|
|
|
extractNestedColumnsAndNullMap(key_columns, null_map);
|
2021-01-31 18:49:27 +00:00
|
|
|
}
|
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));
|
|
|
|
}
|
|
|
|
|
2021-08-18 10:30:02 +00:00
|
|
|
bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns)
|
2022-07-19 17:43:18 +00:00
|
|
|
{
|
|
|
|
Columns cols;
|
|
|
|
cols.reserve(columns.size());
|
|
|
|
for (const auto & column : columns)
|
|
|
|
cols.emplace_back(column.column);
|
|
|
|
return insertFromBlock(cols);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool Set::insertFromBlock(const Columns & columns)
|
2018-04-19 21:34:04 +00:00
|
|
|
{
|
2022-03-16 11:21:18 +00:00
|
|
|
std::lock_guard<std::shared_mutex> lock(rwlock);
|
2018-04-19 21:34:04 +00:00
|
|
|
|
2020-12-29 18:16:24 +00:00
|
|
|
if (data.empty())
|
2018-04-19 21:34:04 +00:00
|
|
|
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.
|
2015-03-15 07:56:46 +00:00
|
|
|
Columns materialized_columns;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// Remember the columns we will work with
|
2014-03-04 11:26:55 +00:00
|
|
|
for (size_t i = 0; i < keys_size; ++i)
|
2014-03-04 11:26:55 +00:00
|
|
|
{
|
2022-07-19 17:43:18 +00:00
|
|
|
materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded());
|
2018-12-21 16:00:07 +00:00
|
|
|
key_columns.emplace_back(materialized_columns.back().get());
|
2017-03-01 14:48:21 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-07-19 17:43:18 +00:00
|
|
|
size_t rows = columns.at(0)->size();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-28 03:00:33 +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)
|
2021-01-31 18:49:27 +00:00
|
|
|
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)
|
2021-08-18 10:30:02 +00:00
|
|
|
filter = ColumnUInt8::create(rows);
|
2018-06-30 21:35:01 +00:00
|
|
|
|
2016-12-22 16:35:54 +00:00
|
|
|
switch (data.type)
|
|
|
|
{
|
|
|
|
case SetVariants::Type::EMPTY:
|
|
|
|
break;
|
2015-03-02 05:41:21 +00:00
|
|
|
#define M(NAME) \
|
2016-12-22 16:35: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); \
|
2016-12-22 16:35:54 +00:00
|
|
|
break;
|
2015-03-02 05:41:21 +00:00
|
|
|
APPLY_FOR_SET_VARIANTS(M)
|
|
|
|
#undef M
|
2016-12-22 16:35: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)
|
|
|
|
{
|
2020-11-21 05:30:16 +00:00
|
|
|
auto filtered_column = key_columns[i]->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());
|
2021-01-02 09:47:38 +00:00
|
|
|
if (transform_null_in && null_map_holder)
|
|
|
|
set_elements[i]->insert(Null{});
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
2018-01-21 07:30:07 +00:00
|
|
|
}
|
2014-03-04 14:19:32 +00:00
|
|
|
|
2020-12-29 18:16:24 +00:00
|
|
|
return limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|
|
|
|
|
2018-04-19 21:34:04 +00:00
|
|
|
|
2021-08-18 10:30:02 +00:00
|
|
|
ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const
|
2012-08-23 20:22:44 +00:00
|
|
|
{
|
2021-08-18 10:30:02 +00:00
|
|
|
size_t num_key_columns = columns.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2015-10-08 03:41:11 +00:00
|
|
|
if (0 == num_key_columns)
|
|
|
|
throw Exception("Logical error: no columns passed to Set::execute method.", ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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();
|
2021-08-18 10:30:02 +00:00
|
|
|
vec_res.resize(columns.at(0).column->size());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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
|
|
|
|
2017-03-03 21:15:46 +00:00
|
|
|
/// If the set is empty.
|
2012-08-24 20:40:34 +00:00
|
|
|
if (data_types.empty())
|
2013-02-25 18:17:54 +00:00
|
|
|
{
|
|
|
|
if (negative)
|
2018-09-02 03:00:04 +00:00
|
|
|
memset(vec_res.data(), 1, vec_res.size());
|
2014-02-26 16:37:27 +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;
|
2013-02-25 18:17:54 +00:00
|
|
|
}
|
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;
|
2020-12-06 11:58:54 +00:00
|
|
|
materialized_columns.reserve(num_key_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)
|
2013-03-19 12:25:59 +00:00
|
|
|
{
|
2020-12-05 13:18:56 +00:00
|
|
|
ColumnPtr result;
|
|
|
|
|
2021-08-18 10:30:02 +00:00
|
|
|
const auto & column_before_cast = columns.at(i);
|
2020-12-05 13:18:56 +00:00
|
|
|
ColumnWithTypeAndName column_to_cast
|
2020-12-02 10:09:13 +00:00
|
|
|
= {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name};
|
2020-12-05 13:18:56 +00:00
|
|
|
|
2020-12-05 14:55:37 +00:00
|
|
|
if (!transform_null_in && data_types[i]->canBeInsideNullable())
|
|
|
|
{
|
2020-12-17 18:32:25 +00:00
|
|
|
result = castColumnAccurateOrNull(column_to_cast, data_types[i]);
|
2020-12-05 14:55:37 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-12-17 18:32:25 +00:00
|
|
|
result = castColumnAccurate(column_to_cast, data_types[i]);
|
2020-12-05 13:18:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
materialized_columns.emplace_back() = result;
|
2018-12-21 16:00:07 +00:00
|
|
|
key_columns.emplace_back() = materialized_columns.back().get();
|
2018-04-05 20:52:01 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
/// We will check existence in Set only for keys whose components do not contain any NULL value.
|
2018-04-05 20:52:01 +00:00
|
|
|
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
|
|
|
|
2020-12-23 02:10:37 +00:00
|
|
|
bool Set::empty() const
|
|
|
|
{
|
|
|
|
std::shared_lock lock(rwlock);
|
|
|
|
return data.empty();
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t Set::getTotalRowCount() const
|
|
|
|
{
|
|
|
|
std::shared_lock lock(rwlock);
|
|
|
|
return data.getTotalRowCount();
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t Set::getTotalByteCount() const
|
|
|
|
{
|
|
|
|
std::shared_lock lock(rwlock);
|
|
|
|
return data.getTotalByteCount();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-03-02 01:11:37 +00:00
|
|
|
template <typename Method>
|
2015-03-02 01:39:42 +00:00
|
|
|
void NO_INLINE Set::executeImpl(
|
2015-03-02 01:11:37 +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,
|
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
|
|
|
|
{
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Method, bool has_null_map>
|
|
|
|
void NO_INLINE Set::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
|
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
|
2015-03-02 01:11:37 +00:00
|
|
|
for (size_t i = 0; i < rows; ++i)
|
2012-08-23 20:35:05 +00:00
|
|
|
{
|
2017-03-28 03:00:33 +00:00
|
|
|
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-03-28 03:00:33 +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-03-28 03:00:33 +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-03-28 03:00:33 +00:00
|
|
|
bool negative,
|
|
|
|
ConstNullMapPtr null_map) const
|
2015-03-02 01:11:37 +00:00
|
|
|
{
|
|
|
|
size_t rows = key_columns[0]->size();
|
|
|
|
|
2016-12-22 16:35:54 +00:00
|
|
|
switch (data.type)
|
|
|
|
{
|
|
|
|
case SetVariants::Type::EMPTY:
|
|
|
|
break;
|
2015-03-02 05:41:21 +00:00
|
|
|
#define M(NAME) \
|
2016-12-22 16:35:54 +00:00
|
|
|
case SetVariants::Type::NAME: \
|
2017-03-28 03:00:33 +00:00
|
|
|
executeImpl(*data.NAME, key_columns, vec_res, negative, rows, null_map); \
|
2016-12-22 16:35:54 +00:00
|
|
|
break;
|
2015-03-02 01:11:37 +00:00
|
|
|
APPLY_FOR_SET_VARIANTS(M)
|
|
|
|
#undef M
|
2016-12-22 16:35: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)
|
|
|
|
{
|
2020-11-10 18:22:26 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH,
|
|
|
|
"Number of columns in section IN doesn't match. {} at left, {} at right.",
|
|
|
|
num_key_columns, data_types.size());
|
2019-08-19 16:51:50 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-07-01 14:05:54 +00:00
|
|
|
bool Set::areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const
|
|
|
|
{
|
2022-09-02 03:56:13 +00:00
|
|
|
/// Out-of-bound access can happen when same set expression built with different columns.
|
|
|
|
/// Caller may call this method to make sure that the set is indeed the one they want
|
|
|
|
/// without awaring data_types.size().
|
|
|
|
if (set_type_idx >= data_types.size())
|
|
|
|
return false;
|
|
|
|
return removeNullable(recursiveRemoveLowCardinality(data_types[set_type_idx]))
|
|
|
|
->equals(*removeNullable(recursiveRemoveLowCardinality(other_type)));
|
2020-07-01 14:05:54 +00:00
|
|
|
}
|
|
|
|
|
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
|
|
|
|
2021-08-29 08:49:30 +00:00
|
|
|
MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<KeyTuplePositionMapping> && indexes_mapping_)
|
|
|
|
: has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_))
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2022-01-30 19:49:48 +00:00
|
|
|
::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
|
|
|
{
|
2021-01-02 09:47:38 +00:00
|
|
|
return std::tie(l.key_index, l.tuple_index) < std::tie(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)
|
|
|
|
{
|
2022-07-31 00:38:52 +00:00
|
|
|
String column_name = "_" + toString(i);
|
|
|
|
block_to_sort.insert({ordered_set[i], nullptr, column_name});
|
|
|
|
sort_description.emplace_back(column_name, 1, 1);
|
2016-04-08 20:34:32 +00:00
|
|
|
}
|
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
|
|
|
|
*/
|
2022-03-09 07:48:42 +00:00
|
|
|
BoolMask MergeTreeSetIndex::checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types, bool single_point) const
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2018-06-30 21:35:01 +00:00
|
|
|
size_t tuple_size = indexes_mapping.size();
|
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
FieldValues left_point;
|
|
|
|
FieldValues right_point;
|
2020-07-21 11:15:40 +00:00
|
|
|
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());
|
|
|
|
}
|
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
bool left_included = true;
|
|
|
|
bool right_included = true;
|
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)
|
2016-04-08 20:34:32 +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,
|
2022-03-09 07:48:42 +00:00
|
|
|
data_types[indexes_mapping[i].key_index],
|
|
|
|
single_point);
|
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
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
left_point[i].update(new_range->left);
|
|
|
|
left_included &= new_range->left_included;
|
|
|
|
right_point[i].update(new_range->right);
|
|
|
|
right_included &= new_range->right_included;
|
|
|
|
}
|
2018-02-08 15:31:37 +00:00
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
/// lhs < rhs return -1
|
|
|
|
/// lhs == rhs return 0
|
|
|
|
/// lhs > rhs return 1
|
|
|
|
auto compare = [](const IColumn & lhs, const FieldValue & rhs, size_t row)
|
|
|
|
{
|
|
|
|
if (rhs.isNegativeInfinity())
|
|
|
|
return 1;
|
|
|
|
if (rhs.isPositiveInfinity())
|
2018-02-02 13:19:40 +00:00
|
|
|
{
|
2021-01-02 09:47:38 +00:00
|
|
|
Field f;
|
|
|
|
lhs.get(row, f);
|
|
|
|
if (f.isNull())
|
|
|
|
return 0; // +Inf == +Inf
|
2018-02-08 15:31:37 +00:00
|
|
|
else
|
2021-01-02 09:47:38 +00:00
|
|
|
return -1;
|
2018-02-02 13:19:40 +00:00
|
|
|
}
|
2021-01-02 09:47:38 +00:00
|
|
|
return lhs.compareAt(row, 0, *rhs.column, 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
|
2021-01-02 09:47:38 +00:00
|
|
|
* laid out in the lexicographically increasing order, the set intersects the range
|
|
|
|
* if and only if either bound coincides with an element or at least one element
|
|
|
|
* is between the lower bounds
|
|
|
|
*/
|
2021-06-15 19:55:21 +00:00
|
|
|
auto indices = collections::range(0, size());
|
2020-02-26 14:21:56 +00:00
|
|
|
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
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
/// A special case of 1-element KeyRange. It's useful for partition pruning.
|
2021-06-03 02:46:01 +00:00
|
|
|
bool one_element_range = true;
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
{
|
|
|
|
auto & left = left_point[i];
|
|
|
|
auto & right = right_point[i];
|
2021-01-02 09:47:38 +00:00
|
|
|
if (left.isNormal() && right.isNormal())
|
2021-06-03 02:46:01 +00:00
|
|
|
{
|
2021-01-02 09:47:38 +00:00
|
|
|
if (0 != left.column->compareAt(0, 0, *right.column, 1))
|
2021-06-03 02:46:01 +00:00
|
|
|
{
|
2021-01-02 09:47:38 +00:00
|
|
|
one_element_range = false;
|
|
|
|
break;
|
2021-06-03 02:46:01 +00:00
|
|
|
}
|
|
|
|
}
|
2021-06-19 06:41:37 +00:00
|
|
|
else if ((left.isPositiveInfinity() && right.isPositiveInfinity()) || (left.isNegativeInfinity() && right.isNegativeInfinity()))
|
|
|
|
{
|
|
|
|
/// Special value equality.
|
|
|
|
}
|
2021-06-03 02:46:01 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
one_element_range = false;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2021-08-29 08:49:30 +00:00
|
|
|
if (one_element_range && has_all_keys)
|
2021-06-03 02:46:01 +00:00
|
|
|
{
|
2021-06-07 09:36:34 +00:00
|
|
|
/// Here we know that there is one element in range.
|
|
|
|
/// The main difference with the normal case is that we can definitely say that
|
2021-08-29 08:49:30 +00:00
|
|
|
/// condition in this range is always TRUE (can_be_false = 0) or always FALSE (can_be_true = 0).
|
2021-06-19 06:41:37 +00:00
|
|
|
|
|
|
|
/// Check if it's an empty range
|
|
|
|
if (!left_included || !right_included)
|
|
|
|
return {false, true};
|
|
|
|
else if (left_lower != indices.end() && equals(*left_lower, left_point))
|
2021-06-03 02:46:01 +00:00
|
|
|
return {true, false};
|
|
|
|
else
|
|
|
|
return {false, true};
|
|
|
|
}
|
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
/// If there are more than one element in the range, it can always be false. Thus we only need to check if it may be true or not.
|
2021-06-19 06:41:37 +00:00
|
|
|
/// Given left_lower >= left_point, right_lower >= right_point, find if there may be a match in between left_lower and right_lower.
|
|
|
|
if (left_lower + 1 < right_lower)
|
2018-06-30 21:35:01 +00:00
|
|
|
{
|
2021-06-19 06:41:37 +00:00
|
|
|
/// There is an point in between: left_lower + 1
|
|
|
|
return {true, true};
|
|
|
|
}
|
|
|
|
else if (left_lower + 1 == right_lower)
|
|
|
|
{
|
|
|
|
/// Need to check if left_lower is a valid match, as left_point <= left_lower < right_point <= right_lower.
|
|
|
|
/// Note: left_lower is valid.
|
|
|
|
if (left_included || !equals(*left_lower, left_point))
|
|
|
|
return {true, true};
|
2021-01-02 09:47:38 +00:00
|
|
|
|
2021-06-19 06:41:37 +00:00
|
|
|
/// We are unlucky that left_point fails to cover a point. Now we need to check if right_point can cover right_lower.
|
|
|
|
/// Check if there is a match at the right boundary.
|
|
|
|
return {right_included && right_lower != indices.end() && equals(*right_lower, right_point), true};
|
|
|
|
}
|
|
|
|
else // left_lower == right_lower
|
|
|
|
{
|
|
|
|
/// Need to check if right_point is a valid match, as left_point < right_point <= left_lower = right_lower.
|
|
|
|
/// Check if there is a match at the left boundary.
|
|
|
|
return {right_included && right_lower != indices.end() && equals(*right_lower, right_point), 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;
|
|
|
|
}
|
|
|
|
|
2021-01-02 09:47:38 +00:00
|
|
|
void FieldValue::update(const Field & x)
|
2020-02-26 14:21:56 +00:00
|
|
|
{
|
2021-01-02 09:47:38 +00:00
|
|
|
if (x.isNegativeInfinity() || x.isPositiveInfinity())
|
|
|
|
value = x;
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Keep at most one element in column.
|
|
|
|
if (!column->empty())
|
|
|
|
column->popBack(1);
|
|
|
|
column->insert(x);
|
|
|
|
value = Field(); // Set back to normal value.
|
|
|
|
}
|
2020-02-26 14:21:56 +00:00
|
|
|
}
|
|
|
|
|
2012-08-23 20:22:44 +00:00
|
|
|
}
|