Removed (obsolete) special handling for Arrays in the left hand side of IN [#CLICKHOUSE-2]

This commit is contained in:
Alexey Milovidov 2018-04-05 23:52:01 +03:00
parent d9b54a6aae
commit 6452a3bb76
4 changed files with 36 additions and 130 deletions

View File

@ -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;
}

View File

@ -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;

View File

@ -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

View File

@ -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>;