mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
init
This commit is contained in:
parent
63d7697115
commit
f2731841de
@ -3,6 +3,7 @@
|
|||||||
#include <utility>
|
#include <utility>
|
||||||
|
|
||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
|
#include <DataTypes/DataTypeInterval.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -212,6 +213,8 @@ static bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... ar
|
|||||||
case TypeIndex::IPv4: return f(TypePair<DataTypeIPv4, T>(), std::forward<ExtraArgs>(args)...);
|
case TypeIndex::IPv4: return f(TypePair<DataTypeIPv4, T>(), std::forward<ExtraArgs>(args)...);
|
||||||
case TypeIndex::IPv6: return f(TypePair<DataTypeIPv6, T>(), std::forward<ExtraArgs>(args)...);
|
case TypeIndex::IPv6: return f(TypePair<DataTypeIPv6, T>(), std::forward<ExtraArgs>(args)...);
|
||||||
|
|
||||||
|
case TypeIndex::Interval: return f(TypePair<DataTypeInterval, T>(), std::forward<ExtraArgs>(args)...);
|
||||||
|
|
||||||
default:
|
default:
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -228,6 +228,40 @@ void convertUInt64toInt64IfPossible(const DataTypes & types, TypeIndexSet & type
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
DataTypePtr findSmallestIntervalSuperType(const DataTypes &types, TypeIndexSet &types_set)
|
||||||
|
{
|
||||||
|
const auto& granularity_map = getGranularityMap();
|
||||||
|
int min_granularity = std::get<0>(granularity_map.at(IntervalKind::Kind::Year));
|
||||||
|
DataTypePtr smallest_type;
|
||||||
|
|
||||||
|
bool is_higher_interval = false; // For Years, Quarters and Months
|
||||||
|
|
||||||
|
for (const auto &type : types)
|
||||||
|
{
|
||||||
|
if (const auto * interval_type = typeid_cast<const DataTypeInterval *>(type.get()))
|
||||||
|
{
|
||||||
|
int current_granularity = std::get<0>(granularity_map.at(interval_type->getKind()));
|
||||||
|
if (current_granularity > 8)
|
||||||
|
is_higher_interval = true;
|
||||||
|
if (current_granularity < min_granularity)
|
||||||
|
{
|
||||||
|
min_granularity = current_granularity;
|
||||||
|
smallest_type = type;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (is_higher_interval && min_granularity <= 8)
|
||||||
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot compare intervals {} and {} because the amount of days in month is not determined", types[0]->getName(), types[1]->getName());
|
||||||
|
|
||||||
|
if (smallest_type)
|
||||||
|
{
|
||||||
|
types_set.clear();
|
||||||
|
types_set.insert(smallest_type->getTypeId());
|
||||||
|
}
|
||||||
|
|
||||||
|
return smallest_type;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
template <LeastSupertypeOnError on_error>
|
template <LeastSupertypeOnError on_error>
|
||||||
@ -652,6 +686,13 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
|||||||
return numeric_type;
|
return numeric_type;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// For interval data types.
|
||||||
|
{
|
||||||
|
auto res = findSmallestIntervalSuperType(types, type_ids);
|
||||||
|
if (res)
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
/// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases).
|
/// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases).
|
||||||
return throwOrReturn<on_error>(types, "", ErrorCodes::NO_COMMON_TYPE);
|
return throwOrReturn<on_error>(types, "", ErrorCodes::NO_COMMON_TYPE);
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
#include <DataTypes/IDataType.h>
|
#include <DataTypes/IDataType.h>
|
||||||
|
#include <DataTypes/DataTypeInterval.h>
|
||||||
|
#include <Common/IntervalKind.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -48,4 +50,24 @@ DataTypePtr getLeastSupertypeOrString(const TypeIndexSet & types);
|
|||||||
|
|
||||||
DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types);
|
DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types);
|
||||||
|
|
||||||
|
/// A map that enumerated all interval kinds in ascending order with a conversion value to a next interval
|
||||||
|
inline const std::unordered_map<IntervalKind::Kind, std::pair<int, int>> & getGranularityMap()
|
||||||
|
{
|
||||||
|
static std::unordered_map<IntervalKind::Kind, std::pair<int, int>> granularity_map =
|
||||||
|
{
|
||||||
|
{IntervalKind::Kind::Nanosecond, {1, 1000}},
|
||||||
|
{IntervalKind::Kind::Microsecond, {2, 1000}},
|
||||||
|
{IntervalKind::Kind::Millisecond, {3, 1000}},
|
||||||
|
{IntervalKind::Kind::Second, {4, 60}},
|
||||||
|
{IntervalKind::Kind::Minute, {5, 60}},
|
||||||
|
{IntervalKind::Kind::Hour, {6, 24}},
|
||||||
|
{IntervalKind::Kind::Day, {7, 7}},
|
||||||
|
{IntervalKind::Kind::Week, {8, 4}},
|
||||||
|
{IntervalKind::Kind::Month, {9, 3}},
|
||||||
|
{IntervalKind::Kind::Quarter, {10, 4}},
|
||||||
|
{IntervalKind::Kind::Year, {11, 1}}
|
||||||
|
};
|
||||||
|
return granularity_map;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -45,6 +45,7 @@
|
|||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/ObjectUtils.h>
|
#include <DataTypes/ObjectUtils.h>
|
||||||
#include <DataTypes/Serializations/SerializationDecimal.h>
|
#include <DataTypes/Serializations/SerializationDecimal.h>
|
||||||
|
#include <DataTypes/getLeastSupertype.h>
|
||||||
#include <Formats/FormatSettings.h>
|
#include <Formats/FormatSettings.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
#include <Functions/CastOverloadResolver.h>
|
#include <Functions/CastOverloadResolver.h>
|
||||||
@ -1573,6 +1574,55 @@ struct ConvertImpl
|
|||||||
arguments, result_type, input_rows_count, additions);
|
arguments, result_type, input_rows_count, additions);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
else if constexpr (std::is_same_v<FromDataType, DataTypeInterval> && std::is_same_v<ToDataType, DataTypeInterval>)
|
||||||
|
{
|
||||||
|
IntervalKind to = typeid_cast<const DataTypeInterval *>(result_type.get())->getKind();
|
||||||
|
IntervalKind from = typeid_cast<const DataTypeInterval *>(arguments[0].type.get())->getKind();
|
||||||
|
|
||||||
|
if (from == to)
|
||||||
|
return arguments[0].column;
|
||||||
|
|
||||||
|
const auto &map = getGranularityMap();
|
||||||
|
Int64 conversion_factor = 1;
|
||||||
|
Int64 result_value;
|
||||||
|
|
||||||
|
int from_position = map.at(from).first;
|
||||||
|
int to_position = map.at(to).first; // Positions of each interval according to granurality map
|
||||||
|
|
||||||
|
if (from_position < to_position)
|
||||||
|
{
|
||||||
|
for (int i = from_position - 1; i <= to_position; ++i)
|
||||||
|
{
|
||||||
|
// Find the kind that matches this position
|
||||||
|
for (const auto &entry : map)
|
||||||
|
{
|
||||||
|
if (entry.second.first == i)
|
||||||
|
{
|
||||||
|
conversion_factor *= entry.second.second;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
result_value = arguments[0].column->getInt(0) / conversion_factor;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
for (int i = from_position - 1; i >= to_position; --i)
|
||||||
|
{
|
||||||
|
for (const auto &entry : map)
|
||||||
|
{
|
||||||
|
if (entry.second.first == i)
|
||||||
|
{
|
||||||
|
conversion_factor *= entry.second.second;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
result_value = arguments[0].column->getInt(0) * conversion_factor;
|
||||||
|
}
|
||||||
|
|
||||||
|
return ColumnConst::create(ColumnInt64::create(1, result_value), input_rows_count);
|
||||||
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
using FromFieldType = typename FromDataType::FieldType;
|
using FromFieldType = typename FromDataType::FieldType;
|
||||||
@ -2181,7 +2231,7 @@ private:
|
|||||||
const DataTypePtr from_type = removeNullable(arguments[0].type);
|
const DataTypePtr from_type = removeNullable(arguments[0].type);
|
||||||
ColumnPtr result_column;
|
ColumnPtr result_column;
|
||||||
|
|
||||||
[[maybe_unused]] FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior;
|
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior;
|
||||||
|
|
||||||
if (context)
|
if (context)
|
||||||
date_time_overflow_behavior = context->getSettingsRef().date_time_overflow_behavior.value;
|
date_time_overflow_behavior = context->getSettingsRef().date_time_overflow_behavior.value;
|
||||||
@ -2277,7 +2327,7 @@ private:
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name>::execute(arguments, result_type, input_rows_count, from_string_tag);
|
result_column = ConvertImpl<LeftDataType, RightDataType, Name>::execute(arguments, result_type, input_rows_count, from_string_tag);
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
};
|
};
|
||||||
@ -2334,6 +2384,11 @@ private:
|
|||||||
else
|
else
|
||||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag);
|
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if constexpr (std::is_same_v<ToDataType, DataTypeInterval>)
|
||||||
|
{
|
||||||
|
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!done)
|
if (!done)
|
||||||
@ -5224,7 +5279,7 @@ REGISTER_FUNCTION(Conversion)
|
|||||||
/// MySQL compatibility alias. Cannot be registered as alias,
|
/// MySQL compatibility alias. Cannot be registered as alias,
|
||||||
/// because we don't want it to be normalized to toDate in queries,
|
/// because we don't want it to be normalized to toDate in queries,
|
||||||
/// otherwise CREATE DICTIONARY query breaks.
|
/// otherwise CREATE DICTIONARY query breaks.
|
||||||
factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive);
|
factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::CaseInsensitive);
|
||||||
|
|
||||||
factory.registerFunction<FunctionToDate32>();
|
factory.registerFunction<FunctionToDate32>();
|
||||||
factory.registerFunction<FunctionToDateTime>();
|
factory.registerFunction<FunctionToDateTime>();
|
||||||
|
@ -0,0 +1,99 @@
|
|||||||
|
Comparing nanoseconds
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing microseconds
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing milliseconds
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing seconds
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing minutes
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing hours
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing days
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing weeks
|
||||||
|
1
|
||||||
|
0
|
||||||
|
Comparing months
|
||||||
|
1
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing quarters
|
||||||
|
1
|
||||||
|
1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Comparing years
|
||||||
|
1
|
||||||
|
0
|
@ -0,0 +1,142 @@
|
|||||||
|
SELECT('Comparing nanoseconds');
|
||||||
|
SELECT toIntervalNanosecond(500) > toIntervalNanosecond(300);
|
||||||
|
SELECT toIntervalNanosecond(1000) < toIntervalNanosecond(1500);
|
||||||
|
SELECT toIntervalNanosecond(2000) = toIntervalNanosecond(2000);
|
||||||
|
SELECT toIntervalNanosecond(1000) >= toIntervalMicrosecond(1);
|
||||||
|
SELECT toIntervalNanosecond(1000001) > toIntervalMillisecond(1);
|
||||||
|
SELECT toIntervalNanosecond(2000000001) > toIntervalSecond(2);
|
||||||
|
SELECT toIntervalNanosecond(60000000000) = toIntervalMinute(1);
|
||||||
|
SELECT toIntervalNanosecond(7199999999999) < toIntervalHour(2);
|
||||||
|
SELECT toIntervalNanosecond(1) < toIntervalDay(2);
|
||||||
|
SELECT toIntervalNanosecond(5) < toIntervalWeek(1);
|
||||||
|
|
||||||
|
SELECT toIntervalNanosecond(500) < toIntervalNanosecond(300);
|
||||||
|
SELECT toIntervalNanosecond(1000) > toIntervalNanosecond(1500);
|
||||||
|
SELECT toIntervalNanosecond(2000) != toIntervalNanosecond(2000);
|
||||||
|
SELECT toIntervalNanosecond(1000) < toIntervalMicrosecond(1);
|
||||||
|
SELECT toIntervalNanosecond(1000001) < toIntervalMillisecond(1);
|
||||||
|
SELECT toIntervalNanosecond(2000000001) < toIntervalSecond(2);
|
||||||
|
SELECT toIntervalNanosecond(60000000000) != toIntervalMinute(1);
|
||||||
|
SELECT toIntervalNanosecond(7199999999999) > toIntervalHour(2);
|
||||||
|
SELECT toIntervalNanosecond(1) > toIntervalDay(2);
|
||||||
|
SELECT toIntervalNanosecond(5) > toIntervalWeek(1);
|
||||||
|
|
||||||
|
SELECT toIntervalNanosecond(1) < toIntervalMonth(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing microseconds');
|
||||||
|
SELECT toIntervalMicrosecond(1) < toIntervalMicrosecond(999);
|
||||||
|
SELECT toIntervalMicrosecond(1001) > toIntervalMillisecond(1);
|
||||||
|
SELECT toIntervalMicrosecond(2000000) = toIntervalSecond(2);
|
||||||
|
SELECT toIntervalMicrosecond(179999999) < toIntervalMinute(3);
|
||||||
|
SELECT toIntervalMicrosecond(3600000000) = toIntervalHour(1);
|
||||||
|
SELECT toIntervalMicrosecond(36000000000000) > toIntervalDay(2);
|
||||||
|
SELECT toIntervalMicrosecond(1209600000000) = toIntervalWeek(2);
|
||||||
|
|
||||||
|
SELECT toIntervalMicrosecond(1) > toIntervalMicrosecond(999);
|
||||||
|
SELECT toIntervalMicrosecond(1001) < toIntervalMillisecond(1);
|
||||||
|
SELECT toIntervalMicrosecond(2000000) != toIntervalSecond(2);
|
||||||
|
SELECT toIntervalMicrosecond(179999999) > toIntervalMinute(3);
|
||||||
|
SELECT toIntervalMicrosecond(3600000000) != toIntervalHour(1);
|
||||||
|
SELECT toIntervalMicrosecond(36000000000000) < toIntervalDay(2);
|
||||||
|
SELECT toIntervalMicrosecond(1209600000000) != toIntervalWeek(2);
|
||||||
|
|
||||||
|
SELECT toIntervalMicrosecond(36000000000000) < toIntervalQuarter(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing milliseconds');
|
||||||
|
SELECT toIntervalMillisecond(2000) > toIntervalMillisecond(2);
|
||||||
|
SELECT toIntervalMillisecond(2000) = toIntervalSecond(2);
|
||||||
|
SELECT toIntervalMillisecond(170000) < toIntervalMinute(3);
|
||||||
|
SELECT toIntervalMillisecond(144000001) > toIntervalHour(40);
|
||||||
|
SELECT toIntervalMillisecond(1728000000) = toIntervalDay(20);
|
||||||
|
SELECT toIntervalMillisecond(1198599999) < toIntervalWeek(2);
|
||||||
|
|
||||||
|
SELECT toIntervalMillisecond(2000) < toIntervalMillisecond(2);
|
||||||
|
SELECT toIntervalMillisecond(2000) != toIntervalSecond(2);
|
||||||
|
SELECT toIntervalMillisecond(170000) > toIntervalMinute(3);
|
||||||
|
SELECT toIntervalMillisecond(144000001) < toIntervalHour(40);
|
||||||
|
SELECT toIntervalMillisecond(1728000000) != toIntervalDay(20);
|
||||||
|
SELECT toIntervalMillisecond(1198599999) > toIntervalWeek(2);
|
||||||
|
|
||||||
|
SELECT toIntervalMillisecond(36000000000000) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing seconds');
|
||||||
|
SELECT toIntervalSecond(120) > toIntervalSecond(2);
|
||||||
|
SELECT toIntervalSecond(120) = toIntervalMinute(2);
|
||||||
|
SELECT toIntervalSecond(1) < toIntervalHour(2);
|
||||||
|
SELECT toIntervalSecond(86401) >= toIntervalDay(1);
|
||||||
|
SELECT toIntervalSecond(1209600) = toIntervalWeek(2);
|
||||||
|
|
||||||
|
SELECT toIntervalSecond(120) < toIntervalSecond(2);
|
||||||
|
SELECT toIntervalSecond(120) != toIntervalMinute(2);
|
||||||
|
SELECT toIntervalSecond(1) > toIntervalHour(2);
|
||||||
|
SELECT toIntervalSecond(86401) < toIntervalDay(1);
|
||||||
|
SELECT toIntervalSecond(1209600) != toIntervalWeek(2);
|
||||||
|
|
||||||
|
SELECT toIntervalSecond(36000000000000) < toIntervalMonth(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing minutes');
|
||||||
|
SELECT toIntervalMinute(1) < toIntervalMinute(59);
|
||||||
|
SELECT toIntervalMinute(1) < toIntervalHour(59);
|
||||||
|
SELECT toIntervalMinute(1440) = toIntervalDay(1);
|
||||||
|
SELECT toIntervalMinute(30241) > toIntervalWeek(3);
|
||||||
|
|
||||||
|
SELECT toIntervalMinute(1) > toIntervalMinute(59);
|
||||||
|
SELECT toIntervalMinute(1) > toIntervalHour(59);
|
||||||
|
SELECT toIntervalMinute(1440) != toIntervalDay(1);
|
||||||
|
SELECT toIntervalMinute(30241) < toIntervalWeek(3);
|
||||||
|
|
||||||
|
SELECT toIntervalMinute(2) = toIntervalQuarter(120); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing hours');
|
||||||
|
SELECT toIntervalHour(48) > toIntervalHour(2);
|
||||||
|
SELECT toIntervalHour(48) >= toIntervalDay(2);
|
||||||
|
SELECT toIntervalHour(672) = toIntervalWeek(4);
|
||||||
|
|
||||||
|
SELECT toIntervalHour(48) < toIntervalHour(2);
|
||||||
|
SELECT toIntervalHour(48) < toIntervalDay(2);
|
||||||
|
SELECT toIntervalHour(672) != toIntervalWeek(4);
|
||||||
|
|
||||||
|
SELECT toIntervalHour(2) < toIntervalYear(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing days');
|
||||||
|
SELECT toIntervalDay(1) < toIntervalDay(23);
|
||||||
|
SELECT toIntervalDay(25) > toIntervalWeek(3);
|
||||||
|
|
||||||
|
SELECT toIntervalDay(1) > toIntervalDay(23);
|
||||||
|
SELECT toIntervalDay(25) < toIntervalWeek(3);
|
||||||
|
|
||||||
|
SELECT toIntervalDay(2) = toIntervalMonth(48); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing weeks');
|
||||||
|
SELECT toIntervalWeek(1) < toIntervalWeek(6);
|
||||||
|
|
||||||
|
SELECT toIntervalWeek(1) > toIntervalWeek(6);
|
||||||
|
|
||||||
|
SELECT toIntervalWeek(124) > toIntervalQuarter(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing months');
|
||||||
|
SELECT toIntervalMonth(1) < toIntervalMonth(3);
|
||||||
|
SELECT toIntervalMonth(124) > toIntervalQuarter(5);
|
||||||
|
SELECT toIntervalMonth(36) = toIntervalYear(3);
|
||||||
|
|
||||||
|
SELECT toIntervalMonth(1) > toIntervalMonth(3);
|
||||||
|
SELECT toIntervalMonth(124) < toIntervalQuarter(5);
|
||||||
|
SELECT toIntervalMonth(36) != toIntervalYear(3);
|
||||||
|
|
||||||
|
SELECT toIntervalMonth(6) = toIntervalMicrosecond(26); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing quarters');
|
||||||
|
SELECT toIntervalQuarter(5) > toIntervalQuarter(4);
|
||||||
|
SELECT toIntervalQuarter(20) = toIntervalYear(5);
|
||||||
|
|
||||||
|
SELECT toIntervalQuarter(5) < toIntervalQuarter(4);
|
||||||
|
SELECT toIntervalQuarter(20) != toIntervalYear(5);
|
||||||
|
|
||||||
|
SELECT toIntervalQuarter(2) = toIntervalNanosecond(6); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||||
|
|
||||||
|
SELECT('Comparing years');
|
||||||
|
SELECT toIntervalYear(1) < toIntervalYear(3);
|
||||||
|
|
||||||
|
SELECT toIntervalYear(1) > toIntervalYear(3);
|
||||||
|
|
||||||
|
SELECT toIntervalYear(2) = toIntervalSecond(8); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
Loading…
Reference in New Issue
Block a user