mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #11647 from ClickHouse/simplify-code-functions-comparison-2
Allow comparison of numbers with constant string in IN and VALUES
This commit is contained in:
commit
550838adbb
@ -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.
|
||||
|
@ -16,13 +16,18 @@ void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBu
|
||||
writeText(UUID(assert_cast<const ColumnUInt128 &>(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<ColumnUInt128 &>(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);
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
#include <Functions/IFunctionAdaptors.h>
|
||||
@ -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<ColumnString>(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<const ColumnConst *>(parsed_const_date_holder.get());
|
||||
executeNumLeftType<DataTypeDate::FieldType>(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<Op<int, int>, NotEqualsOp<int, int>>);
|
||||
}
|
||||
else if (which.isDateTime())
|
||||
else
|
||||
{
|
||||
time_t date_time;
|
||||
ReadBufferFromMemory in(string_value.data, string_value.size);
|
||||
readDateTimeText(date_time, in, dynamic_cast<const DataTypeDateTime &>(*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<const ColumnConst *>(parsed_const_date_time_holder.get());
|
||||
executeNumLeftType<DataTypeDateTime::FieldType>(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<const ColumnConst *>(parsed_const_uuid_holder.get());
|
||||
executeNumLeftType<DataTypeUUID::FieldType>(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<DataTypeEnum8>(block, result, column_number, column_string,
|
||||
number_type, left_is_num, input_rows_count);
|
||||
else if (which.isEnum16())
|
||||
executeEnumWithConstString<DataTypeEnum16>(block, result, column_number, column_string,
|
||||
number_type, left_is_num, input_rows_count);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/// Comparison between DataTypeEnum<T> and string constant containing the name of an enum element
|
||||
template <typename EnumType>
|
||||
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<const EnumType *>(type_untyped);
|
||||
|
||||
const Field x = castToNearestFieldType(type->getValue(column_string->getValue<String>()));
|
||||
const auto enum_col = type->createColumnConst(input_rows_count, x);
|
||||
|
||||
executeNumLeftType<typename EnumType::FieldType>(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<DefaultOverloadResolver>(FunctionComparison<Op, Name>::create(context)));
|
||||
auto adaptor = FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(
|
||||
FunctionComparison<Op, Name>::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);
|
||||
|
@ -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<const DataTypeDateTime &>(type).getTimeZone().fromDayNum(DayNum(src.get<UInt64>()));
|
||||
}
|
||||
else if (type.isValueRepresentedByNumber())
|
||||
else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String)
|
||||
{
|
||||
if (which_type.isUInt8()) return convertNumericType<UInt8>(src, type);
|
||||
if (which_type.isUInt16()) return convertNumericType<UInt16>(src, type);
|
||||
@ -200,9 +162,6 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
if (const auto * ptype = typeid_cast<const DataTypeDecimal<Decimal64> *>(&type)) return convertDecimalType(src, *ptype);
|
||||
if (const auto * ptype = typeid_cast<const DataTypeDecimal<Decimal128> *>(&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<const String &>());
|
||||
}
|
||||
else if (which_type.isDateTime())
|
||||
{
|
||||
/// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime
|
||||
return stringToDateTime(src.get<const String &>());
|
||||
}
|
||||
else if (which_type.isDateTime64())
|
||||
{
|
||||
const auto * date_time64 = typeid_cast<const DataTypeDateTime64 *>(&type);
|
||||
/// Convert 'YYYY-MM-DD hh:mm:ss.NNNNNNNNN' Strings to DateTime
|
||||
return stringToDateTime64(src.get<const String &>(), date_time64->getScale());
|
||||
}
|
||||
else if (which_type.isUUID())
|
||||
{
|
||||
return stringToUUID(src.get<const String &>());
|
||||
}
|
||||
else if (which_type.isEnum())
|
||||
{
|
||||
/// Convert String to Enum's value
|
||||
return dynamic_cast<const IDataTypeEnum &>(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<String>());
|
||||
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<String>());
|
||||
try
|
||||
{
|
||||
type_to_parse->deserializeAsWholeText(*col, in_buffer, FormatSettings{});
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("while converting '{}' to {}", src.get<String>(), type.getName()));
|
||||
throw;
|
||||
}
|
||||
if (!in_buffer.eof())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string {} to type {}", src.get<String>(), 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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -1 +1 @@
|
||||
SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 70 }
|
||||
SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 53 }
|
||||
|
@ -0,0 +1,2 @@
|
||||
1
|
||||
0
|
6
tests/queries/0_stateless/01310_enum_comparison.sql
Normal file
6
tests/queries/0_stateless/01310_enum_comparison.sql
Normal file
@ -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 }
|
@ -0,0 +1,32 @@
|
||||
0
|
||||
1
|
||||
0
|
||||
---
|
||||
1
|
||||
0
|
||||
1
|
||||
---
|
||||
1
|
||||
0
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
1
|
||||
0
|
||||
---
|
||||
---
|
||||
0
|
||||
---
|
||||
1
|
||||
---
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
1
|
||||
---
|
@ -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 }
|
Loading…
Reference in New Issue
Block a user