Merge pull request #192 from ludv1x/enums-fix

Fixed Enum usage in PK expressions.
This commit is contained in:
alexey-milovidov 2016-11-24 03:13:16 +04:00 committed by GitHub
commit fdabcd4e38
12 changed files with 275 additions and 42 deletions

View File

@ -250,9 +250,15 @@ public:
}; };
/// Преобразование строки с датой или датой-с-временем в UInt64, содержащим числовое значение даты или даты-с-временем. /// Converts string with date or datetime (in format 'YYYY-MM-DD hh:mm:ss') to UInt64 containing numeric value of date (or datetime)
UInt64 stringToDateOrDateTime(const String & s); UInt64 stringToDateOrDateTime(const String & s);
/// Converts string with date to UInt16 (which is alias of DayNum_t) containing numeric value of date
DayNum_t stringToDate(const String & s);
/// Converts string with date to UInt64 containing numeric value of datetime
UInt64 stringToDateTime(const String & s);
/** More precise comparison, used for index. /** More precise comparison, used for index.
* Differs from Field::operator< and Field::operator== in that it also compares values of different types. * Differs from Field::operator< and Field::operator== in that it also compares values of different types.

View File

@ -16,9 +16,17 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
class IDataTypeEnum : public IDataType
{
public:
virtual Field castToName(const Field & value_or_name) const = 0;
virtual Field castToValue(const Field & value_or_name) const = 0;
};
template <typename Type> template <typename Type>
class DataTypeEnum final : public IDataType class DataTypeEnum final : public IDataTypeEnum
{ {
public: public:
using FieldType = Type; using FieldType = Type;
@ -70,6 +78,10 @@ public:
return it->second; return it->second;
} }
Field castToName(const Field & value_or_name) const override;
Field castToValue(const Field & value_or_name) const override;
DataTypePtr clone() const override; DataTypePtr clone() const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override;

View File

@ -10,4 +10,10 @@ namespace DB
using StringRange = std::pair<const char *, const char *>; using StringRange = std::pair<const char *, const char *>;
using StringPtr = std::shared_ptr<String>; using StringPtr = std::shared_ptr<String>;
inline String toString(const StringRange & range)
{
return String(range.first, range.second);
}
} }

View File

@ -223,7 +223,8 @@ public:
static const AtomMap atom_map; static const AtomMap atom_map;
/// Не учитывает секцию SAMPLE. all_columns - набор всех столбцов таблицы. /// Не учитывает секцию SAMPLE. all_columns - набор всех столбцов таблицы.
PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr); PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr,
const Block & pk_sample_block);
/// Выполнимо ли условие в диапазоне ключей. /// Выполнимо ли условие в диапазоне ключей.
/// left_pk и right_pk должны содержать все поля из sort_descr в соответствующем порядке. /// left_pk и right_pk должны содержать все поля из sort_descr в соответствующем порядке.
@ -312,26 +313,30 @@ private:
bool atomFromAST(ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out); bool atomFromAST(ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
bool operatorFromAST(const ASTFunction * func, RPNElement & out); bool operatorFromAST(const ASTFunction * func, RPNElement & out);
/** Является ли node столбцом первичного ключа /** Is node the primary key column
* или выражением, в котором столбец первичного ключа завёрнут в цепочку функций, * or expression in which column of primary key is wrapped by chain of functions,
* которые могут быть монотонными на некоторых диапазонах. * that can be monotomic on certain ranges?
* Если да - вернуть номер этого столбца в первичном ключе, а также заполнить цепочку возможно-монотонных функций. * If these conditions are true, then returns number of column in primary key, type of resulting expression
* and fills chain of possibly-monotonic functions.
*/ */
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctions( bool isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node, const ASTPtr & node,
const Context & context, const Context & context,
size_t & out_primary_key_column_num, size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_res_column_type,
RPNElement::MonotonicFunctionsChain & out_functions_chain); RPNElement::MonotonicFunctionsChain & out_functions_chain);
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl( bool isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node, const ASTPtr & node,
size_t & out_primary_key_column_num, size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_column_type,
std::vector<const ASTFunction *> & out_functions_chain); std::vector<const ASTFunction *> & out_functions_chain);
RPN rpn; RPN rpn;
SortDescription sort_descr; SortDescription sort_descr;
ColumnIndices pk_columns; ColumnIndices pk_columns;
const Block & pk_sample_block;
}; };
} }

View File

