ClickHouse/dbms/src/Interpreters/Set.cpp

714 lines
23 KiB
C++
Raw Normal View History

2012-08-23 20:22:44 +00:00
#include <DB/Core/Field.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnsNumber.h>
2012-08-23 23:49:28 +00:00
#include <DB/DataStreams/IProfilingBlockInputStream.h>
2012-08-24 19:42:03 +00:00
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTLiteral.h>
2012-08-23 23:49:28 +00:00
2012-08-23 20:22:44 +00:00
#include <DB/Interpreters/Set.h>
#include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
2012-08-23 20:22:44 +00:00
namespace DB
{
void SetVariants::init(Type type_)
{
type = type_;
switch (type)
{
case Type::EMPTY: break;
#define M(NAME) \
case Type::NAME: NAME.reset(new decltype(NAME)::element_type); break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
size_t SetVariants::getTotalRowCount() const
{
switch (type)
{
case Type::EMPTY: return 0;
#define M(NAME) \
case Type::NAME: return NAME->data.size();
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
size_t SetVariants::getTotalByteCount() const
{
switch (type)
{
case Type::EMPTY: return 0;
#define M(NAME) \
case Type::NAME: return NAME->data.getBufferSizeInBytes();
APPLY_FOR_SET_VARIANTS(M)
#undef M
default:
throw Exception("Unknown Set variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
}
}
bool Set::checkSetSizeLimits() const
{
if (max_rows && data.getTotalRowCount() > max_rows)
return false;
if (max_bytes && data.getTotalByteCount() > max_bytes)
return false;
return true;
}
2012-08-23 20:22:44 +00:00
SetVariants::Type SetVariants::chooseMethod(const ConstColumnPlainPtrs & key_columns, Sizes & key_sizes)
2012-08-23 20:22:44 +00:00
{
2013-07-19 19:53:51 +00:00
size_t keys_size = key_columns.size();
2012-08-23 20:22:44 +00:00
bool all_fixed = true;
2012-08-23 20:22:44 +00:00
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isFixed())
2012-08-23 20:22:44 +00:00
{
all_fixed = false;
2012-08-23 20:22:44 +00:00
break;
}
2013-07-19 19:53:51 +00:00
key_sizes[j] = key_columns[j]->sizeOfField();
2012-08-23 20:22:44 +00:00
keys_bytes += key_sizes[j];
}
/// Если есть один числовой ключ, который помещается в 64 бита
2013-07-19 19:53:51 +00:00
if (keys_size == 1 && key_columns[0]->isNumeric())
{
size_t size_of_field = key_columns[0]->sizeOfField();
if (size_of_field == 1)
return SetVariants::Type::key8;
if (size_of_field == 2)
return SetVariants::Type::key16;
if (size_of_field == 4)
return SetVariants::Type::key32;
if (size_of_field == 8)
return SetVariants::Type::key64;
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8.", ErrorCodes::LOGICAL_ERROR);
}
/// Если ключи помещаются в N бит, будем использовать хэш-таблицу по упакованным в N-бит ключам
if (all_fixed && keys_bytes <= 16)
return SetVariants::Type::keys128;
if (all_fixed && keys_bytes <= 32)
return SetVariants::Type::keys256;
2012-08-23 20:22:44 +00:00
/// Если есть один строковый ключ, то используем хэш-таблицу с ним
if (keys_size == 1 && (typeid_cast<const ColumnString *>(key_columns[0]) || typeid_cast<const ColumnConstString *>(key_columns[0])))
return SetVariants::Type::key_string;
if (keys_size == 1 && typeid_cast<const ColumnFixedString *>(key_columns[0]))
return SetVariants::Type::key_fixed_string;
/// Иначе будем агрегировать по конкатенации ключей.
return SetVariants::Type::hashed;
}
template <typename Method>
void NO_INLINE Set::insertFromBlockImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
size_t rows,
SetVariants & variants)
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
{
/// Строим ключ
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
typename Method::Data::iterator it = method.data.find(key);
bool inserted;
method.data.emplace(key, it, inserted);
if (inserted)
method.onNewKey(*it, keys_size, i, variants.string_pool);
}
2012-08-23 20:22:44 +00:00
}
2015-01-27 00:52:03 +00:00
bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
2012-08-23 20:22:44 +00:00
{
2015-01-27 00:52:03 +00:00
Poco::ScopedWriteRWLock lock(rwlock);
size_t keys_size = block.columns();
ConstColumnPlainPtrs key_columns(keys_size);
data_types.resize(keys_size);
/// Константные столбцы справа от IN поддерживается не напрямую. Для этого, они сначала материализуется.
Columns materialized_columns;
/// Запоминаем столбцы, с которыми будем работать
for (size_t i = 0; i < keys_size; ++i)
{
key_columns[i] = block.getByPosition(i).column;
data_types[i] = block.getByPosition(i).type;
if (key_columns[i]->isConst())
{
materialized_columns.emplace_back(static_cast<const IColumnConst *>(key_columns[i])->convertToFullColumn());
key_columns[i] = materialized_columns.back().get();
}
}
2012-08-23 20:22:44 +00:00
size_t rows = block.rows();
/// Какую структуру данных для множества использовать?
2014-05-01 15:02:36 +00:00
if (empty())
data.init(data.chooseMethod(key_columns, key_sizes));
if (false) {}
#define M(NAME) \
else if (data.type == SetVariants::Type::NAME) \
insertFromBlockImpl(*data.NAME, key_columns, rows, data);
APPLY_FOR_SET_VARIANTS(M)
#undef M
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
2012-08-23 23:49:28 +00:00
if (create_ordered_set)
for (size_t i = 0; i < rows; ++i)
ordered_set_elements->push_back((*key_columns[0])[i]); /// ordered_set для индекса работает только если IN одному ключу.
if (!checkSetSizeLimits())
2012-08-23 23:49:28 +00:00
{
if (overflow_mode == OverflowMode::THROW)
throw Exception("IN-set size exceeded."
" Rows: " + toString(data.getTotalRowCount()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(data.getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
if (overflow_mode == OverflowMode::BREAK)
return false;
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
2012-08-23 23:49:28 +00:00
}
return true;
2012-08-23 20:22:44 +00:00
}
/** Чтобы корректно работали выражения вида 1.0 IN (1).
* Проверяет совместимость типов, проверяет попадание значений в диапазон допустимых значений типа, делает преобразование типа.
* Код слегка дурацкий.
*/
static Field convertToType(const Field & src, const IDataType & type)
{
if (type.isNumeric())
{
bool is_uint8 = false;
bool is_uint16 = false;
bool is_uint32 = false;
bool is_uint64 = false;
bool is_int8 = false;
bool is_int16 = false;
bool is_int32 = false;
bool is_int64 = false;
bool is_float32 = false;
bool is_float64 = false;
bool is_date = false;
bool is_datetime = false;
false
|| (is_uint8 = typeid_cast<const DataTypeUInt8 * >(&type))
|| (is_uint16 = typeid_cast<const DataTypeUInt16 * >(&type))
|| (is_uint32 = typeid_cast<const DataTypeUInt32 * >(&type))
|| (is_uint64 = typeid_cast<const DataTypeUInt64 * >(&type))
|| (is_int8 = typeid_cast<const DataTypeInt8 * >(&type))
|| (is_int16 = typeid_cast<const DataTypeInt16 * >(&type))
|| (is_int32 = typeid_cast<const DataTypeInt32 * >(&type))
|| (is_int64 = typeid_cast<const DataTypeInt64 * >(&type))
|| (is_float32 = typeid_cast<const DataTypeFloat32 * >(&type))
|| (is_float64 = typeid_cast<const DataTypeFloat64 * >(&type))
|| (is_date = typeid_cast<const DataTypeDate * >(&type))
|| (is_datetime = typeid_cast<const DataTypeDateTime * >(&type))
;
if (is_uint8 || is_uint16 || is_uint32 || is_uint64)
{
if (src.getType() == Field::Types::Int64)
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, signed at right");
if (src.getType() == Field::Types::Float64)
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right");
if (src.getType() == Field::Types::UInt64)
{
UInt64 value = src.get<const UInt64 &>();
if ((is_uint8 && value > std::numeric_limits<uint8_t>::max())
|| (is_uint16 && value > std::numeric_limits<uint16_t>::max())
|| (is_uint32 && value > std::numeric_limits<uint32_t>::max()))
throw Exception("Value (" + toString(value) + ") in IN section is out of range of type " + type.getName() + " at left");
return src;
}
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
+ Field::Types::toString(src.getType()) + " at right");
}
else if (is_int8 || is_int16 || is_int32 || is_int64)
{
if (src.getType() == Field::Types::Float64)
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right");
if (src.getType() == Field::Types::UInt64)
{
UInt64 value = src.get<const UInt64 &>();
if ((is_int8 && value > uint8_t(std::numeric_limits<int8_t>::max()))
|| (is_int16 && value > uint16_t(std::numeric_limits<int16_t>::max()))
|| (is_int32 && value > uint32_t(std::numeric_limits<int32_t>::max()))
|| (is_int64 && value > uint64_t(std::numeric_limits<int64_t>::max())))
throw Exception("Value (" + toString(value) + ") in IN section is out of range of type " + type.getName() + " at left");
return Field(Int64(value));
}
if (src.getType() == Field::Types::Int64)
{
Int64 value = src.get<const Int64 &>();
if ((is_int8 && (value < std::numeric_limits<int8_t>::min() || value > std::numeric_limits<int8_t>::max()))
|| (is_int16 && (value < std::numeric_limits<int16_t>::min() || value > std::numeric_limits<int16_t>::max()))
|| (is_int32 && (value < std::numeric_limits<int32_t>::min() || value > std::numeric_limits<int32_t>::max())))
throw Exception("Value (" + toString(value) + ") in IN section is out of range of type " + type.getName() + " at left");
return src;
}
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
+ Field::Types::toString(src.getType()) + " at right");
}
else if (is_float32 || is_float64)
{
if (src.getType() == Field::Types::UInt64)
return Field(Float64(src.get<UInt64>()));
if (src.getType() == Field::Types::Int64)
return Field(Float64(src.get<Int64>()));
if (src.getType() == Field::Types::Float64)
return src;
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
+ Field::Types::toString(src.getType()) + " at right");
}
else if (is_date || is_datetime)
{
if (src.getType() == Field::Types::UInt64)
return src;
if (src.getType() == Field::Types::String)
{
/// Возможность сравнивать даты и даты-с-временем со строкой.
const String & str = src.get<const String &>();
ReadBufferFromString in(str);
if (is_date)
{
DayNum_t date{};
readDateText(date, in);
if (!in.eof())
throw Exception("String is too long for Date: " + str);
return Field(UInt64(date));
}
else
{
time_t date_time{};
readDateTimeText(date_time, in);
if (!in.eof())
throw Exception("String is too long for DateTime: " + str);
return Field(UInt64(date_time));
}
}
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
+ Field::Types::toString(src.getType()) + " at right");
}
}
else
{
if (src.getType() == Field::Types::UInt64
|| src.getType() == Field::Types::Int64
|| src.getType() == Field::Types::Float64
|| src.getType() == Field::Types::Null
|| (src.getType() == Field::Types::String
&& !typeid_cast<const DataTypeString *>(&type)
&& !typeid_cast<const DataTypeFixedString *>(&type))
|| (src.getType() == Field::Types::Array
&& !typeid_cast<const DataTypeArray *>(&type)))
throw Exception("Type mismatch in IN section: " + type.getName() + " at left, "
+ Field::Types::toString(src.getType()) + " at right");
}
return src;
}
/** Выполнить константное выражение (для элемента множества в IN). Весьма неоптимально. */
static Field evaluateConstantExpression(ASTPtr & node, const Context & context)
{
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(
node, context, NamesAndTypesList{{ "_dummy", new DataTypeUInt8 }}).getConstActions();
/// В блоке должен быть хотя бы один столбец, чтобы у него было известно число строк.
Block block_with_constants{{ new ColumnConstUInt8(1, 0), new DataTypeUInt8, "_dummy" }};
expr_for_constant_folding->execute(block_with_constants);
if (!block_with_constants || block_with_constants.rows() == 0)
throw Exception("Logical error: empty block after evaluation constant expression for IN", ErrorCodes::LOGICAL_ERROR);
String name = node->getColumnName();
if (!block_with_constants.has(name))
throw Exception("Element of set in IN is not a constant expression: " + name, ErrorCodes::BAD_ARGUMENTS);
const IColumn & result_column = *block_with_constants.getByName(name).column;
if (!result_column.isConst())
throw Exception("Element of set in IN is not a constant expression: " + name, ErrorCodes::BAD_ARGUMENTS);
return result_column[0];
}
static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const Context & context)
2012-08-24 19:42:03 +00:00
{
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(node.get()))
return convertToType(lit->value, type);
else if (typeid_cast<ASTFunction *>(node.get()))
return convertToType(evaluateConstantExpression(node, context), type);
else
throw Exception("Incorrect element of set. Must be literal or constant expression.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
}
void Set::createFromAST(DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set)
{
2012-08-24 19:42:03 +00:00
data_types = types;
/// Засунем множество в блок.
Block block;
for (size_t i = 0, size = data_types.size(); i < size; ++i)
{
ColumnWithNameAndType col;
col.type = data_types[i];
col.column = data_types[i]->createColumn();
2013-06-21 20:34:19 +00:00
col.name = "_" + toString(i);
2012-08-24 19:42:03 +00:00
block.insert(col);
}
ASTExpressionList & list = typeid_cast<ASTExpressionList &>(*node);
2012-08-24 19:42:03 +00:00
for (ASTs::iterator it = list.children.begin(); it != list.children.end(); ++it)
{
if (data_types.size() == 1)
{
block.getByPosition(0).column->insert(extractValueFromNode(*it, *data_types[0], context));
2012-08-24 19:42:03 +00:00
}
else if (ASTFunction * func = typeid_cast<ASTFunction *>(&**it))
2012-08-24 19:42:03 +00:00
{
if (func->name != "tuple")
throw Exception("Incorrect element of set. Must be tuple.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
size_t tuple_size = func->arguments->children.size();
if (tuple_size != data_types.size())
throw Exception("Incorrect size of tuple in set.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
2012-08-24 19:42:03 +00:00
for (size_t j = 0; j < tuple_size; ++j)
{
block.getByPosition(j).column->insert(extractValueFromNode(func->arguments->children[j], *data_types[j], context));
2012-08-24 19:42:03 +00:00
}
}
else
throw Exception("Incorrect element of set", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
}
2014-04-01 10:09:22 +00:00
if (create_ordered_set)
2014-04-08 12:54:32 +00:00
ordered_set_elements = OrderedSetElementsPtr(new OrderedSetElements());
2014-04-01 10:09:22 +00:00
insertFromBlock(block, create_ordered_set);
if (create_ordered_set)
2014-04-08 12:54:32 +00:00
std::sort(ordered_set_elements->begin(), ordered_set_elements->end());
2012-08-24 19:42:03 +00:00
}
2012-08-23 20:22:44 +00:00
void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result, bool negative) const
{
ColumnUInt8 * c_res = new ColumnUInt8;
block.getByPosition(result).column = c_res;
ColumnUInt8::Container_t & vec_res = c_res->getData();
vec_res.resize(block.getByPosition(arguments[0]).column->size());
2012-08-23 20:35:05 +00:00
2015-01-27 00:52:03 +00:00
Poco::ScopedReadRWLock lock(rwlock);
2012-08-24 20:40:34 +00:00
/// Если множество пусто
if (data_types.empty())
{
if (negative)
memset(&vec_res[0], 1, vec_res.size());
else
memset(&vec_res[0], 0, vec_res.size());
2012-08-24 20:40:34 +00:00
return;
}
DataTypeArray * array_type = typeid_cast<DataTypeArray *>(&*block.getByPosition(arguments[0]).type);
if (array_type)
2012-08-23 22:40:51 +00:00
{
if (data_types.size() != 1 || arguments.size() != 1)
throw Exception("Number of columns in section IN doesn't match.", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
if (array_type->getNestedType()->getName() != data_types[0]->getName())
throw Exception(std::string() + "Types in section IN don't match: " + data_types[0]->getName() + " on the right, " + array_type->getNestedType()->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
IColumn * in_column = &*block.getByPosition(arguments[0]).column;
/// Константный столбец слева от IN поддерживается не напрямую. Для этого, он сначала материализуется.
ColumnPtr materialized_column;
if (in_column->isConst())
{
materialized_column = static_cast<const IColumnConst *>(in_column)->convertToFullColumn();
in_column = materialized_column.get();
}
if (ColumnArray * col = typeid_cast<ColumnArray *>(in_column))
executeArray(col, vec_res, negative);
else
throw Exception("Unexpected array column type: " + in_column->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
else
{
if (data_types.size() != arguments.size())
throw Exception("Number of columns in section IN doesn't match.", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
/// Запоминаем столбцы, с которыми будем работать. Также проверим, что типы данных правильные.
ConstColumnPlainPtrs key_columns(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
{
key_columns[i] = block.getByPosition(arguments[i]).column;
if (data_types[i]->getName() != block.getByPosition(arguments[i]).type->getName())
2013-06-21 20:34:19 +00:00
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.getByPosition(arguments[i]).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH);
}
/// Константные столбцы слева от IN поддерживается не напрямую. Для этого, они сначала материализуется.
Columns materialized_columns;
for (auto & column_ptr : key_columns)
{
if (column_ptr->isConst())
{
materialized_columns.emplace_back(static_cast<const IColumnConst *>(column_ptr)->convertToFullColumn());
column_ptr = materialized_columns.back().get();
}
}
executeOrdinary(key_columns, vec_res, negative);
2012-08-23 22:40:51 +00:00
}
}
2012-08-23 22:40:51 +00:00
template <typename Method>
void NO_INLINE Set::executeImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
ColumnUInt8::Container_t & vec_res,
bool negative,
size_t rows) const
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
2012-08-23 20:35:05 +00:00
/// NOTE Не используется оптимизация для подряд идущих одинаковых значений.
/// Для всех строчек
for (size_t i = 0; i < rows; ++i)
2012-08-23 20:35:05 +00:00
{
/// Строим ключ
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
vec_res[i] = negative ^ (method.data.end() != method.data.find(key));
}
}
template <typename Method>
void NO_INLINE Set::executeArrayImpl(
Method & method,
const ConstColumnPlainPtrs & key_columns,
const ColumnArray::Offsets_t & offsets,
ColumnUInt8::Container_t & 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 (size_t i = 0; i < rows; ++i)
{
UInt8 res = 0;
/// Для всех элементов
for (size_t j = prev_offset; j < offsets[i]; ++j)
2012-08-24 19:42:03 +00:00
{
/// Строим ключ
2015-03-03 21:11:54 +00:00
typename Method::Key key = state.getKey(key_columns, keys_size, j, key_sizes);
res |= negative ^ (method.data.end() != method.data.find(key));
if (res)
break;
2012-08-24 19:42:03 +00:00
}
vec_res[i] = res;
prev_offset = offsets[i];
2012-08-23 20:35:05 +00:00
}
}
void Set::executeOrdinary(const ConstColumnPlainPtrs & key_columns, ColumnUInt8::Container_t & vec_res, bool negative) const
{
size_t rows = key_columns[0]->size();
if (false) {}
#define M(NAME) \
else if (data.type == SetVariants::Type::NAME) \
executeImpl(*data.NAME, key_columns, vec_res, negative, rows);
APPLY_FOR_SET_VARIANTS(M)
#undef M
2012-08-23 20:35:05 +00:00
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
2012-08-23 20:22:44 +00:00
}
void Set::executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t & vec_res, bool negative) const
{
size_t rows = key_column->size();
const ColumnArray::Offsets_t & offsets = key_column->getOffsets();
const IColumn & nested_column = key_column->getData();
if (false) {}
#define M(NAME) \
else if (data.type == SetVariants::Type::NAME) \
2015-03-03 21:11:54 +00:00
executeArrayImpl(*data.NAME, ConstColumnPlainPtrs{&nested_column}, offsets, vec_res, negative, rows);
APPLY_FOR_SET_VARIANTS(M)
#undef M
else
throw Exception("Unknown set variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
}
2015-03-27 03:37:46 +00:00
BoolMask Set::mayBeTrueInRange(const Range & range) const
2014-04-01 10:09:22 +00:00
{
2014-04-08 12:54:32 +00:00
if (!ordered_set_elements)
2014-04-01 10:09:22 +00:00
throw DB::Exception("Ordered set in not created.");
2014-04-08 12:54:32 +00:00
if (ordered_set_elements->empty())
2014-04-01 10:09:22 +00:00
return BoolMask(false, true);
const Field & left = range.left;
const Field & right = range.right;
bool can_be_true;
bool can_be_false = true;
/// Если во всем диапазоне одинаковый ключ и он есть в Set, то выбираем блок для in и не выбираем для notIn
if (range.left_bounded && range.right_bounded && range.right_included && range.left_included && left == right)
{
2014-04-18 14:24:06 +00:00
if (std::binary_search(ordered_set_elements->begin(), ordered_set_elements->end(), left))
2014-04-01 10:09:22 +00:00
{
can_be_false = false;
can_be_true = true;
}
else
{
can_be_true = false;
can_be_false = true;
}
}
else
{
2015-03-27 03:37:46 +00:00
auto left_it = range.left_bounded
? std::lower_bound(ordered_set_elements->begin(), ordered_set_elements->end(), left)
: ordered_set_elements->begin();
2014-04-08 12:54:32 +00:00
if (range.left_bounded && !range.left_included && left_it != ordered_set_elements->end() && *left_it == left)
2014-04-01 10:09:22 +00:00
++left_it;
/// если весь диапазон, правее in
2014-04-08 12:54:32 +00:00
if (left_it == ordered_set_elements->end())
2014-04-01 10:09:22 +00:00
{
can_be_true = false;
}
else
{
2015-03-27 03:37:46 +00:00
auto right_it = range.right_bounded
? std::upper_bound(ordered_set_elements->begin(), ordered_set_elements->end(), right)
: ordered_set_elements->end();
2014-04-08 12:54:32 +00:00
if (range.right_bounded && !range.right_included && right_it != ordered_set_elements->begin() && *(right_it--) == right)
2014-04-01 10:09:22 +00:00
--right_it;
/// весь диапазон, левее in
2014-04-08 12:54:32 +00:00
if (right_it == ordered_set_elements->begin())
2014-04-01 10:09:22 +00:00
{
can_be_true = false;
}
else
{
--right_it;
/// в диапазон не попадает ни одного ключа из in
if (*right_it < *left_it)
can_be_true = false;
else
can_be_true = true;
}
}
}
return BoolMask(can_be_true, can_be_false);
}
2012-08-23 20:22:44 +00:00
}