mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-11 01:54:55 +00:00
Removed (obsolete) special handling for Arrays in the left hand side of IN [#CLICKHOUSE-2]
This commit is contained in:
parent
d9b54a6aae
commit
6452a3bb76
@ -10,6 +10,7 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Common/UnicodeBar.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
@ -41,6 +42,8 @@ namespace ErrorCodes
|
||||
extern const int FUNCTION_IS_SPECIAL;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int TOO_SLOW;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO;
|
||||
}
|
||||
|
||||
|
@ -67,6 +67,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS;
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
extern const int CYCLIC_ALIASES;
|
||||
|
@ -9,7 +9,6 @@
|
||||
|
||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
@ -282,78 +281,44 @@ ColumnPtr Set::execute(const Block & block, bool negative) const
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(block.safeGetByPosition(0).type.get());
|
||||
|
||||
if (array_type)
|
||||
if (data_types.size() != num_key_columns)
|
||||
{
|
||||
/// Special treatment of Arrays in left hand side of IN:
|
||||
/// check that at least one array element is in Set.
|
||||
/// This is deprecated functionality and will be removed.
|
||||
|
||||
if (data_types.size() != 1 || num_key_columns != 1)
|
||||
throw Exception("Number of columns in section IN doesn't match.", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
if (array_type->getNestedType()->isNullable())
|
||||
throw Exception("Array(Nullable(...)) for left hand side of IN is not supported.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
if (!array_type->getNestedType()->equals(*data_types[0]))
|
||||
throw Exception("Types in section IN don't match: " + data_types[0]->getName() +
|
||||
" on the right, " + array_type->getNestedType()->getName() + " on the left.",
|
||||
ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
const IColumn * in_column = block.safeGetByPosition(0).column.get();
|
||||
|
||||
/// The constant column to the left of IN is not supported directly. For this, it first materializes.
|
||||
ColumnPtr materialized_column = in_column->convertToFullColumnIfConst();
|
||||
if (materialized_column)
|
||||
in_column = materialized_column.get();
|
||||
|
||||
if (const ColumnArray * col = typeid_cast<const ColumnArray *>(in_column))
|
||||
executeArray(col, vec_res, negative);
|
||||
else
|
||||
throw Exception("Unexpected array column type: " + in_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
std::stringstream message;
|
||||
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);
|
||||
}
|
||||
else
|
||||
|
||||
/// Remember the columns we will work with. Also check that the data types are correct.
|
||||
ColumnRawPtrs key_columns;
|
||||
key_columns.reserve(num_key_columns);
|
||||
|
||||
/// The constant columns to the left of IN are not supported directly. For this, they first materialize.
|
||||
Columns materialized_columns;
|
||||
|
||||
for (size_t i = 0; i < num_key_columns; ++i)
|
||||
{
|
||||
if (data_types.size() != num_key_columns)
|
||||
key_columns.push_back(block.safeGetByPosition(i).column.get());
|
||||
|
||||
if (!removeNullable(data_types[i])->equals(*removeNullable(block.safeGetByPosition(i).type)))
|
||||
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: "
|
||||
+ data_types[i]->getName() + " on the right, " + block.safeGetByPosition(i).type->getName() +
|
||||
" on the left.", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst())
|
||||
{
|
||||
std::stringstream message;
|
||||
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);
|
||||
materialized_columns.emplace_back(converted);
|
||||
key_columns.back() = materialized_columns.back().get();
|
||||
}
|
||||
|
||||
/// Remember the columns we will work with. Also check that the data types are correct.
|
||||
ColumnRawPtrs key_columns;
|
||||
key_columns.reserve(num_key_columns);
|
||||
|
||||
/// The constant columns to the left of IN are not supported directly. For this, they first materialize.
|
||||
Columns materialized_columns;
|
||||
|
||||
for (size_t i = 0; i < num_key_columns; ++i)
|
||||
{
|
||||
key_columns.push_back(block.safeGetByPosition(i).column.get());
|
||||
|
||||
if (!removeNullable(data_types[i])->equals(*removeNullable(block.safeGetByPosition(i).type)))
|
||||
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: "
|
||||
+ data_types[i]->getName() + " on the right, " + block.safeGetByPosition(i).type->getName() +
|
||||
" on the left.", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst())
|
||||
{
|
||||
materialized_columns.emplace_back(converted);
|
||||
key_columns.back() = materialized_columns.back().get();
|
||||
}
|
||||
}
|
||||
|
||||
/// We will check existence in Set only for keys, where all components are not NULL.
|
||||
ColumnPtr null_map_holder;
|
||||
ConstNullMapPtr null_map{};
|
||||
extractNestedColumnsAndNullMap(key_columns, null_map_holder, null_map);
|
||||
|
||||
executeOrdinary(key_columns, vec_res, negative, null_map);
|
||||
}
|
||||
|
||||
/// We will check existence in Set only for keys, where all components are not NULL.
|
||||
ColumnPtr null_map_holder;
|
||||
ConstNullMapPtr null_map{};
|
||||
extractNestedColumnsAndNullMap(key_columns, null_map_holder, null_map);
|
||||
|
||||
executeOrdinary(key_columns, vec_res, negative, null_map);
|
||||
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
@ -403,38 +368,6 @@ void NO_INLINE Set::executeImplCase(
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Method>
|
||||
void NO_INLINE Set::executeArrayImpl(
|
||||
Method & method,
|
||||
const ColumnRawPtrs & key_columns,
|
||||
const ColumnArray::Offsets & offsets,
|
||||
ColumnUInt8::Container & vec_res,
|
||||
bool negative,
|
||||
size_t rows) const
|
||||
{
|
||||
typename Method::State state;
|
||||
state.init(key_columns);
|
||||
size_t keys_size = key_columns.size();
|
||||
|
||||
size_t prev_offset = 0;
|
||||
/// For all rows
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
UInt8 res = 0;
|
||||
/// For all elements
|
||||
for (size_t j = prev_offset; j < offsets[i]; ++j)
|
||||
{
|
||||
/// Build the key
|
||||
typename Method::Key key = state.getKey(key_columns, keys_size, j, key_sizes);
|
||||
res |= negative ^ method.data.has(key);
|
||||
if (res)
|
||||
break;
|
||||
}
|
||||
vec_res[i] = res;
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void Set::executeOrdinary(
|
||||
const ColumnRawPtrs & key_columns,
|
||||
@ -457,25 +390,6 @@ void Set::executeOrdinary(
|
||||
}
|
||||
}
|
||||
|
||||
void Set::executeArray(const ColumnArray * key_column, ColumnUInt8::Container & vec_res, bool negative) const
|
||||
{
|
||||
size_t rows = key_column->size();
|
||||
const ColumnArray::Offsets & offsets = key_column->getOffsets();
|
||||
const IColumn & nested_column = key_column->getData();
|
||||
|
||||
switch (data.type)
|
||||
{
|
||||
case SetVariants::Type::EMPTY:
|
||||
break;
|
||||
#define M(NAME) \
|
||||
case SetVariants::Type::NAME: \
|
||||
executeArrayImpl(*data.NAME, ColumnRawPtrs{&nested_column}, offsets, vec_res, negative, rows); \
|
||||
break;
|
||||
APPLY_FOR_SET_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vector<PKTuplePositionMapping> && index_mapping_)
|
||||
: ordered_set(),
|
||||
@ -507,6 +421,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vect
|
||||
std::sort(ordered_set.begin(), ordered_set.end());
|
||||
}
|
||||
|
||||
|
||||
/** 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
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <shared_mutex>
|
||||
#include <Core/Block.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Interpreters/SetVariants.h>
|
||||
@ -89,9 +88,6 @@ private:
|
||||
/// Limitations on the maximum size of the set
|
||||
SizeLimits limits;
|
||||
|
||||
/// 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,
|
||||
@ -143,15 +139,6 @@ private:
|
||||
bool negative,
|
||||
size_t rows,
|
||||
ConstNullMapPtr null_map) const;
|
||||
|
||||
template <typename Method>
|
||||
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<Set>;
|
||||
|
Loading…
Reference in New Issue
Block a user