@ -104,22 +104,33 @@ String FieldVisitorToString::operator() (const Tuple & x_def) const
UInt64 stringToDateOrDateTime(const String & s) UInt64 stringToDateOrDateTime(const String & s)
{ {
ReadBufferFromString in(s);
if (s.size() == strlen("YYYY-MM-DD")) if (s.size() == strlen("YYYY-MM-DD"))
{ return stringToDate(s);
DayNum_t date{};
readDateText(date, in);
return UInt64(date);
}
else else
{ return stringToDateTime(s);
time_t date_time{}; }
readDateTimeText(date_time, in);
if (!in.eof())
throw Exception("String is too long for DateTime: " + s); DayNum_t stringToDate(const String & s)
return UInt64(date_time); {
} ReadBufferFromString in(s);
DayNum_t date{};
readDateText(date, in);
return date;
}
UInt64 stringToDateTime(const String & s)
{
ReadBufferFromString in(s);
time_t date_time{};
readDateTimeText(date_time, in);
if (!in.eof())
throw Exception("String is too long for DateTime: " + s);
return UInt64(date_time);
} }

View File

@ -1,6 +1,7 @@
#include <DB/IO/WriteBufferFromString.h> #include <DB/IO/WriteBufferFromString.h>
#include <DB/DataTypes/DataTypeEnum.h> #include <DB/DataTypes/DataTypeEnum.h>
#include <limits>
namespace DB namespace DB
{ {
@ -239,6 +240,49 @@ Field DataTypeEnum<Type>::getDefault() const
return typename NearestFieldType<FieldType>::Type(values.front().second); return typename NearestFieldType<FieldType>::Type(values.front().second);
} }
template <typename Type>
static void checkOverflow(Int64 value)
{
if (!(std::numeric_limits<Type>::min() <= value && value <= std::numeric_limits<Type>::max()))
throw Exception("DataTypeEnum: Unexpected value " + toString(value), ErrorCodes::BAD_TYPE_OF_FIELD);
}
template <typename Type>
Field DataTypeEnum<Type>::castToName(const Field & value_or_name) const
{
if (value_or_name.getType() == Field::Types::String)
{
getValue(value_or_name.get<String>()); /// Check correctness
return value_or_name.get<String>();
}
else if (value_or_name.getType() == Field::Types::Int64)
{
Int64 value = value_or_name.get<Int64>();
checkOverflow<Type>(value);
return getNameForValue(static_cast<Type>(value)).toString();
}
else
throw Exception(String("DataTypeEnum: Unsupported type of field ") + value_or_name.getTypeName(), ErrorCodes::BAD_TYPE_OF_FIELD);
}
template <typename Type>
Field DataTypeEnum<Type>::castToValue(const Field & value_or_name) const
{
if (value_or_name.getType() == Field::Types::String)
{
return static_cast<Int64>(getValue(value_or_name.get<String>()));
}
else if (value_or_name.getType() == Field::Types::Int64)
{
Int64 value = value_or_name.get<Int64>();
checkOverflow<Type>(value);
getNameForValue(static_cast<Type>(value)); /// Check correctness
return value;
}
else
throw Exception(String("DataTypeEnum: Unsupported type of field ") + value_or_name.getTypeName(), ErrorCodes::BAD_TYPE_OF_FIELD);
}
/// Явные инстанцирования. /// Явные инстанцирования.
template class DataTypeEnum<Int8>; template class DataTypeEnum<Int8>;

View File

@ -52,6 +52,7 @@ namespace ErrorCodes
extern const int ENGINE_REQUIRED; extern const int ENGINE_REQUIRED;
extern const int TABLE_METADATA_ALREADY_EXISTS; extern const int TABLE_METADATA_ALREADY_EXISTS;
extern const int UNKNOWN_DATABASE_ENGINE; extern const int UNKNOWN_DATABASE_ENGINE;
extern const int DUPLICATE_COLUMN;
} }
@ -395,6 +396,21 @@ InterpreterCreateQuery::ColumnsInfo InterpreterCreateQuery::setColumns(
create.children.push_back(new_columns); create.children.push_back(new_columns);
create.columns = new_columns; create.columns = new_columns;
/// Check for duplicates
std::set<String> all_columns;
auto check_column_already_exists = [&all_columns](const NameAndTypePair & column_name_and_type)
{
if (!all_columns.emplace(column_name_and_type.name).second)
throw Exception("Column " + backQuoteIfNeed(column_name_and_type.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN);
};
for (const auto & elem : *res.columns)
check_column_already_exists(elem);
for (const auto & elem : res.materialized_columns)
check_column_already_exists(elem);
for (const auto & elem : res.alias_columns)
check_column_already_exists(elem);
return res; return res;
} }

