This commit is contained in:
Alexander Kuzmenkov 2020-10-23 17:28:55 +03:00
parent d17a49f127
commit b0a14a41c3
4 changed files with 194 additions and 184 deletions

View File

@ -154,22 +154,6 @@ struct NumIfImpl<Decimal<A>, Decimal<B>, Decimal<R>>
} }
}; };
template <typename A, typename B>
struct NumIfImpl<A, B, NumberTraits::Error>
{
private:
[[noreturn]] static void throwError()
{
throw Exception("Invalid types of arguments 2 and 3 of if", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
public:
template <typename... Args> static void vectorVector(Args &&...) { throwError(); }
template <typename... Args> static void vectorConstant(Args &&...) { throwError(); }
template <typename... Args> static void constantVector(Args &&...) { throwError(); }
template <typename... Args> static void constantConstant(Args &&...) { throwError(); }
};
class FunctionIf : public FunctionIfBase</*null_is_false=*/false> class FunctionIf : public FunctionIfBase</*null_is_false=*/false>
{ {
public: public:
@ -205,6 +189,17 @@ private:
const IColumn * col_right_untyped = columns[arguments[2]].column.get(); const IColumn * col_right_untyped = columns[arguments[2]].column.get();
UInt32 scale = decimalScale<T0, T1>(columns, arguments); UInt32 scale = decimalScale<T0, T1>(columns, arguments);
if constexpr (std::is_same_v<ResultType, NumberTraits::Error>)
{
const auto & arg_left = columns[arguments[1]];
const auto & arg_right = columns[arguments[2]];
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Incompatible types of arguments of function {}:"
" '{}' and '{}'", getName(), arg_left.type->getName(),
arg_right.type->getName());
}
else
{
if (const auto * col_right_vec = checkAndGetColumn<ColVecT1>(col_right_untyped)) if (const auto * col_right_vec = checkAndGetColumn<ColVecT1>(col_right_untyped))
{ {
NumIfImpl<T0, T1, ResultType>::vectorVector( NumIfImpl<T0, T1, ResultType>::vectorVector(
@ -217,6 +212,7 @@ private:
cond_col->getData(), col_left->getData(), col_right_const->template getValue<T1>(), columns, result, scale); cond_col->getData(), col_left->getData(), col_right_const->template getValue<T1>(), columns, result, scale);
return true; return true;
} }
}
return false; return false;
} }
@ -234,6 +230,17 @@ private:
const IColumn * col_right_untyped = columns[arguments[2]].column.get(); const IColumn * col_right_untyped = columns[arguments[2]].column.get();
UInt32 scale = decimalScale<T0, T1>(columns, arguments); UInt32 scale = decimalScale<T0, T1>(columns, arguments);
if constexpr (std::is_same_v<ResultType, NumberTraits::Error>)
{
const auto & arg_left = columns[arguments[1]];
const auto & arg_right = columns[arguments[2]];
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Incompatible types of arguments of function {}:"
" '{}' and '{}'", getName(), arg_left.type->getName(),
arg_right.type->getName());
}
else
{
if (const auto * col_right_vec = checkAndGetColumn<ColVecT1>(col_right_untyped)) if (const auto * col_right_vec = checkAndGetColumn<ColVecT1>(col_right_untyped))
{ {
NumIfImpl<T0, T1, ResultType>::constantVector( NumIfImpl<T0, T1, ResultType>::constantVector(
@ -246,6 +253,7 @@ private:
cond_col->getData(), col_left->template getValue<T0>(), col_right_const->template getValue<T1>(), columns, result, scale); cond_col->getData(), col_left->template getValue<T0>(), col_right_const->template getValue<T1>(), columns, result, scale);
return true; return true;
} }
}
return false; return false;
} }

View File

@ -316,6 +316,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
{ {
auto & result = sample_block.getByName(result_name); auto & result = sample_block.getByName(result_name);
result.type = result_type; result.type = result_type;
result.name = result_name;
result.column = source.column; result.column = source.column;
} }
else else
@ -1561,6 +1562,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
ColumnWithTypeAndName argument; ColumnWithTypeAndName argument;
argument.column = child.column; argument.column = child.column;
argument.type = child.result_type; argument.type = child.result_type;
argument.name = child.result_name;
if (!argument.column || !isColumnConst(*argument.column)) if (!argument.column || !isColumnConst(*argument.column))
all_const = false; all_const = false;

View File

