diff --git a/src/Common/Arena.h b/src/Common/Arena.h index d203a92d4a3..44a9b444ff2 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -4,10 +4,10 @@ #include #include #include -#if __has_include() +#include +#if __has_include() && defined(ADDRESS_SANITIZER) # include #endif -#include #include #include #include diff --git a/src/Common/ArenaWithFreeLists.h b/src/Common/ArenaWithFreeLists.h index 6092f03ce19..1284c3586c0 100644 --- a/src/Common/ArenaWithFreeLists.h +++ b/src/Common/ArenaWithFreeLists.h @@ -1,9 +1,9 @@ #pragma once -#if __has_include() +#include +#if __has_include() && defined(ADDRESS_SANITIZER) # include #endif -#include #include #include diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index 90f80974ab1..ddeddb8fbf6 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include @@ -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>() { return tr template <> constexpr bool isDecimalField>() { 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 -{ -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 - bool operator() (const Null &, const T &) const - { - return std::is_same_v; - } - - template - bool operator() (const String & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return stringToUUID(l) == r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const UInt128 & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return l == stringToUUID(r); - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Array & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Tuple & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const DecimalField & l, const U & r) const - { - if constexpr (isDecimalField()) - return l == r; - if constexpr (std::is_same_v || std::is_same_v) - return l == DecimalField(r, 0); - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template bool operator() (const UInt64 & l, const DecimalField & r) const { return DecimalField(l, 0) == r; } - template bool operator() (const Int64 & l, const DecimalField & r) const { return DecimalField(l, 0) == r; } - template bool operator() (const Float64 & l, const DecimalField & r) const { return cantCompare(l, r); } - - template - bool operator() (const AggregateFunctionStateData & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - return cantCompare(l, r); - } - -private: - template - bool cantCompare(const T &, const U &) const - { - if constexpr (std::is_same_v) - return false; - throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()), - ErrorCodes::BAD_TYPE_OF_FIELD); - } -}; - -class FieldVisitorAccurateLess : public StaticVisitor -{ -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 - bool operator() (const Null &, const T &) const - { - return !std::is_same_v; - } - - template - bool operator() (const String & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return stringToUUID(l) < r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const UInt128 & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return l < stringToUUID(r); - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Array & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Tuple & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const DecimalField & l, const U & r) const - { - if constexpr (isDecimalField()) - return l < r; - if constexpr (std::is_same_v || std::is_same_v) - return l < DecimalField(r, 0); - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template bool operator() (const UInt64 & l, const DecimalField & r) const { return DecimalField(l, 0) < r; } - template bool operator() (const Int64 & l, const DecimalField & r) const { return DecimalField(l, 0) < r; } - template bool operator() (const Float64 &, const DecimalField &) const { return false; } - - template - bool operator() (const AggregateFunctionStateData & l, const T & r) const - { - return cantCompare(l, r); - } - -private: - template - 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. */ diff --git a/src/Common/FieldVisitorsAccurateComparison.h b/src/Common/FieldVisitorsAccurateComparison.h new file mode 100644 index 00000000000..91fa4bf28de --- /dev/null +++ b/src/Common/FieldVisitorsAccurateComparison.h @@ -0,0 +1,142 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +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 +{ +public: + template + bool operator() (const T & l, const U & r) const + { + if constexpr (std::is_same_v || std::is_same_v) + return std::is_same_v; + else + { + if constexpr (std::is_same_v) + return l == r; + + if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + return accurate::equalsOp(l, r); + + if constexpr (isDecimalField() && isDecimalField()) + return l == r; + + if constexpr (isDecimalField() && std::is_arithmetic_v) + return l == DecimalField(r, 0); + + if constexpr (std::is_arithmetic_v && isDecimalField()) + return DecimalField(l, 0) == r; + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return stringToUUID(l) == r; + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(l); + T parsed; + readText(parsed, in); + return operator()(parsed, r); + } + } + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return l == stringToUUID(r); + + if constexpr (std::is_arithmetic_v) + { + 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 +{ +public: + template + bool operator() (const T & l, const U & r) const + { + if constexpr (std::is_same_v || std::is_same_v) + return false; + else + { + if constexpr (std::is_same_v) + return l < r; + + if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + return accurate::lessOp(l, r); + + if constexpr (isDecimalField() && isDecimalField()) + return l < r; + + if constexpr (isDecimalField() && std::is_arithmetic_v) + return l < DecimalField(r, 0); + + if constexpr (std::is_arithmetic_v && isDecimalField()) + return DecimalField(l, 0) < r; + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return stringToUUID(l) < r; + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(l); + T parsed; + readText(parsed, in); + return operator()(parsed, r); + } + } + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return l < stringToUUID(r); + + if constexpr (std::is_arithmetic_v) + { + 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); + } +}; + +} diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 13070c565b4..8b26f486c9d 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -87,7 +87,7 @@ #define DBMS_DISTRIBUTED_SIGNATURE_HEADER 0xCAFEDACEull #define DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT 0xCAFECABEull -#if !__has_include() +#if !__has_include() || !defined(ADDRESS_SANITIZER) # define ASAN_UNPOISON_MEMORY_REGION(a, b) # define ASAN_POISON_MEMORY_REGION(a, b) #endif diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index b575e9d81c1..249da255eb0 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -45,6 +45,8 @@ public: void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; bool equals(const IDataType & rhs) const override; + + bool canBePromoted() const override { return false; } }; /** Tansform-type wrapper for DateTime64, applies given Transform to DateTime64 value or only to a whole part of it. diff --git a/src/DataTypes/DataTypeUUID.cpp b/src/DataTypes/DataTypeUUID.cpp index 87e306cf477..94a043eb472 100644 --- a/src/DataTypes/DataTypeUUID.cpp +++ b/src/DataTypes/DataTypeUUID.cpp @@ -16,13 +16,18 @@ void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBu writeText(UUID(assert_cast(column).getData()[row_num]), ostr); } -void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +void DataTypeUUID::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { UUID x; readText(x, istr); assert_cast(column).getData().push_back(x); } +void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeText(column, istr, settings); +} + void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeText(column, row_num, ostr, settings); diff --git a/src/DataTypes/DataTypeUUID.h b/src/DataTypes/DataTypeUUID.h index 75e634bc625..e9f1d22325b 100644 --- a/src/DataTypes/DataTypeUUID.h +++ b/src/DataTypes/DataTypeUUID.h @@ -17,6 +17,7 @@ public: bool equals(const IDataType & rhs) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; @@ -30,6 +31,8 @@ public: bool canBeUsedInBitOperations() const override { return true; } bool canBeInsideNullable() const override { return true; } + + bool canBePromoted() const override { return false; } }; } diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 642995974b5..91525f84c14 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -22,6 +22,7 @@ #include #include +#include #include #include @@ -51,7 +52,6 @@ namespace DB namespace ErrorCodes { - extern const int TOO_LARGE_STRING_SIZE; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; @@ -812,94 +812,51 @@ private: } } - bool executeDateOrDateTimeOrEnumOrUUIDWithConstString( + bool executeWithConstString( Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped, - const DataTypePtr & left_type, const DataTypePtr & right_type, bool left_is_num, size_t input_rows_count) + const DataTypePtr & left_type, const DataTypePtr & right_type, size_t input_rows_count) { - /// This is no longer very special case - comparing dates, datetimes, and enumerations with a string constant. - const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped; - const IColumn * column_number = left_is_num ? col_left_untyped : col_right_untyped; - const IDataType * number_type = left_is_num ? left_type.get() : right_type.get(); + /// To compare something with const string, we cast constant to appropriate type and compare as usual. + /// It is ok to throw exception if value is not convertible. + /// We should deal with possible overflows, e.g. toUInt8(1) = '257' should return false. - WhichDataType which(number_type); + const ColumnConst * left_const = checkAndGetColumnConstStringOrFixedString(col_left_untyped); + const ColumnConst * right_const = checkAndGetColumnConstStringOrFixedString(col_right_untyped); - const bool legal_types = which.isDateOrDateTime() || which.isEnum() || which.isUUID(); - - const auto column_string = checkAndGetColumnConst(column_string_untyped); - if (!column_string || !legal_types) + if (!left_const && !right_const) return false; - StringRef string_value = column_string->getDataAt(0); + const IDataType * type_string = left_const ? left_type.get() : right_type.get(); + const DataTypePtr & type_to_compare = !left_const ? left_type : right_type; - if (which.isDate()) + Field string_value = left_const ? left_const->getField() : right_const->getField(); + Field converted = convertFieldToType(string_value, *type_to_compare, type_string); + + /// If not possible to convert, comparison with =, <, >, <=, >= yields to false and comparison with != yields to true. + if (converted.isNull()) { - DayNum date; - ReadBufferFromMemory in(string_value.data, string_value.size); - readDateText(date, in); - if (!in.eof()) - throw Exception("String is too long for Date: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - - ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, date); - const ColumnConst * parsed_const_date = assert_cast(parsed_const_date_holder.get()); - executeNumLeftType(block, result, - left_is_num ? col_left_untyped : parsed_const_date, - left_is_num ? parsed_const_date : col_right_untyped); + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, + std::is_same_v, NotEqualsOp>); } - else if (which.isDateTime()) + else { - time_t date_time; - ReadBufferFromMemory in(string_value.data, string_value.size); - readDateTimeText(date_time, in, dynamic_cast(*number_type).getTimeZone()); - if (!in.eof()) - throw Exception("String is too long for DateTime: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); + auto column_converted = type_to_compare->createColumnConst(input_rows_count, converted); - ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time)); - const ColumnConst * parsed_const_date_time = assert_cast(parsed_const_date_time_holder.get()); - executeNumLeftType(block, result, - left_is_num ? col_left_untyped : parsed_const_date_time, - left_is_num ? parsed_const_date_time : col_right_untyped); + Block tmp_block + { + { left_const ? column_converted : col_left_untyped->getPtr(), type_to_compare, "" }, + { !left_const ? column_converted : col_right_untyped->getPtr(), type_to_compare, "" }, + block.getByPosition(result) + }; + + executeImpl(tmp_block, {0, 1}, 2, input_rows_count); + + block.getByPosition(result).column = std::move(tmp_block.getByPosition(2).column); } - else if (which.isUUID()) - { - UUID uuid; - ReadBufferFromMemory in(string_value.data, string_value.size); - readText(uuid, in); - if (!in.eof()) - throw Exception("String is too long for UUID: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - - ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, uuid); - const ColumnConst * parsed_const_uuid = assert_cast(parsed_const_uuid_holder.get()); - executeNumLeftType(block, result, - left_is_num ? col_left_untyped : parsed_const_uuid, - left_is_num ? parsed_const_uuid : col_right_untyped); - } - - else if (which.isEnum8()) - executeEnumWithConstString(block, result, column_number, column_string, - number_type, left_is_num, input_rows_count); - else if (which.isEnum16()) - executeEnumWithConstString(block, result, column_number, column_string, - number_type, left_is_num, input_rows_count); return true; } - /// Comparison between DataTypeEnum and string constant containing the name of an enum element - template - void executeEnumWithConstString( - Block & block, const size_t result, const IColumn * column_number, const ColumnConst * column_string, - const IDataType * type_untyped, const bool left_is_num, size_t input_rows_count) - { - const auto type = static_cast(type_untyped); - - const Field x = castToNearestFieldType(type->getValue(column_string->getValue())); - const auto enum_col = type->createColumnConst(input_rows_count, x); - - executeNumLeftType(block, result, - left_is_num ? column_number : enum_col.get(), - left_is_num ? enum_col.get() : column_number); - } - void executeTuple(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1, size_t input_rows_count) { @@ -1124,17 +1081,11 @@ public: bool has_date = left.isDate() || right.isDate(); if (!((both_represented_by_number && !has_date) /// Do not allow compare date and number. - || (left.isStringOrFixedString() && right.isStringOrFixedString()) + || (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion. /// You can compare the date, datetime, or datatime64 and an enumeration with a constant string. - || (left.isString() && right.isDateOrDateTime()) - || (left.isDateOrDateTime() && right.isString()) || (left.isDateOrDateTime() && right.isDateOrDateTime() && left.idx == right.idx) /// only date vs date, or datetime vs datetime || (left.isUUID() && right.isUUID()) - || (left.isUUID() && right.isString()) - || (left.isString() && right.isUUID()) || (left.isEnum() && right.isEnum() && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against - || (left.isEnum() && right.isString()) - || (left.isString() && right.isEnum()) || (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size()) || (arguments[0]->equals(*arguments[1])))) { @@ -1151,7 +1102,8 @@ public: if (left_tuple && right_tuple) { - auto adaptor = FunctionOverloadResolverAdaptor(std::make_unique(FunctionComparison::create(context))); + auto adaptor = FunctionOverloadResolverAdaptor(std::make_unique( + FunctionComparison::create(context))); size_t size = left_tuple->getElements().size(); for (size_t i = 0; i < size; ++i) @@ -1201,6 +1153,9 @@ public: const bool left_is_num = col_left_untyped->isNumeric(); const bool right_is_num = col_right_untyped->isNumeric(); + const bool left_is_string = isStringOrFixedString(which_left); + const bool right_is_string = isStringOrFixedString(which_right); + bool date_and_datetime = (left_type != right_type) && which_left.isDateOrDateTime() && which_right.isDateOrDateTime(); @@ -1226,64 +1181,14 @@ public: { executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count); } - else if (which_left.idx != which_right.idx - && (which_left.isDateTime64() || which_right.isDateTime64()) - && (which_left.isStringOrFixedString() || which_right.isStringOrFixedString())) + else if (left_is_string && right_is_string && executeString(block, result, col_left_untyped, col_right_untyped)) + { + } + else if (executeWithConstString( + block, result, col_left_untyped, col_right_untyped, + left_type, right_type, + input_rows_count)) { - /** Special case of comparing DateTime64 against a string. - * - * Can't be moved to executeDateOrDateTimeOrEnumOrUUIDWithConstString() - * since DateTime64 is basically a Decimal, but we do similar things, except type inference. - * Outline: - * - Extract string content - * - Parse it as a ColumnDateTime64 value (same type as DateTime64, means same precision) - * - Fabricate a column with type and name - * - Compare left and right comlumns as DateTime64 columns. - */ - - const size_t datetime64_col_index = which_left.isDateTime64() ? 0 : 1; - const size_t string_col_index = which_left.isStringOrFixedString() ? 0 : 1; - - const auto & datetime64_col_with_type_and_name = block.getByPosition(arguments[datetime64_col_index]); - const auto & string_col_with_type_and_name = block.getByPosition(arguments[string_col_index]); - - if (!isColumnConst(*string_col_with_type_and_name.column)) - throw Exception(getName() + ", illegal column type of argument #" + std::to_string(string_col_index) - + " '" + string_col_with_type_and_name.name + "'" - " expected const String or const FixedString," - " got " + string_col_with_type_and_name.type->getName(), - ErrorCodes::ILLEGAL_COLUMN); - - if (datetime64_col_with_type_and_name.column->size() == 0 || string_col_with_type_and_name.column->size() == 0) - { - // For some reason, when both left and right columns are empty (dry run while building a header block) - // executeDecimal() fills result column with bogus value. - block.getByPosition(result).column = ColumnUInt8::create(); - return; - } - - auto parsed_tmp_column_holder = datetime64_col_with_type_and_name.type->createColumn(); - - { - const StringRef string_value = string_col_with_type_and_name.column->getDataAt(0); - ReadBufferFromMemory in(string_value.data, string_value.size); - datetime64_col_with_type_and_name.type->deserializeAsWholeText(*parsed_tmp_column_holder, in, FormatSettings{}); - - if (!in.eof()) - throw Exception(getName() + ": String is too long for " + datetime64_col_with_type_and_name.type->getName() + " : " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - } - - // It is necessary to wrap tmp column in ColumnConst to avoid overflow when comparing. - // (non-const columns are expected to have same number of rows as every other column in block). - const ColumnWithTypeAndName parsed_tmp_col_with_type_and_name{ - ColumnConst::create(std::move(parsed_tmp_column_holder), 1), - datetime64_col_with_type_and_name.type, - string_col_with_type_and_name.name}; - - executeDecimal(block, result, - which_left.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name, - which_right.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name); - } else if (isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type)) { @@ -1294,19 +1199,10 @@ public: executeDecimal(block, result, col_with_type_and_name_left, col_with_type_and_name_right); } - else if (!left_is_num && !right_is_num && executeString(block, result, col_left_untyped, col_right_untyped)) - { - } else if (left_type->equals(*right_type)) { executeGenericIdenticalTypes(block, result, col_left_untyped, col_right_untyped); } - else if (executeDateOrDateTimeOrEnumOrUUIDWithConstString( - block, result, col_left_untyped, col_right_untyped, - left_type, right_type, - left_is_num, input_rows_count)) - { - } else { executeGeneric(block, result, col_with_type_and_name_left, col_with_type_and_name_right); diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index fab1332cbda..50214ee790f 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index dc48b5347c4..7e32d9514a6 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -1,4 +1,6 @@ #include +#include + namespace DB { diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 1753508e139..0e1d60d0d7a 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include + namespace DB { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 98cf36cc30b..297679c4616 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -37,7 +37,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 3cb774596c0..d46573d0461 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -33,8 +33,6 @@ namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; extern const int TYPE_MISMATCH; - extern const int TOO_LARGE_STRING_SIZE; - extern const int CANNOT_CONVERT_TYPE; } @@ -124,42 +122,6 @@ static Field convertDecimalType(const Field & from, const To & type) } -DayNum stringToDate(const String & s) -{ - ReadBufferFromString in(s); - DayNum date{}; - - readDateText(date, in); - if (!in.eof()) - throw Exception("String is too long for Date: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE); - - 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, ErrorCodes::TOO_LARGE_STRING_SIZE); - - return UInt64(date_time); -} - -DateTime64::NativeType stringToDateTime64(const String & s, UInt32 scale) -{ - ReadBufferFromString in(s); - DateTime64 datetime64 {0}; - - readDateTime64Text(datetime64, scale, in); - if (!in.eof()) - throw Exception("String is too long for DateTime64: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE); - - return datetime64.value; -} - Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const IDataType * from_type_hint) { WhichDataType which_type(type); @@ -184,7 +146,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } - else if (type.isValueRepresentedByNumber()) + else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { if (which_type.isUInt8()) return convertNumericType(src, type); if (which_type.isUInt16()) return convertNumericType(src, type); @@ -200,9 +162,6 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); - if (!which_type.isDateOrDateTime() && !which_type.isUUID() && !which_type.isEnum()) - throw Exception{"Cannot convert field to type " + type.getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; - if (which_type.isEnum() && (src.getType() == Field::Types::UInt64 || src.getType() == Field::Types::Int64)) { /// Convert UInt64 or Int64 to Enum's value @@ -214,36 +173,20 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID /// We don't need any conversion UInt64 is under type of Date and DateTime return src; } - // TODO (vnemkov): extra cases for DateTime64: converting from integer, converting from Decimal - if (src.getType() == Field::Types::String) + if (which_type.isUUID() && src.getType() == Field::Types::UInt128) { - if (which_type.isDate()) - { - /// Convert 'YYYY-MM-DD' Strings to Date - return stringToDate(src.get()); - } - else if (which_type.isDateTime()) - { - /// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime - return stringToDateTime(src.get()); - } - else if (which_type.isDateTime64()) - { - const auto * date_time64 = typeid_cast(&type); - /// Convert 'YYYY-MM-DD hh:mm:ss.NNNNNNNNN' Strings to DateTime - return stringToDateTime64(src.get(), date_time64->getScale()); - } - else if (which_type.isUUID()) - { - return stringToUUID(src.get()); - } - else if (which_type.isEnum()) - { - /// Convert String to Enum's value - return dynamic_cast(type).castToValue(src); - } + /// Already in needed type. + return src; } + + if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64) + { + /// Already in needed type. + return src; + } + + /// TODO Conversion from integers to DateTime64 } else if (which_type.isStringOrFixedString()) { @@ -328,17 +271,37 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return src; } + /// Conversion from string by parsing. if (src.getType() == Field::Types::String) { - const auto col = type.createColumn(); - ReadBufferFromString buffer(src.get()); - type.deserializeAsTextEscaped(*col, buffer, FormatSettings{}); + /// Promote data type to avoid overflows. Note that overflows in the largest data type are still possible. + const IDataType * type_to_parse = &type; + DataTypePtr holder; - return (*col)[0]; + if (type.canBePromoted()) + { + holder = type.promoteNumericType(); + type_to_parse = holder.get(); + } + + const auto col = type_to_parse->createColumn(); + ReadBufferFromString in_buffer(src.get()); + try + { + type_to_parse->deserializeAsWholeText(*col, in_buffer, FormatSettings{}); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while converting '{}' to {}", src.get(), type.getName())); + throw; + } + if (!in_buffer.eof()) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string {} to type {}", src.get(), type.getName()); + + Field parsed = (*col)[0]; + return convertFieldToType(parsed, type, from_type_hint); } - - // TODO (nemkov): should we attempt to parse value using or `type.deserializeAsTextEscaped()` type.deserializeAsTextEscaped() ? throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: " + Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dad73b6a003..281f8511a59 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -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); diff --git a/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql b/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql index 3071e307517..e729bfdf188 100644 --- a/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql +++ b/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS numbers2; CREATE TABLE numbers1 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); CREATE TABLE numbers2 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); -SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 43 } +SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 53 } SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table=1; DROP TABLE numbers1; diff --git a/tests/queries/0_stateless/00700_decimal_compare.reference b/tests/queries/0_stateless/00700_decimal_compare.reference index 32f0b0a6dea..2325847045f 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.reference +++ b/tests/queries/0_stateless/00700_decimal_compare.reference @@ -1,3 +1,5 @@ +0 +1 -42 -42 1 0 0 0 1 1 42 42 1 0 0 0 1 1 -42 -42.42000 0 0 1 1 0 1 diff --git a/tests/queries/0_stateless/00700_decimal_compare.sql b/tests/queries/0_stateless/00700_decimal_compare.sql index 24b4ce588e5..ae2f5790570 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/tests/queries/0_stateless/00700_decimal_compare.sql @@ -19,7 +19,7 @@ INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, SELECT a > toFloat64(0) FROM decimal; -- { serverError 43 } SELECT g > toFloat32(0) FROM decimal; -- { serverError 43 } -SELECT a > '0.0' FROM decimal; -- { serverError 43 } +SELECT a > '0.0' FROM decimal ORDER BY a; SELECT a, b, a = b, a < b, a > b, a != b, a <= b, a >= b FROM decimal ORDER BY a; SELECT a, g, a = g, a < g, a > g, a != g, a <= g, a >= g FROM decimal ORDER BY a; diff --git a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql index c65bf668d71..4ffcf3be3c9 100644 --- a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql +++ b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql @@ -1,12 +1,12 @@ -- Error cases: -- non-const string column -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = materialize(S); -- {serverError 44} -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materialize(S) = toDateTime64(S, 3); -- {serverError 44} -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 44} -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 44} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = materialize(S); -- {serverError 43} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materialize(S) = toDateTime64(S, 3); -- {serverError 43} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 43} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 43} -SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 131} -- invalid DateTime64 string -SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; -- {serverError 131} -- invalid string length +SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 53} -- invalid DateTime64 string +SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; -- {serverError 53} -- invalid string length SELECT 'in SELECT'; WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = S; diff --git a/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql b/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql index 05295575cf1..4712c124237 100644 --- a/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql +++ b/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql @@ -1 +1 @@ -SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 70 } +SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 53 } diff --git a/tests/queries/0_stateless/01307_multiple_leaders.reference b/tests/queries/0_stateless/01307_multiple_leaders.reference index 576441b288d..62cda31dff8 100644 --- a/tests/queries/0_stateless/01307_multiple_leaders.reference +++ b/tests/queries/0_stateless/01307_multiple_leaders.reference @@ -1,2 +1,2 @@ -2000 1999000 -2000 1999000 +400 79800 +400 79800 diff --git a/tests/queries/0_stateless/01307_multiple_leaders.sh b/tests/queries/0_stateless/01307_multiple_leaders.sh index b16feaeb591..a43aa074c43 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e NUM_REPLICAS=2 -DATA_SIZE=1000 +DATA_SIZE=200 SEQ=$(seq 0 $(($NUM_REPLICAS - 1))) diff --git a/tests/queries/0_stateless/01310_enum_comparison.reference b/tests/queries/0_stateless/01310_enum_comparison.reference new file mode 100644 index 00000000000..b261da18d51 --- /dev/null +++ b/tests/queries/0_stateless/01310_enum_comparison.reference @@ -0,0 +1,2 @@ +1 +0 diff --git a/tests/queries/0_stateless/01310_enum_comparison.sql b/tests/queries/0_stateless/01310_enum_comparison.sql new file mode 100644 index 00000000000..26901a61b2b --- /dev/null +++ b/tests/queries/0_stateless/01310_enum_comparison.sql @@ -0,0 +1,6 @@ +CREATE TEMPORARY TABLE enum (x Enum('hello' = 1, 'world' = 2)); +INSERT INTO enum VALUES ('hello'); + +SELECT count() FROM enum WHERE x = 'hello'; +SELECT count() FROM enum WHERE x = 'world'; +SELECT count() FROM enum WHERE x = 'xyz'; -- { serverError 36 } diff --git a/tests/queries/0_stateless/01311_comparison_with_constant_string.reference b/tests/queries/0_stateless/01311_comparison_with_constant_string.reference new file mode 100644 index 00000000000..7865f42932d --- /dev/null +++ b/tests/queries/0_stateless/01311_comparison_with_constant_string.reference @@ -0,0 +1,32 @@ +0 +1 +0 +--- +1 +0 +1 +--- +1 +0 +0 +--- +0 +--- +1 +0 +--- +--- +0 +--- +1 +--- +0 +--- +0 +--- +0 +--- +0 +--- +1 +--- diff --git a/tests/queries/0_stateless/01311_comparison_with_constant_string.sql b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql new file mode 100644 index 00000000000..d6641a50c45 --- /dev/null +++ b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql @@ -0,0 +1,33 @@ +SELECT number = '1' FROM numbers(3); +SELECT '---'; +SELECT '1' != number FROM numbers(3); +SELECT '---'; +SELECT '1' > number FROM numbers(3); +SELECT '---'; +SELECT 1 = '257'; +SELECT '---'; +SELECT 1 IN (1.23, '1', 2); +SELECT 1 IN (1.23, '2', 2); +SELECT '---'; + +-- it should work but it doesn't. +SELECT 1 = '1.0'; -- { serverError 53 } +SELECT '---'; + +SELECT 1 = '257'; +SELECT '---'; +SELECT 1 != '257'; +SELECT '---'; +SELECT 1 < '257'; -- this is wrong for now +SELECT '---'; +SELECT 1 > '257'; +SELECT '---'; +SELECT 1 <= '257'; -- this is wrong for now +SELECT '---'; +SELECT 1 >= '257'; +SELECT '---'; + +SELECT toDateTime('2020-06-13 01:02:03') = '2020-06-13T01:02:03'; +SELECT '---'; + +SELECT 0 = ''; -- { serverError 32 } diff --git a/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.reference b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.reference new file mode 100644 index 00000000000..ee98bdf033b --- /dev/null +++ b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.reference @@ -0,0 +1,12 @@ +1 +999999 +100000 +899999 +100001 +900000 +1 +999999 +100000 +899999 +100001 +900000 diff --git a/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql new file mode 100644 index 00000000000..e37f647e81f --- /dev/null +++ b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql @@ -0,0 +1,32 @@ +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;