View File

@ -33,6 +33,7 @@
#include <DB/DataStreams/AggregatingSortedBlockInputStream.h> #include <DB/DataStreams/AggregatingSortedBlockInputStream.h>
#include <DB/DataTypes/DataTypesNumberFixed.h> #include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDate.h> #include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeEnum.h>
#include <DB/Common/VirtualColumnUtils.h> #include <DB/Common/VirtualColumnUtils.h>
@ -194,9 +195,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
SortDescription sort_descr = data.getSortDescription(); SortDescription sort_descr = data.getSortDescription();
PKCondition key_condition(query, context, available_real_and_virtual_columns, sort_descr); PKCondition key_condition(query, context, available_real_and_virtual_columns, sort_descr,
data.getPrimaryExpression()->getSampleBlock());
PKCondition date_condition(query, context, available_real_and_virtual_columns, PKCondition date_condition(query, context, available_real_and_virtual_columns,
SortDescription(1, SortColumnDescription(data.date_column_name, 1))); SortDescription(1, SortColumnDescription(data.date_column_name, 1)),
Block{{DataTypeDate{}.createColumn(), std::make_shared<DataTypeDate>(), data.date_column_name}});
if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue())
{ {

View File

@ -33,7 +33,7 @@ BlocksWithDateIntervals MergeTreeDataWriter::splitBlockIntoParts(const Block & b
/// Минимальная и максимальная дата. /// Минимальная и максимальная дата.
UInt16 min_date = std::numeric_limits<UInt16>::max(); UInt16 min_date = std::numeric_limits<UInt16>::max();
UInt16 max_date = std::numeric_limits<UInt16>::min(); UInt16 max_date = std::numeric_limits<UInt16>::min();
for (ColumnUInt16::Container_t::const_iterator it = dates.begin(); it != dates.end(); ++it) for (auto it = dates.begin(); it != dates.end(); ++it)
{ {
if (*it < min_date) if (*it < min_date)
min_date = *it; min_date = *it;

View File

@ -2,10 +2,15 @@
#include <DB/DataTypes/DataTypesNumberFixed.h> #include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Interpreters/ExpressionAnalyzer.h> #include <DB/Interpreters/ExpressionAnalyzer.h>
#include <DB/Interpreters/ExpressionActions.h> #include <DB/Interpreters/ExpressionActions.h>
#include <DB/DataTypes/DataTypeEnum.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/Columns/ColumnSet.h> #include <DB/Columns/ColumnSet.h>
#include <DB/Columns/ColumnTuple.h> #include <DB/Columns/ColumnTuple.h>
#include <DB/Parsers/ASTSet.h> #include <DB/Parsers/ASTSet.h>
#include <DB/Functions/FunctionFactory.h> #include <DB/Functions/FunctionFactory.h>
#include <DB/Core/FieldVisitors.h>
namespace DB namespace DB
@ -187,8 +192,9 @@ Block PKCondition::getBlockWithConstants(
} }
PKCondition::PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_) PKCondition::PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns,
: sort_descr(sort_descr_) const SortDescription & sort_descr_, const Block & pk_sample_block_)
: sort_descr(sort_descr_), pk_sample_block(pk_sample_block_)
{ {
for (size_t i = 0; i < sort_descr.size(); ++i) for (size_t i = 0; i < sort_descr.size(); ++i)
{ {
@ -202,7 +208,7 @@ PKCondition::PKCondition(ASTPtr & query, const Context & context, const NamesAnd
*/ */
Block block_with_constants = getBlockWithConstants(query, context, all_columns); Block block_with_constants = getBlockWithConstants(query, context, all_columns);
/// Преобразуем секцию WHERE в обратную польскую строку. /// Trasform WHERE section to Reverse Polish notation
ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*query); ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*query);
if (select.where_expression) if (select.where_expression)
{ {
@ -293,11 +299,13 @@ bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node, const ASTPtr & node,
const Context & context, const Context & context,
size_t & out_primary_key_column_num, size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_res_column_type,
RPNElement::MonotonicFunctionsChain & out_functions_chain) RPNElement::MonotonicFunctionsChain & out_functions_chain)
{ {
std::vector<const ASTFunction *> chain_not_tested_for_monotonicity; std::vector<const ASTFunction *> chain_not_tested_for_monotonicity;
DataTypePtr primary_key_column_type;
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_primary_key_column_num, chain_not_tested_for_monotonicity)) if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_primary_key_column_num, primary_key_column_type, chain_not_tested_for_monotonicity))
return false; return false;
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it) for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
@ -306,9 +314,12 @@ bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
if (!func || !func->hasInformationAboutMonotonicity()) if (!func || !func->hasInformationAboutMonotonicity())
return false; return false;
primary_key_column_type = func->getReturnType({primary_key_column_type});
out_functions_chain.push_back(func); out_functions_chain.push_back(func);
} }
out_primary_key_res_column_type = primary_key_column_type;
return true; return true;
} }
@ -316,6 +327,7 @@ bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl( bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node, const ASTPtr & node,
size_t & out_primary_key_column_num, size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_column_type,
std::vector<const ASTFunction *> & out_functions_chain) std::vector<const ASTFunction *> & out_functions_chain)
{ {
/** Сам по себе, столбец первичного ключа может быть функциональным выражением. Например, intHash32(UserID). /** Сам по себе, столбец первичного ключа может быть функциональным выражением. Например, intHash32(UserID).
@ -327,6 +339,7 @@ bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
if (pk_columns.end() != it) if (pk_columns.end() != it)
{ {
out_primary_key_column_num = it->second; out_primary_key_column_num = it->second;
out_primary_key_column_type = pk_sample_block.getByName(name).type;
return true; return true;
} }
@ -338,7 +351,8 @@ bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
out_functions_chain.push_back(func); out_functions_chain.push_back(func);
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_primary_key_column_num, out_functions_chain)) if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_primary_key_column_num, out_primary_key_column_type,
out_functions_chain))
return false; return false;
return true; return true;
@ -348,6 +362,49 @@ bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
} }
/// NOTE: Keep in the mind that such behavior could be incompatible inside ordinary expression.
/// TODO: Use common methods for types conversions.
static bool tryCastValueToType(const DataTypePtr & desired_type, const DataTypePtr & src_type, Field & src_value)
{
if (desired_type->getName() == src_type->getName())
return true;
/// Try to correct type of constant for correct comparison
try
{
/// Convert String to Enum's value
if (auto data_type_enum = dynamic_cast<const IDataTypeEnum *>(desired_type.get()))
{
src_value = data_type_enum->castToValue(src_value);
}
/// Convert 'YYYY-MM-DD' Strings to Date
else if (typeid_cast<const DataTypeDate *>(desired_type.get()) && typeid_cast<const DataTypeString *>(src_type.get()))
{
src_value = UInt64(stringToDate(src_value.safeGet<String>()));
}
/// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime
else if (typeid_cast<const DataTypeDateTime *>(desired_type.get()) && typeid_cast<const DataTypeString *>(src_type.get()))
{
src_value = stringToDateTime(src_value.safeGet<String>());
}
else if (desired_type->behavesAsNumber() && src_type->behavesAsNumber())
{
/// Ok, numeric types are almost mutually convertible
}
else
{
return false;
}
}
catch (...)
{
return false;
}
return true;
}
bool PKCondition::atomFromAST(ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out) bool PKCondition::atomFromAST(ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
{ {
/** Функции < > = != <= >= in notIn, у которых один агрумент константа, другой - один из столбцов первичного ключа, /** Функции < > = != <= >= in notIn, у которых один агрумент константа, другой - один из столбцов первичного ключа,
@ -362,33 +419,33 @@ bool PKCondition::atomFromAST(ASTPtr & node, const Context & context, Block & bl
if (args.size() != 2) if (args.size() != 2)
return false; return false;
/// Если true, слева константа. DataTypePtr key_expr_type; /// Type of expression containing primary key column
bool inverted; size_t key_arg_pos; /// Position of argument with primary key column (non-const argument)
size_t column; size_t key_column_num; /// Number of a primary key column (inside sort_descr array)
RPNElement::MonotonicFunctionsChain chain; RPNElement::MonotonicFunctionsChain chain;
if (getConstant(args[1], block_with_constants, value) if (getConstant(args[1], block_with_constants, value)
&& isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, column, chain)) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
{ {
inverted = false; key_arg_pos = 0;
} }
else if (getConstant(args[0], block_with_constants, value) else if (getConstant(args[0], block_with_constants, value)
&& isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[1], context, column, chain)) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
{ {
inverted = true; key_arg_pos = 1;
} }
else if (typeid_cast<const ASTSet *>(args[1].get()) else if (typeid_cast<const ASTSet *>(args[1].get())
&& isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, column, chain)) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
{ {
inverted = false; key_arg_pos = 0;
} }
else else
return false; return false;
std::string func_name = func->name; std::string func_name = func->name;
/// Заменим <const> <sign> <column> на <column> <-sign> <const> /// Replace <const> <sign> <data> on to <data> <-sign> <const>
if (inverted) if (key_arg_pos == 1)
{ {
if (func_name == "less") if (func_name == "less")
func_name = "greater"; func_name = "greater";
@ -400,18 +457,28 @@ bool PKCondition::atomFromAST(ASTPtr & node, const Context & context, Block & bl
func_name = "greaterOrEquals"; func_name = "greaterOrEquals";
else if (func_name == "in" || func_name == "notIn" || func_name == "like") else if (func_name == "in" || func_name == "notIn" || func_name == "like")
{ {
/// const IN x не имеет смысла (в отличие от x IN const). /// "const IN data_column" doesn't make sense (unlike "data_column IN const")
return false; return false;
} }
} }
out.key_column = column; out.key_column = key_column_num;
out.monotonic_functions_chain = std::move(chain); out.monotonic_functions_chain = std::move(chain);
const auto atom_it = atom_map.find(func_name); const auto atom_it = atom_map.find(func_name);
if (atom_it == std::end(atom_map)) if (atom_it == std::end(atom_map))
return false; return false;
const DataTypePtr & const_type = block_with_constants.getByName(args[1 - key_arg_pos]->getColumnName()).type;
if (!tryCastValueToType(key_expr_type, const_type, value))
{
throw Exception("Primary key expression contains comparison between inconvertible types: " +
key_expr_type->getName() + " and " + const_type->getName() +
" inside " + DB::toString(func->range),
ErrorCodes::BAD_TYPE_OF_FIELD);
}
return atom_it->second(out, value, node); return atom_it->second(out, value, node);
} }
else if (getConstant(node, block_with_constants, value)) /// Для случаев, когда написано, например, WHERE 0 AND something else if (getConstant(node, block_with_constants, value)) /// Для случаев, когда написано, например, WHERE 0 AND something

View File

@ -0,0 +1,24 @@
17783603973940894060
17783603973940894060
1466602371144054938
1466602371144054938
7021055163408905169
7021055163408905169
15043609445345024647
15043609445345024647
17783603973940894060
17783603973940894060
17783603973940894060
17783603973940894060
15043609445345024647
15043609445345024647
1466602371144054938
1466602371144054938
1466602371144054938
1466602371144054938
3447905173014179293
3447905173014179293
3051197876967004596
3051197876967004596
463667963421364848
463667963421364848

View File

@ -0,0 +1,39 @@
DROP TABLE IF EXISTS test.enum_pk;
CREATE TABLE test.enum_pk (date Date DEFAULT '0000-00-00', x Enum8('0' = 0, '1' = 1, '2' = 2), d Enum8('0' = 0, '1' = 1, '2' = 2)) ENGINE = MergeTree(date, x, 1);
INSERT INTO test.enum_pk (x, d) VALUES ('0', '0')('1', '1')('0', '0')('1', '1')('1', '1')('0', '0')('0', '0')('2', '2')('0', '0')('1', '1')('1', '1')('1', '1')('1', '1')('0', '0');
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x = '0';
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d = '0';
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x != '0';
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d != '0';
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x = '1';
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d = '1';
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE exp2(toInt64(x != '1')) > 1;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE exp2(toInt64(d != '1')) > 1;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x = toString(0);
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d = toString(0);
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (x = toString(0)) > 0;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (d = toString(0)) > 0;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE ((x != toString(1)) > 0) > 0;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE ((d != toString(1)) > 0) > 0;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE exp2((x != toString(0)) != 0) > 1;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE exp2((d != toString(0)) != 0) > 1;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (-(x != toString(0)) = -1) > 0;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (-(d != toString(0)) = -1) > 0;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE 1 = 1;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE 1 = 1;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (x = '0' OR x = '1');
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (d = '0' OR d = '1');
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (x != '0' AND x != '1');
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (d != '0' AND d != '1');