Merge pull request #42410 from ClickHouse/fix-multi-arg-monotonicity

Fix possible LOGICAL_ERROR in binary ariphmetics monotonicity.
This commit is contained in:
Nikolai Kochetov 2022-10-26 14:11:23 +02:00 committed by GitHub
commit db422fa903
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 155 additions and 12 deletions

View File

@ -39,6 +39,7 @@
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Interpreters/Context.h>
#if USE_EMBEDDED_COMPILER
@ -1790,25 +1791,32 @@ public:
// const +|- variable
if (left.column && isColumnConst(*left.column))
{
auto left_type = removeNullable(removeLowCardinality(left.type));
auto right_type = removeNullable(removeLowCardinality(right.type));
auto ret_type = removeNullable(removeLowCardinality(return_type));
auto transform = [&](const Field & point)
{
ColumnsWithTypeAndName columns_with_constant
= {{left.column->cloneResized(1), left.type, left.name},
{right.type->createColumnConst(1, point), right.type, right.name}};
= {{left_type->createColumnConst(1, (*left.column)[0]), left_type, left.name},
{right_type->createColumnConst(1, point), right_type, right.name}};
auto col = Base::executeImpl(columns_with_constant, return_type, 1);
/// This is a bit dangerous to call Base::executeImpl cause it ignores `use Default Implementation For XXX` flags.
/// It was possible to check monotonicity for nullable right type which result to exception.
/// Adding removeNullable above fixes the issue, but some other inconsistency may left.
auto col = Base::executeImpl(columns_with_constant, ret_type, 1);
Field point_transformed;
col->get(0, point_transformed);
return point_transformed;
};
transform(left_point);
transform(right_point);
bool is_positive_monotonicity = applyVisitor(FieldVisitorAccurateLess(), left_point, right_point)
== applyVisitor(FieldVisitorAccurateLess(), transform(left_point), transform(right_point));
if (name_view == "plus")
{
// Check if there is an overflow
if (applyVisitor(FieldVisitorAccurateLess(), left_point, right_point)
== applyVisitor(FieldVisitorAccurateLess(), transform(left_point), transform(right_point)))
if (is_positive_monotonicity)
return {true, true, false, true};
else
return {false, true, false, false};
@ -1816,8 +1824,7 @@ public:
else
{
// Check if there is an overflow
if (applyVisitor(FieldVisitorAccurateLess(), left_point, right_point)
!= applyVisitor(FieldVisitorAccurateLess(), transform(left_point), transform(right_point)))
if (!is_positive_monotonicity)
return {true, false, false, true};
else
return {false, false, false, false};
@ -1826,13 +1833,17 @@ public:
// variable +|- constant
else if (right.column && isColumnConst(*right.column))
{
auto left_type = removeNullable(removeLowCardinality(left.type));
auto right_type = removeNullable(removeLowCardinality(right.type));
auto ret_type = removeNullable(removeLowCardinality(return_type));
auto transform = [&](const Field & point)
{
ColumnsWithTypeAndName columns_with_constant
= {{left.type->createColumnConst(1, point), left.type, left.name},
{right.column->cloneResized(1), right.type, right.name}};
= {{left_type->createColumnConst(1, point), left_type, left.name},
{right_type->createColumnConst(1, (*right.column)[0]), right_type, right.name}};
auto col = Base::executeImpl(columns_with_constant, return_type, 1);
auto col = Base::executeImpl(columns_with_constant, ret_type, 1);
Field point_transformed;
col->get(0, point_transformed);
return point_transformed;

View File

@ -2173,6 +2173,10 @@ struct ToNumberMonotonicity
const size_t size_of_from = type.getSizeOfValueInMemory();
const size_t size_of_to = sizeof(T);
/// Do not support 128 bit integers and decimals for now.
if (size_of_from > sizeof(Int64))
return {};
const bool left_in_first_half = left.isNull()
? from_is_unsigned
: (left.get<Int64>() >= 0);

View File

@ -1424,6 +1424,7 @@ public:
ColumnsWithTypeAndName new_arguments;
new_arguments.reserve(arguments.size() + 1);
new_arguments.push_back(const_arg);
new_arguments.front().column = new_arguments.front().column->cloneResized(input_rows_count);
for (const auto & arg : arguments)
new_arguments.push_back(arg);
return func->prepare(new_arguments)->execute(new_arguments, result_type, input_rows_count, dry_run);
@ -1432,6 +1433,7 @@ public:
{
auto new_arguments = arguments;
new_arguments.push_back(const_arg);
new_arguments.back().column = new_arguments.back().column->cloneResized(input_rows_count);
return func->prepare(new_arguments)->execute(new_arguments, result_type, input_rows_count, dry_run);
}
else

View File

@ -0,0 +1,64 @@
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
1
2
2022-02-02 00:00:01
2022-02-02 00:00:02
2022-02-02 00:00:01
2022-02-02 00:00:02
2022-02-02 00:00:01
2022-02-02 00:00:02
2022-02-02 00:00:01
2022-02-02 00:00:02
2022-02-02 00:00:01
2022-02-02 00:00:02
2022-02-02 00:00:01
2022-02-02 00:00:02
2022-02-02 00:00:01
2022-02-02 00:00:02
2022-02-02 00:00:01
2022-02-02 00:00:02

View File

@ -0,0 +1,62 @@
create table tab (x Nullable(UInt8)) engine = MergeTree order by x settings allow_nullable_key = 1, index_granularity = 2;
insert into tab select number from numbers(4);
set allow_suspicious_low_cardinality_types=1;
set max_rows_to_read = 2;
SELECT x + 1 FROM tab where plus(x, 1) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::Nullable(UInt8)) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(UInt8)) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(Nullable(UInt8))) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1, x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::Nullable(UInt8), x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(UInt8), x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(Nullable(UInt8)), x) <= 2 order by x;
drop table tab;
set max_rows_to_read = 100;
create table tab (x LowCardinality(UInt8)) engine = MergeTree order by x settings allow_nullable_key = 1, index_granularity = 2;
insert into tab select number from numbers(4);
set max_rows_to_read = 2;
SELECT x + 1 FROM tab where plus(x, 1) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::Nullable(UInt8)) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(UInt8)) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(Nullable(UInt8))) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1, x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::Nullable(UInt8), x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(UInt8), x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(Nullable(UInt8)), x) <= 2 order by x;
drop table tab;
set max_rows_to_read = 100;
create table tab (x UInt128) engine = MergeTree order by x settings allow_nullable_key = 1, index_granularity = 2;
insert into tab select number from numbers(4);
set max_rows_to_read = 2;
SELECT x + 1 FROM tab where plus(x, 1) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::Nullable(UInt8)) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(UInt8)) <= 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(Nullable(UInt8))) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1, x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::Nullable(UInt8), x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(UInt8), x) <= 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(Nullable(UInt8)), x) <= 2 order by x;
set max_rows_to_read = 100;
SELECT x + 1 FROM tab WHERE (x + 1::LowCardinality(UInt8)) <= -9223372036854775808 order by x;
drop table tab;
create table tab (x DateTime) engine = MergeTree order by x settings allow_nullable_key = 1, index_granularity = 2;
insert into tab select toDateTime('2022-02-02') + number from numbers(4);
set max_rows_to_read = 2;
SELECT x + 1 FROM tab where plus(x, 1) <= toDateTime('2022-02-02') + 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::Nullable(UInt8)) <= toDateTime('2022-02-02') + 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(UInt8)) <= toDateTime('2022-02-02') + 2 order by x;
SELECT x + 1 FROM tab where plus(x, 1::LowCardinality(Nullable(UInt8))) <= toDateTime('2022-02-02') + 2 order by x;
SELECT 1 + x FROM tab where plus(1, x) <= toDateTime('2022-02-02') + 2 order by x;
SELECT 1 + x FROM tab where plus(1::Nullable(UInt8), x) <= toDateTime('2022-02-02') + 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(UInt8), x) <= toDateTime('2022-02-02') + 2 order by x;
SELECT 1 + x FROM tab where plus(1::LowCardinality(Nullable(UInt8)), x) <= toDateTime('2022-02-02') + 2 order by x;
SELECT x + 1 FROM tab WHERE (x + CAST('1', 'Nullable(UInt8)')) <= -2147483647 ORDER BY x ASC NULLS FIRST;