mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Merge pull request #17145 from amosbird/cddt
Fix unmatched type comparison in KeyCondition
This commit is contained in:
commit
fabceebbce
@ -358,6 +358,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
|||||||
maximize(max_bits_of_unsigned_integer, 32);
|
maximize(max_bits_of_unsigned_integer, 32);
|
||||||
else if (typeid_cast<const DataTypeUInt64 *>(type.get()))
|
else if (typeid_cast<const DataTypeUInt64 *>(type.get()))
|
||||||
maximize(max_bits_of_unsigned_integer, 64);
|
maximize(max_bits_of_unsigned_integer, 64);
|
||||||
|
else if (typeid_cast<const DataTypeUInt256 *>(type.get()))
|
||||||
|
maximize(max_bits_of_unsigned_integer, 256);
|
||||||
else if (typeid_cast<const DataTypeInt8 *>(type.get()))
|
else if (typeid_cast<const DataTypeInt8 *>(type.get()))
|
||||||
maximize(max_bits_of_signed_integer, 8);
|
maximize(max_bits_of_signed_integer, 8);
|
||||||
else if (typeid_cast<const DataTypeInt16 *>(type.get()))
|
else if (typeid_cast<const DataTypeInt16 *>(type.get()))
|
||||||
@ -366,6 +368,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
|||||||
maximize(max_bits_of_signed_integer, 32);
|
maximize(max_bits_of_signed_integer, 32);
|
||||||
else if (typeid_cast<const DataTypeInt64 *>(type.get()))
|
else if (typeid_cast<const DataTypeInt64 *>(type.get()))
|
||||||
maximize(max_bits_of_signed_integer, 64);
|
maximize(max_bits_of_signed_integer, 64);
|
||||||
|
else if (typeid_cast<const DataTypeInt128 *>(type.get()))
|
||||||
|
maximize(max_bits_of_signed_integer, 128);
|
||||||
|
else if (typeid_cast<const DataTypeInt256 *>(type.get()))
|
||||||
|
maximize(max_bits_of_signed_integer, 256);
|
||||||
else if (typeid_cast<const DataTypeFloat32 *>(type.get()))
|
else if (typeid_cast<const DataTypeFloat32 *>(type.get()))
|
||||||
maximize(max_mantissa_bits_of_floating, 24);
|
maximize(max_mantissa_bits_of_floating, 24);
|
||||||
else if (typeid_cast<const DataTypeFloat64 *>(type.get()))
|
else if (typeid_cast<const DataTypeFloat64 *>(type.get()))
|
||||||
@ -386,7 +392,18 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
|||||||
|
|
||||||
/// If unsigned is not covered by signed.
|
/// If unsigned is not covered by signed.
|
||||||
if (max_bits_of_signed_integer && max_bits_of_unsigned_integer >= max_bits_of_signed_integer)
|
if (max_bits_of_signed_integer && max_bits_of_unsigned_integer >= max_bits_of_signed_integer)
|
||||||
++min_bit_width_of_integer;
|
{
|
||||||
|
// Because 128 and 256 bit integers are significantly slower, we should not promote to them.
|
||||||
|
// But if we already have wide numbers, promotion is necessary.
|
||||||
|
if (min_bit_width_of_integer != 64)
|
||||||
|
++min_bit_width_of_integer;
|
||||||
|
else
|
||||||
|
throw Exception(
|
||||||
|
getExceptionMessagePrefix(types)
|
||||||
|
+ " because some of them are signed integers and some are unsigned integers,"
|
||||||
|
" but there is no signed integer type, that can exactly represent all required unsigned integer values",
|
||||||
|
ErrorCodes::NO_COMMON_TYPE);
|
||||||
|
}
|
||||||
|
|
||||||
/// If the result must be floating.
|
/// If the result must be floating.
|
||||||
if (max_mantissa_bits_of_floating)
|
if (max_mantissa_bits_of_floating)
|
||||||
@ -413,6 +430,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
|||||||
return std::make_shared<DataTypeInt32>();
|
return std::make_shared<DataTypeInt32>();
|
||||||
else if (min_bit_width_of_integer <= 64)
|
else if (min_bit_width_of_integer <= 64)
|
||||||
return std::make_shared<DataTypeInt64>();
|
return std::make_shared<DataTypeInt64>();
|
||||||
|
else if (min_bit_width_of_integer <= 128)
|
||||||
|
return std::make_shared<DataTypeInt128>();
|
||||||
|
else if (min_bit_width_of_integer <= 256)
|
||||||
|
return std::make_shared<DataTypeInt256>();
|
||||||
else
|
else
|
||||||
throw Exception(getExceptionMessagePrefix(types)
|
throw Exception(getExceptionMessagePrefix(types)
|
||||||
+ " because some of them are signed integers and some are unsigned integers,"
|
+ " because some of them are signed integers and some are unsigned integers,"
|
||||||
@ -429,6 +450,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
|||||||
return std::make_shared<DataTypeUInt32>();
|
return std::make_shared<DataTypeUInt32>();
|
||||||
else if (min_bit_width_of_integer <= 64)
|
else if (min_bit_width_of_integer <= 64)
|
||||||
return std::make_shared<DataTypeUInt64>();
|
return std::make_shared<DataTypeUInt64>();
|
||||||
|
else if (min_bit_width_of_integer <= 256)
|
||||||
|
return std::make_shared<DataTypeUInt256>();
|
||||||
else
|
else
|
||||||
throw Exception("Logical error: " + getExceptionMessagePrefix(types)
|
throw Exception("Logical error: " + getExceptionMessagePrefix(types)
|
||||||
+ " but as all data types are unsigned integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE);
|
+ " but as all data types are unsigned integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE);
|
||||||
|
@ -2,12 +2,14 @@
|
|||||||
#include <Storages/MergeTree/BoolMask.h>
|
#include <Storages/MergeTree/BoolMask.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/FieldToDataType.h>
|
#include <DataTypes/FieldToDataType.h>
|
||||||
|
#include <DataTypes/getLeastSupertype.h>
|
||||||
#include <Interpreters/TreeRewriter.h>
|
#include <Interpreters/TreeRewriter.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
#include <Interpreters/castColumn.h>
|
#include <Interpreters/castColumn.h>
|
||||||
#include <Interpreters/misc.h>
|
#include <Interpreters/misc.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <Functions/FunctionsConversion.h>
|
||||||
#include <Functions/IFunction.h>
|
#include <Functions/IFunction.h>
|
||||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
@ -23,7 +25,6 @@
|
|||||||
#include <cassert>
|
#include <cassert>
|
||||||
#include <stack>
|
#include <stack>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -975,9 +976,6 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
|
|||||||
|
|
||||||
static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const ASTPtr & node)
|
static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const ASTPtr & node)
|
||||||
{
|
{
|
||||||
if (desired_type->equals(*src_type))
|
|
||||||
return;
|
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
src_value = convertFieldToType(src_value, *desired_type, src_type.get());
|
src_value = convertFieldToType(src_value, *desired_type, src_type.get());
|
||||||
@ -1084,15 +1082,6 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
|
|||||||
if (key_column_num == static_cast<size_t>(-1))
|
if (key_column_num == static_cast<size_t>(-1))
|
||||||
throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
/// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5"
|
|
||||||
if (is_constant_transformed)
|
|
||||||
{
|
|
||||||
if (func_name == "less")
|
|
||||||
func_name = "lessOrEquals";
|
|
||||||
else if (func_name == "greater")
|
|
||||||
func_name = "greaterOrEquals";
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Replace <const> <sign> <data> on to <data> <-sign> <const>
|
/// Replace <const> <sign> <data> on to <data> <-sign> <const>
|
||||||
if (key_arg_pos == 1)
|
if (key_arg_pos == 1)
|
||||||
{
|
{
|
||||||
@ -1114,12 +1103,55 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bool cast_not_needed =
|
bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST
|
||||||
is_set_const /// Set args are already casted inside Set::createFromAST
|
|| ((isNativeNumber(key_expr_type) || isDateTime(key_expr_type))
|
||||||
|| (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast.
|
&& (isNativeNumber(const_type) || isDateTime(const_type))); /// Numbers and DateTime are accurately compared without cast.
|
||||||
|
|
||||||
|
if (!cast_not_needed && !key_expr_type->equals(*const_type))
|
||||||
|
{
|
||||||
|
if (const_value.getType() == Field::Types::String)
|
||||||
|
{
|
||||||
|
const_value = convertFieldToType(const_value, *key_expr_type);
|
||||||
|
if (const_value.isNull())
|
||||||
|
return false;
|
||||||
|
// No need to set is_constant_transformed because we're doing exact conversion
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
DataTypePtr common_type = getLeastSupertype({key_expr_type, const_type});
|
||||||
|
if (!const_type->equals(*common_type))
|
||||||
|
{
|
||||||
|
castValueToType(common_type, const_value, const_type, node);
|
||||||
|
|
||||||
|
// Need to set is_constant_transformed unless we're doing exact conversion
|
||||||
|
if (!key_expr_type->equals(*common_type))
|
||||||
|
is_constant_transformed = true;
|
||||||
|
}
|
||||||
|
if (!key_expr_type->equals(*common_type))
|
||||||
|
{
|
||||||
|
ColumnsWithTypeAndName arguments{
|
||||||
|
{nullptr, key_expr_type, ""}, {DataTypeString().createColumnConst(1, common_type->getName()), common_type, ""}};
|
||||||
|
FunctionOverloadResolverPtr func_builder_cast
|
||||||
|
= std::make_shared<FunctionOverloadResolverAdaptor>(CastOverloadResolver::createImpl(false));
|
||||||
|
auto func_cast = func_builder_cast->build(arguments);
|
||||||
|
|
||||||
|
/// If we know the given range only contains one value, then we treat all functions as positive monotonic.
|
||||||
|
if (!func_cast || (!single_point && !func_cast->hasInformationAboutMonotonicity()))
|
||||||
|
return false;
|
||||||
|
chain.push_back(func_cast);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5"
|
||||||
|
if (is_constant_transformed)
|
||||||
|
{
|
||||||
|
if (func_name == "less")
|
||||||
|
func_name = "lessOrEquals";
|
||||||
|
else if (func_name == "greater")
|
||||||
|
func_name = "greaterOrEquals";
|
||||||
|
}
|
||||||
|
|
||||||
if (!cast_not_needed)
|
|
||||||
castValueToType(key_expr_type, const_value, const_type, node);
|
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
return false;
|
return false;
|
||||||
|
@ -0,0 +1,5 @@
|
|||||||
|
1
|
||||||
|
Array(Int64)
|
||||||
|
Array(Int128)
|
||||||
|
Array(Int128)
|
||||||
|
Array(UInt256)
|
@ -0,0 +1,16 @@
|
|||||||
|
drop table if exists test_index;
|
||||||
|
|
||||||
|
create table test_index(date Date) engine MergeTree partition by toYYYYMM(date) order by date;
|
||||||
|
|
||||||
|
insert into test_index values('2020-10-30');
|
||||||
|
|
||||||
|
select 1 from test_index where date < toDateTime('2020-10-30 06:00:00');
|
||||||
|
|
||||||
|
drop table if exists test_index;
|
||||||
|
|
||||||
|
select toTypeName([-1, toUInt32(1)]);
|
||||||
|
-- We don't promote to wide integers
|
||||||
|
select toTypeName([-1, toUInt64(1)]); -- { serverError 386 }
|
||||||
|
select toTypeName([-1, toInt128(1)]);
|
||||||
|
select toTypeName([toInt64(-1), toInt128(1)]);
|
||||||
|
select toTypeName([toUInt64(1), toUInt256(1)]);
|
Loading…
Reference in New Issue
Block a user