@ -1,6 +1,6 @@
SELECT toTimeZone(N, \'UTC\') SELECT toTimeZone(N, \'UTC\')
Code: 43: Illegal type Date of argument of function toTimeZone. Should be DateTime or DateTime64. Code: 43
"DateTime('UTC')","2019-09-16 16:20:11" "DateTime('UTC')","2019-09-16 16:20:11"
"DateTime64(3, 'UTC')","2019-09-16 16:20:11.234" "DateTime64(3, 'UTC')","2019-09-16 16:20:11.234"
------------------------------------------ ------------------------------------------
@ -36,19 +36,19 @@ SELECT toDayOfWeek(N)
------------------------------------------ ------------------------------------------
SELECT toHour(N) SELECT toHour(N)
Code: 43: Illegal type Date of argument for function toHour. Code: 43
"UInt8",19 "UInt8",19
"UInt8",19 "UInt8",19
------------------------------------------ ------------------------------------------
SELECT toMinute(N) SELECT toMinute(N)
Code: 43: Illegal type Date of argument for function toMinute. Code: 43
"UInt8",20 "UInt8",20
"UInt8",20 "UInt8",20
------------------------------------------ ------------------------------------------
SELECT toSecond(N) SELECT toSecond(N)
Code: 43: Illegal type Date of argument for function toSecond. Code: 43
"UInt8",11 "UInt8",11
"UInt8",11 "UInt8",11
------------------------------------------ ------------------------------------------
@ -94,31 +94,31 @@ SELECT toStartOfDay(N)
------------------------------------------ ------------------------------------------
SELECT toStartOfHour(N) SELECT toStartOfHour(N)
Code: 43: Illegal type Date of argument for function toStartOfHour. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:00:00" "DateTime('Europe/Minsk')","2019-09-16 19:00:00"
"DateTime('Europe/Minsk')","2019-09-16 19:00:00" "DateTime('Europe/Minsk')","2019-09-16 19:00:00"
------------------------------------------ ------------------------------------------
SELECT toStartOfMinute(N) SELECT toStartOfMinute(N)
Code: 43: Illegal type Date of argument for function toStartOfMinute. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
------------------------------------------ ------------------------------------------
SELECT toStartOfFiveMinute(N) SELECT toStartOfFiveMinute(N)
Code: 43: Illegal type Date of argument for function toStartOfFiveMinute. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
------------------------------------------ ------------------------------------------
SELECT toStartOfTenMinutes(N) SELECT toStartOfTenMinutes(N)
Code: 43: Illegal type Date of argument for function toStartOfTenMinutes. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
------------------------------------------ ------------------------------------------
SELECT toStartOfFifteenMinutes(N) SELECT toStartOfFifteenMinutes(N)
Code: 43: Illegal type Date of argument for function toStartOfFifteenMinutes. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:15:00" "DateTime('Europe/Minsk')","2019-09-16 19:15:00"
"DateTime('Europe/Minsk')","2019-09-16 19:15:00" "DateTime('Europe/Minsk')","2019-09-16 19:15:00"
------------------------------------------ ------------------------------------------
@ -139,7 +139,7 @@ SELECT toStartOfInterval(N, INTERVAL 1 day)
------------------------------------------ ------------------------------------------
SELECT toStartOfInterval(N, INTERVAL 15 minute) SELECT toStartOfInterval(N, INTERVAL 15 minute)
Code: 43: Illegal type Date of argument for function toStartOfInterval. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:15:00" "DateTime('Europe/Minsk')","2019-09-16 19:15:00"
"DateTime('Europe/Minsk')","2019-09-16 19:15:00" "DateTime('Europe/Minsk')","2019-09-16 19:15:00"
------------------------------------------ ------------------------------------------
@ -160,13 +160,13 @@ SELECT date_trunc(\'day\', N)
------------------------------------------ ------------------------------------------
SELECT date_trunc(\'minute\', N) SELECT date_trunc(\'minute\', N)
Code: 43: Illegal type Date of argument for function date_trunc. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
"DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00"
------------------------------------------ ------------------------------------------
SELECT toTime(N) SELECT toTime(N)
Code: 43: Illegal type Date of argument for function toTime. Code: 43
"DateTime('Europe/Minsk')","1970-01-02 19:20:11" "DateTime('Europe/Minsk')","1970-01-02 19:20:11"
"DateTime('Europe/Minsk')","1970-01-02 19:20:11" "DateTime('Europe/Minsk')","1970-01-02 19:20:11"
------------------------------------------ ------------------------------------------
@ -232,7 +232,7 @@ SELECT toYearWeek(N)
------------------------------------------ ------------------------------------------
SELECT timeSlot(N) SELECT timeSlot(N)
Code: 43: Illegal type Date of argument for function timeSlot. Code: 43
"DateTime('Europe/Minsk')","2019-09-16 19:00:00" "DateTime('Europe/Minsk')","2019-09-16 19:00:00"
"DateTime('Europe/Minsk')","2019-09-16 19:00:00" "DateTime('Europe/Minsk')","2019-09-16 19:00:00"
------------------------------------------ ------------------------------------------
@ -375,15 +375,15 @@ SELECT N - N
"Int32",0 "Int32",0
"Int32",0 "Int32",0
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function minus. Code: 43
------------------------------------------ ------------------------------------------
SELECT N + N SELECT N + N
Code: 43: Illegal types Date and Date of arguments of function plus. Code: 43
Code: 43: Illegal types DateTime('Europe/Minsk') and DateTime('Europe/Minsk') of arguments of function plus. Code: 43
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function plus. Code: 43
------------------------------------------ ------------------------------------------
SELECT N != N SELECT N != N
"UInt8",0 "UInt8",0
@ -417,47 +417,47 @@ SELECT N >= N
------------------------------------------ ------------------------------------------
SELECT N - DT SELECT N - DT
Code: 43: Illegal types Date and DateTime('Europe/Minsk') of arguments of function minus. Code: 43
"Int32",0 "Int32",0
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime('Europe/Minsk') of arguments of function minus. Code: 43
------------------------------------------ ------------------------------------------
SELECT DT - N SELECT DT - N
Code: 43: Illegal types DateTime('Europe/Minsk') and Date of arguments of function minus. Code: 43
"Int32",0 "Int32",0
Code: 43: Illegal types DateTime('Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function minus. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - D SELECT N - D
"Int32",0 "Int32",0
Code: 43: Illegal types DateTime('Europe/Minsk') and Date of arguments of function minus. Code: 43
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Date of arguments of function minus. Code: 43
------------------------------------------ ------------------------------------------
SELECT D - N SELECT D - N
"Int32",0 "Int32",0
Code: 43: Illegal types Date and DateTime('Europe/Minsk') of arguments of function minus. Code: 43
Code: 43: Illegal types Date and DateTime64(3, 'Europe/Minsk') of arguments of function minus. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - DT64 SELECT N - DT64
Code: 43: Illegal types Date and DateTime64(3, 'Europe/Minsk') of arguments of function minus. Code: 43
Code: 43: Illegal types DateTime('Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function minus. Code: 43
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function minus. Code: 43
------------------------------------------ ------------------------------------------
SELECT DT64 - N SELECT DT64 - N
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Date of arguments of function minus. Code: 43
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime('Europe/Minsk') of arguments of function minus. Code: 43
Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function minus. Code: 43
------------------------------------------ ------------------------------------------
SELECT N != DT SELECT N != DT
"UInt8",1 "UInt8",1
@ -726,11 +726,11 @@ SELECT N - toUInt8(1)
------------------------------------------ ------------------------------------------
SELECT toUInt8(1) - N SELECT toUInt8(1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - toInt8(-1) SELECT N - toInt8(-1)
"Date","2019-09-17" "Date","2019-09-17"
@ -739,11 +739,11 @@ SELECT N - toInt8(-1)
------------------------------------------ ------------------------------------------
SELECT toInt8(-1) - N SELECT toInt8(-1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - toUInt16(1) SELECT N - toUInt16(1)
"Date","2019-09-15" "Date","2019-09-15"
@ -752,11 +752,11 @@ SELECT N - toUInt16(1)
------------------------------------------ ------------------------------------------
SELECT toUInt16(1) - N SELECT toUInt16(1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - toInt16(-1) SELECT N - toInt16(-1)
"Date","2019-09-17" "Date","2019-09-17"
@ -765,11 +765,11 @@ SELECT N - toInt16(-1)
------------------------------------------ ------------------------------------------
SELECT toInt16(-1) - N SELECT toInt16(-1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - toUInt32(1) SELECT N - toUInt32(1)
"Date","2019-09-15" "Date","2019-09-15"
@ -778,11 +778,11 @@ SELECT N - toUInt32(1)
------------------------------------------ ------------------------------------------
SELECT toUInt32(1) - N SELECT toUInt32(1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - toInt32(-1) SELECT N - toInt32(-1)
"Date","2019-09-17" "Date","2019-09-17"
@ -791,11 +791,11 @@ SELECT N - toInt32(-1)
------------------------------------------ ------------------------------------------
SELECT toInt32(-1) - N SELECT toInt32(-1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - toUInt64(1) SELECT N - toUInt64(1)
"Date","2019-09-15" "Date","2019-09-15"
@ -804,11 +804,11 @@ SELECT N - toUInt64(1)
------------------------------------------ ------------------------------------------
SELECT toUInt64(1) - N SELECT toUInt64(1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N - toInt64(-1) SELECT N - toInt64(-1)
"Date","2019-09-17" "Date","2019-09-17"
@ -817,585 +817,585 @@ SELECT N - toInt64(-1)
------------------------------------------ ------------------------------------------
SELECT toInt64(-1) - N SELECT toInt64(-1) - N
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. Code: 43
------------------------------------------ ------------------------------------------
SELECT N == toUInt8(1) SELECT N == toUInt8(1)
Code: 43: Illegal types of arguments (Date, UInt8) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt8(1) == N SELECT toUInt8(1) == N
Code: 43: Illegal types of arguments (UInt8, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N == toInt8(-1) SELECT N == toInt8(-1)
Code: 43: Illegal types of arguments (Date, Int8) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt8(-1) == N SELECT toInt8(-1) == N
Code: 43: Illegal types of arguments (Int8, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N == toUInt16(1) SELECT N == toUInt16(1)
Code: 43: Illegal types of arguments (Date, UInt16) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt16(1) == N SELECT toUInt16(1) == N
Code: 43: Illegal types of arguments (UInt16, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N == toInt16(-1) SELECT N == toInt16(-1)
Code: 43: Illegal types of arguments (Date, Int16) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt16(-1) == N SELECT toInt16(-1) == N
Code: 43: Illegal types of arguments (Int16, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N == toUInt32(1) SELECT N == toUInt32(1)
Code: 43: Illegal types of arguments (Date, UInt32) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt32(1) == N SELECT toUInt32(1) == N
Code: 43: Illegal types of arguments (UInt32, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N == toInt32(-1) SELECT N == toInt32(-1)
Code: 43: Illegal types of arguments (Date, Int32) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt32(-1) == N SELECT toInt32(-1) == N
Code: 43: Illegal types of arguments (Int32, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N == toUInt64(1) SELECT N == toUInt64(1)
Code: 43: Illegal types of arguments (Date, UInt64) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt64(1) == N SELECT toUInt64(1) == N
Code: 43: Illegal types of arguments (UInt64, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N == toInt64(-1) SELECT N == toInt64(-1)
Code: 43: Illegal types of arguments (Date, Int64) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt64(-1) == N SELECT toInt64(-1) == N
Code: 43: Illegal types of arguments (Int64, Date) of function equals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N != toUInt8(1) SELECT N != toUInt8(1)
Code: 43: Illegal types of arguments (Date, UInt8) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt8(1) != N SELECT toUInt8(1) != N
Code: 43: Illegal types of arguments (UInt8, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N != toInt8(-1) SELECT N != toInt8(-1)
Code: 43: Illegal types of arguments (Date, Int8) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt8(-1) != N SELECT toInt8(-1) != N
Code: 43: Illegal types of arguments (Int8, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N != toUInt16(1) SELECT N != toUInt16(1)
Code: 43: Illegal types of arguments (Date, UInt16) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt16(1) != N SELECT toUInt16(1) != N
Code: 43: Illegal types of arguments (UInt16, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N != toInt16(-1) SELECT N != toInt16(-1)
Code: 43: Illegal types of arguments (Date, Int16) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt16(-1) != N SELECT toInt16(-1) != N
Code: 43: Illegal types of arguments (Int16, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N != toUInt32(1) SELECT N != toUInt32(1)
Code: 43: Illegal types of arguments (Date, UInt32) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt32(1) != N SELECT toUInt32(1) != N
Code: 43: Illegal types of arguments (UInt32, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N != toInt32(-1) SELECT N != toInt32(-1)
Code: 43: Illegal types of arguments (Date, Int32) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt32(-1) != N SELECT toInt32(-1) != N
Code: 43: Illegal types of arguments (Int32, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N != toUInt64(1) SELECT N != toUInt64(1)
Code: 43: Illegal types of arguments (Date, UInt64) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt64(1) != N SELECT toUInt64(1) != N
Code: 43: Illegal types of arguments (UInt64, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N != toInt64(-1) SELECT N != toInt64(-1)
Code: 43: Illegal types of arguments (Date, Int64) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt64(-1) != N SELECT toInt64(-1) != N
Code: 43: Illegal types of arguments (Int64, Date) of function notEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toUInt8(1) SELECT N < toUInt8(1)
Code: 43: Illegal types of arguments (Date, UInt8) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt8(1) < N SELECT toUInt8(1) < N
Code: 43: Illegal types of arguments (UInt8, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toInt8(-1) SELECT N < toInt8(-1)
Code: 43: Illegal types of arguments (Date, Int8) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt8(-1) < N SELECT toInt8(-1) < N
Code: 43: Illegal types of arguments (Int8, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toUInt16(1) SELECT N < toUInt16(1)
Code: 43: Illegal types of arguments (Date, UInt16) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt16(1) < N SELECT toUInt16(1) < N
Code: 43: Illegal types of arguments (UInt16, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toInt16(-1) SELECT N < toInt16(-1)
Code: 43: Illegal types of arguments (Date, Int16) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt16(-1) < N SELECT toInt16(-1) < N
Code: 43: Illegal types of arguments (Int16, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toUInt32(1) SELECT N < toUInt32(1)
Code: 43: Illegal types of arguments (Date, UInt32) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt32(1) < N SELECT toUInt32(1) < N
Code: 43: Illegal types of arguments (UInt32, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toInt32(-1) SELECT N < toInt32(-1)
Code: 43: Illegal types of arguments (Date, Int32) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt32(-1) < N SELECT toInt32(-1) < N
Code: 43: Illegal types of arguments (Int32, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toUInt64(1) SELECT N < toUInt64(1)
Code: 43: Illegal types of arguments (Date, UInt64) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt64(1) < N SELECT toUInt64(1) < N
Code: 43: Illegal types of arguments (UInt64, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N < toInt64(-1) SELECT N < toInt64(-1)
Code: 43: Illegal types of arguments (Date, Int64) of function less. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt64(-1) < N SELECT toInt64(-1) < N
Code: 43: Illegal types of arguments (Int64, Date) of function less. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toUInt8(1) SELECT N <= toUInt8(1)
Code: 43: Illegal types of arguments (Date, UInt8) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt8(1) <= N SELECT toUInt8(1) <= N
Code: 43: Illegal types of arguments (UInt8, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toInt8(-1) SELECT N <= toInt8(-1)
Code: 43: Illegal types of arguments (Date, Int8) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt8(-1) <= N SELECT toInt8(-1) <= N
Code: 43: Illegal types of arguments (Int8, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toUInt16(1) SELECT N <= toUInt16(1)
Code: 43: Illegal types of arguments (Date, UInt16) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt16(1) <= N SELECT toUInt16(1) <= N
Code: 43: Illegal types of arguments (UInt16, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toInt16(-1) SELECT N <= toInt16(-1)
Code: 43: Illegal types of arguments (Date, Int16) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt16(-1) <= N SELECT toInt16(-1) <= N
Code: 43: Illegal types of arguments (Int16, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toUInt32(1) SELECT N <= toUInt32(1)
Code: 43: Illegal types of arguments (Date, UInt32) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt32(1) <= N SELECT toUInt32(1) <= N
Code: 43: Illegal types of arguments (UInt32, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toInt32(-1) SELECT N <= toInt32(-1)
Code: 43: Illegal types of arguments (Date, Int32) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt32(-1) <= N SELECT toInt32(-1) <= N
Code: 43: Illegal types of arguments (Int32, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toUInt64(1) SELECT N <= toUInt64(1)
Code: 43: Illegal types of arguments (Date, UInt64) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toUInt64(1) <= N SELECT toUInt64(1) <= N
Code: 43: Illegal types of arguments (UInt64, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N <= toInt64(-1) SELECT N <= toInt64(-1)
Code: 43: Illegal types of arguments (Date, Int64) of function lessOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT toInt64(-1) <= N SELECT toInt64(-1) <= N
Code: 43: Illegal types of arguments (Int64, Date) of function lessOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT N > toUInt8(1) SELECT N > toUInt8(1)
Code: 43: Illegal types of arguments (Date, UInt8) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt8(1) > N SELECT toUInt8(1) > N
Code: 43: Illegal types of arguments (UInt8, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N > toInt8(-1) SELECT N > toInt8(-1)
Code: 43: Illegal types of arguments (Date, Int8) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt8(-1) > N SELECT toInt8(-1) > N
Code: 43: Illegal types of arguments (Int8, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N > toUInt16(1) SELECT N > toUInt16(1)
Code: 43: Illegal types of arguments (Date, UInt16) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt16(1) > N SELECT toUInt16(1) > N
Code: 43: Illegal types of arguments (UInt16, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N > toInt16(-1) SELECT N > toInt16(-1)
Code: 43: Illegal types of arguments (Date, Int16) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt16(-1) > N SELECT toInt16(-1) > N
Code: 43: Illegal types of arguments (Int16, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N > toUInt32(1) SELECT N > toUInt32(1)
Code: 43: Illegal types of arguments (Date, UInt32) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt32(1) > N SELECT toUInt32(1) > N
Code: 43: Illegal types of arguments (UInt32, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N > toInt32(-1) SELECT N > toInt32(-1)
Code: 43: Illegal types of arguments (Date, Int32) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt32(-1) > N SELECT toInt32(-1) > N
Code: 43: Illegal types of arguments (Int32, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N > toUInt64(1) SELECT N > toUInt64(1)
Code: 43: Illegal types of arguments (Date, UInt64) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt64(1) > N SELECT toUInt64(1) > N
Code: 43: Illegal types of arguments (UInt64, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N > toInt64(-1) SELECT N > toInt64(-1)
Code: 43: Illegal types of arguments (Date, Int64) of function greater. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt64(-1) > N SELECT toInt64(-1) > N
Code: 43: Illegal types of arguments (Int64, Date) of function greater. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toUInt8(1) SELECT N >= toUInt8(1)
Code: 43: Illegal types of arguments (Date, UInt8) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt8(1) >= N SELECT toUInt8(1) >= N
Code: 43: Illegal types of arguments (UInt8, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toInt8(-1) SELECT N >= toInt8(-1)
Code: 43: Illegal types of arguments (Date, Int8) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt8(-1) >= N SELECT toInt8(-1) >= N
Code: 43: Illegal types of arguments (Int8, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toUInt16(1) SELECT N >= toUInt16(1)
Code: 43: Illegal types of arguments (Date, UInt16) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt16(1) >= N SELECT toUInt16(1) >= N
Code: 43: Illegal types of arguments (UInt16, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toInt16(-1) SELECT N >= toInt16(-1)
Code: 43: Illegal types of arguments (Date, Int16) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt16(-1) >= N SELECT toInt16(-1) >= N
Code: 43: Illegal types of arguments (Int16, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toUInt32(1) SELECT N >= toUInt32(1)
Code: 43: Illegal types of arguments (Date, UInt32) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt32(1) >= N SELECT toUInt32(1) >= N
Code: 43: Illegal types of arguments (UInt32, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toInt32(-1) SELECT N >= toInt32(-1)
Code: 43: Illegal types of arguments (Date, Int32) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt32(-1) >= N SELECT toInt32(-1) >= N
Code: 43: Illegal types of arguments (Int32, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toUInt64(1) SELECT N >= toUInt64(1)
Code: 43: Illegal types of arguments (Date, UInt64) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toUInt64(1) >= N SELECT toUInt64(1) >= N
Code: 43: Illegal types of arguments (UInt64, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------
SELECT N >= toInt64(-1) SELECT N >= toInt64(-1)
Code: 43: Illegal types of arguments (Date, Int64) of function greaterOrEquals. Code: 43
"UInt8",1 "UInt8",1
"UInt8",1 "UInt8",1
------------------------------------------ ------------------------------------------
SELECT toInt64(-1) >= N SELECT toInt64(-1) >= N
Code: 43: Illegal types of arguments (Int64, Date) of function greaterOrEquals. Code: 43
"UInt8",0 "UInt8",0
"UInt8",0 "UInt8",0
------------------------------------------ ------------------------------------------

View File

@ -12,4 +12,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
"${CURDIR}"/00921_datetime64_compatibility.python \ "${CURDIR}"/00921_datetime64_compatibility.python \
| ${CLICKHOUSE_CLIENT} --ignore-error -T -nm --calculate_text_stack_trace 0 --log-level 'error' 2>&1 \ | ${CLICKHOUSE_CLIENT} --ignore-error -T -nm --calculate_text_stack_trace 0 --log-level 'error' 2>&1 \
| sed -Ee 's/Received exception from server .*//g; s/(Code: [0-9]+). DB::Exception: Received from .* DB::Exception/\1/g' | sed 's/Received exception .*//g; s/^\(Code: [0-9]\+\).*$/\1/g'