未验证 提交 fd6115cc 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #11648 from ClickHouse/simplify-code-functions-comparison-3

Allow comparison with String in index analysis; simplify code.
......@@ -4,10 +4,10 @@
#include <memory>
#include <vector>
#include <boost/noncopyable.hpp>
#if __has_include(<sanitizer/asan_interface.h>)
#include <Core/Defines.h>
#if __has_include(<sanitizer/asan_interface.h>) && defined(ADDRESS_SANITIZER)
# include <sanitizer/asan_interface.h>
#endif
#include <Core/Defines.h>
#include <Common/memcpySmall.h>
#include <Common/ProfileEvents.h>
#include <Common/Allocator.h>
......
#pragma once
#if __has_include(<sanitizer/asan_interface.h>)
#include <Core/Defines.h>
#if __has_include(<sanitizer/asan_interface.h>) && defined(ADDRESS_SANITIZER)
# include <sanitizer/asan_interface.h>
#endif
#include <Core/Defines.h>
#include <Common/Arena.h>
#include <Common/BitHelpers.h>
......
#pragma once
#include <Core/Field.h>
#include <Core/AccurateComparison.h>
#include <common/demangle.h>
......@@ -14,7 +13,6 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_CONVERT_TYPE;
extern const int BAD_TYPE_OF_FIELD;
extern const int LOGICAL_ERROR;
}
......@@ -177,243 +175,6 @@ template <> constexpr bool isDecimalField<DecimalField<Decimal64>>() { return tr
template <> constexpr bool isDecimalField<DecimalField<Decimal128>>() { return true; }
/** More precise comparison, used for index.
* Differs from Field::operator< and Field::operator== in that it also compares values of different types.
* Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query).
*/
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
{
public:
bool operator() (const UInt64 &, const Null &) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
bool operator() (const UInt64 & l, const UInt128 & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
bool operator() (const Int64 &, const Null &) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Int64 & l, const UInt128 & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
bool operator() (const Float64 &, const Null &) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Float64 & l, const UInt128 & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
template <typename T>
bool operator() (const Null &, const T &) const
{
return std::is_same_v<T, Null>;
}
template <typename T>
bool operator() (const String & l, const T & r) const
{
if constexpr (std::is_same_v<T, String>)
return l == r;
if constexpr (std::is_same_v<T, UInt128>)
return stringToUUID(l) == r;
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T>
bool operator() (const UInt128 & l, const T & r) const
{
if constexpr (std::is_same_v<T, UInt128>)
return l == r;
if constexpr (std::is_same_v<T, String>)
return l == stringToUUID(r);
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T>
bool operator() (const Array & l, const T & r) const
{
if constexpr (std::is_same_v<T, Array>)
return l == r;
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T>
bool operator() (const Tuple & l, const T & r) const
{
if constexpr (std::is_same_v<T, Tuple>)
return l == r;
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T, typename U>
bool operator() (const DecimalField<T> & l, const U & r) const
{
if constexpr (isDecimalField<U>())
return l == r;
if constexpr (std::is_same_v<U, Int64> || std::is_same_v<U, UInt64>)
return l == DecimalField<Decimal128>(r, 0);
if constexpr (std::is_same_v<U, Null>)
return false;
return cantCompare(l, r);
}
template <typename T> bool operator() (const UInt64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) == r; }
template <typename T> bool operator() (const Int64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) == r; }
template <typename T> bool operator() (const Float64 & l, const DecimalField<T> & r) const { return cantCompare(l, r); }
template <typename T>
bool operator() (const AggregateFunctionStateData & l, const T & r) const
{
if constexpr (std::is_same_v<T, AggregateFunctionStateData>)
return l == r;
return cantCompare(l, r);
}
private:
template <typename T, typename U>
bool cantCompare(const T &, const U &) const
{
if constexpr (std::is_same_v<U, Null>)
return false;
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
ErrorCodes::BAD_TYPE_OF_FIELD);
}
};
class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
public:
bool operator() (const UInt64 &, const Null &) const { return false; }
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
bool operator() (const UInt64 & l, const UInt128 & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); }
bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
bool operator() (const Int64 &, const Null &) const { return false; }
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Int64 & l, const UInt128 & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); }
bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
bool operator() (const Float64 &, const Null &) const { return false; }
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Float64 & l, const UInt128 & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); }
bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
template <typename T>
bool operator() (const Null &, const T &) const
{
return !std::is_same_v<T, Null>;
}
template <typename T>
bool operator() (const String & l, const T & r) const
{
if constexpr (std::is_same_v<T, String>)
return l < r;
if constexpr (std::is_same_v<T, UInt128>)
return stringToUUID(l) < r;
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T>
bool operator() (const UInt128 & l, const T & r) const
{
if constexpr (std::is_same_v<T, UInt128>)
return l < r;
if constexpr (std::is_same_v<T, String>)
return l < stringToUUID(r);
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T>
bool operator() (const Array & l, const T & r) const
{
if constexpr (std::is_same_v<T, Array>)
return l < r;
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T>
bool operator() (const Tuple & l, const T & r) const
{
if constexpr (std::is_same_v<T, Tuple>)
return l < r;
if constexpr (std::is_same_v<T, Null>)
return false;
return cantCompare(l, r);
}
template <typename T, typename U>
bool operator() (const DecimalField<T> & l, const U & r) const
{
if constexpr (isDecimalField<U>())
return l < r;
if constexpr (std::is_same_v<U, Int64> || std::is_same_v<U, UInt64>)
return l < DecimalField<Decimal128>(r, 0);
if constexpr (std::is_same_v<U, Null>)
return false;
return cantCompare(l, r);
}
template <typename T> bool operator() (const UInt64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) < r; }
template <typename T> bool operator() (const Int64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) < r; }
template <typename T> bool operator() (const Float64 &, const DecimalField<T> &) const { return false; }
template <typename T>
bool operator() (const AggregateFunctionStateData & l, const T & r) const
{
return cantCompare(l, r);
}
private:
template <typename T, typename U>
bool cantCompare(const T &, const U &) const
{
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
ErrorCodes::BAD_TYPE_OF_FIELD);
}
};
/** Implements `+=` operation.
* Returns false if the result is zero.
*/
......
#pragma once
#include <Core/Field.h>
#include <Core/AccurateComparison.h>
#include <common/demangle.h>
#include <Common/FieldVisitors.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
}
/** More precise comparison, used for index.
* Differs from Field::operator< and Field::operator== in that it also compares values of different types.
* Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query).
*/
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
{
public:
template <typename T, typename U>
bool operator() (const T & l, const U & r) const
{
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
return std::is_same_v<T, U>;
else
{
if constexpr (std::is_same_v<T, U>)
return l == r;
if constexpr (std::is_arithmetic_v<T> && std::is_arithmetic_v<U>)
return accurate::equalsOp(l, r);
if constexpr (isDecimalField<T>() && isDecimalField<U>())
return l == r;
if constexpr (isDecimalField<T>() && std::is_arithmetic_v<U>)
return l == DecimalField<Decimal128>(r, 0);
if constexpr (std::is_arithmetic_v<T> && isDecimalField<U>())
return DecimalField<Decimal128>(l, 0) == r;
if constexpr (std::is_same_v<T, String>)
{
if constexpr (std::is_same_v<U, UInt128>)
return stringToUUID(l) == r;
if constexpr (std::is_arithmetic_v<U>)
{
ReadBufferFromString in(l);
T parsed;
readText(parsed, in);
return operator()(parsed, r);
}
}
if constexpr (std::is_same_v<U, String>)
{
if constexpr (std::is_same_v<T, UInt128>)
return l == stringToUUID(r);
if constexpr (std::is_arithmetic_v<T>)
{
ReadBufferFromString in(r);
T parsed;
readText(parsed, in);
return operator()(l, parsed);
}
}
}
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
ErrorCodes::BAD_TYPE_OF_FIELD);
}
};
class FieldVisitorAccurateLess : public StaticVisitor<bool>
{
public:
template <typename T, typename U>
bool operator() (const T & l, const U & r) const
{
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
return false;
else
{
if constexpr (std::is_same_v<T, U>)
return l < r;
if constexpr (std::is_arithmetic_v<T> && std::is_arithmetic_v<U>)
return accurate::lessOp(l, r);
if constexpr (isDecimalField<T>() && isDecimalField<U>())
return l < r;
if constexpr (isDecimalField<T>() && std::is_arithmetic_v<U>)
return l < DecimalField<Decimal128>(r, 0);
if constexpr (std::is_arithmetic_v<T> && isDecimalField<U>())
return DecimalField<Decimal128>(l, 0) < r;
if constexpr (std::is_same_v<T, String>)
{
if constexpr (std::is_same_v<U, UInt128>)
return stringToUUID(l) < r;
if constexpr (std::is_arithmetic_v<U>)
{
ReadBufferFromString in(l);
T parsed;
readText(parsed, in);
return operator()(parsed, r);
}
}
if constexpr (std::is_same_v<U, String>)
{
if constexpr (std::is_same_v<T, UInt128>)
return l < stringToUUID(r);
if constexpr (std::is_arithmetic_v<T>)
{
ReadBufferFromString in(r);
T parsed;
readText(parsed, in);
return operator()(l, parsed);
}
}
}
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
ErrorCodes::BAD_TYPE_OF_FIELD);
}
};
}
......@@ -87,7 +87,7 @@
#define DBMS_DISTRIBUTED_SIGNATURE_HEADER 0xCAFEDACEull
#define DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT 0xCAFECABEull
#if !__has_include(<sanitizer/asan_interface.h>)
#if !__has_include(<sanitizer/asan_interface.h>) || !defined(ADDRESS_SANITIZER)
# define ASAN_UNPOISON_MEMORY_REGION(a, b)
# define ASAN_POISON_MEMORY_REGION(a, b)
#endif
......
......@@ -9,7 +9,7 @@
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Common/FieldVisitors.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/memcmpSmall.h>
#include <Common/assert_cast.h>
......
#include <Interpreters/FillingRow.h>
#include <Common/FieldVisitorsAccurateComparison.h>
namespace DB
{
......
#pragma once
#include <Core/SortDescription.h>
#include <Columns/IColumn.h>
#include <Common/FieldVisitors.h>
namespace DB
{
......
......@@ -37,7 +37,7 @@
#include <Core/Field.h>
#include <Core/Types.h>
#include <Columns/Collator.h>
#include <Common/FieldVisitors.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <ext/map.h>
......
......@@ -7,7 +7,7 @@
#include <Interpreters/misc.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/FieldVisitors.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/typeid_cast.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/Set.h>
......@@ -826,8 +826,8 @@ 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.
is_set_const /// Set args are already casted inside Set::createFromAST
|| (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast.
if (!cast_not_needed)
castValueToType(key_expr_type, const_value, const_type, node);
......
1
999999
100000
899999
100001
900000
1
999999
100000
899999
100001
900000
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000;
INSERT INTO test SELECT * FROM numbers(1000000);
OPTIMIZE TABLE test;
SET max_rows_to_read = 2000;
SELECT count() FROM test WHERE x = 100000;
SET max_rows_to_read = 1000000;
SELECT count() FROM test WHERE x != 100000;
SET max_rows_to_read = 101000;
SELECT count() FROM test WHERE x < 100000;
SET max_rows_to_read = 900000;
SELECT count() FROM test WHERE x > 100000;
SET max_rows_to_read = 101000;
SELECT count() FROM test WHERE x <= 100000;
SET max_rows_to_read = 901000;
SELECT count() FROM test WHERE x >= 100000;
SET max_rows_to_read = 2000;
SELECT count() FROM test WHERE x = '100000';
SET max_rows_to_read = 1000000;
SELECT count() FROM test WHERE x != '100000';
SET max_rows_to_read = 101000;
SELECT count() FROM test WHERE x < '100000';
SET max_rows_to_read = 900000;
SELECT count() FROM test WHERE x > '100000';
SET max_rows_to_read = 101000;
SELECT count() FROM test WHERE x <= '100000';
SET max_rows_to_read = 901000;
SELECT count() FROM test WHERE x >= '100000';
DROP TABLE test;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册