提交 f1e9f671 编写于 作者: R robot-clickhouse

Backport #17145 to 20.12: Fix unmatched type comparison in KeyCondition

上级 8f2d33f5
......@@ -358,6 +358,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
maximize(max_bits_of_unsigned_integer, 32);
else if (typeid_cast<const DataTypeUInt64 *>(type.get()))
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()))
maximize(max_bits_of_signed_integer, 8);
else if (typeid_cast<const DataTypeInt16 *>(type.get()))
......@@ -366,6 +368,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
maximize(max_bits_of_signed_integer, 32);
else if (typeid_cast<const DataTypeInt64 *>(type.get()))
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()))
maximize(max_mantissa_bits_of_floating, 24);
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 (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 (max_mantissa_bits_of_floating)
......@@ -413,6 +430,10 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
return std::make_shared<DataTypeInt32>();
else if (min_bit_width_of_integer <= 64)
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
throw Exception(getExceptionMessagePrefix(types)
+ " 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>();
else if (min_bit_width_of_integer <= 64)
return std::make_shared<DataTypeUInt64>();
else if (min_bit_width_of_integer <= 256)
return std::make_shared<DataTypeUInt256>();
else
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);
......
......@@ -2,12 +2,14 @@
#include <Storages/MergeTree/BoolMask.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/getLeastSupertype.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/misc.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/IFunction.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/typeid_cast.h>
......@@ -23,7 +25,6 @@
#include <cassert>
#include <stack>
namespace DB
{
......@@ -959,9 +960,6 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
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
{
src_value = convertFieldToType(src_value, *desired_type, src_type.get());
......@@ -1068,15 +1066,6 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
if (key_column_num == static_cast<size_t>(-1))
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>
if (key_arg_pos == 1)
{
......@@ -1098,12 +1087,55 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
}
}
bool cast_not_needed =
is_set_const /// Set args are already casted inside Set::createFromAST
|| (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast.
bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST
|| ((isNativeNumber(key_expr_type) || isDateTime(key_expr_type))
&& (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
return false;
......
1
Array(Int64)
Array(Int128)
Array(Int128)
Array(UInt256)
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)]);
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册