mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Merge pull request #192 from ludv1x/enums-fix
Fixed Enum usage in PK expressions.
This commit is contained in:
commit
fdabcd4e38
@ -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.
|
||||||
|
@ -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;
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -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>;
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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())
|
||||||
{
|
{
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
24
dbms/tests/queries/0_stateless/00386_enum_in_pk.reference
Normal file
24
dbms/tests/queries/0_stateless/00386_enum_in_pk.reference
Normal 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
|
39
dbms/tests/queries/0_stateless/00386_enum_in_pk.sql
Normal file
39
dbms/tests/queries/0_stateless/00386_enum_in_pk.sql
Normal 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');
|
Loading…
Reference in New Issue
Block a user