mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-18 21:51:57 +00:00
Merge remote-tracking branch 'origin/master' into ADQM-974
This commit is contained in:
commit
fdcf434b43
@ -718,7 +718,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d
|
||||
|
||||
## age
|
||||
|
||||
Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 second.
|
||||
Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond.
|
||||
E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit.
|
||||
|
||||
For an alternative to `age`, see function `date\_diff`.
|
||||
@ -734,6 +734,8 @@ age('unit', startdate, enddate, [timezone])
|
||||
- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md).
|
||||
Possible values:
|
||||
|
||||
- `microsecond` (possible abbreviations: `us`, `u`)
|
||||
- `millisecond` (possible abbreviations: `ms`)
|
||||
- `second` (possible abbreviations: `ss`, `s`)
|
||||
- `minute` (possible abbreviations: `mi`, `n`)
|
||||
- `hour` (possible abbreviations: `hh`, `h`)
|
||||
@ -809,6 +811,8 @@ Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_
|
||||
- `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md).
|
||||
Possible values:
|
||||
|
||||
- `microsecond` (possible abbreviations: `us`, `u`)
|
||||
- `millisecond` (possible abbreviations: `ms`)
|
||||
- `second` (possible abbreviations: `ss`, `s`)
|
||||
- `minute` (possible abbreviations: `mi`, `n`)
|
||||
- `hour` (possible abbreviations: `hh`, `h`)
|
||||
|
@ -621,7 +621,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d
|
||||
|
||||
## age
|
||||
|
||||
Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 секунду.
|
||||
Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 микросекунду.
|
||||
Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`.
|
||||
|
||||
**Синтаксис**
|
||||
@ -635,6 +635,8 @@ age('unit', startdate, enddate, [timezone])
|
||||
- `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md).
|
||||
Возможные значения:
|
||||
|
||||
- `microsecond` (возможные сокращения: `us`, `u`)
|
||||
- `millisecond` (возможные сокращения: `ms`)
|
||||
- `second` (возможные сокращения: `ss`, `s`)
|
||||
- `minute` (возможные сокращения: `mi`, `n`)
|
||||
- `hour` (возможные сокращения: `hh`, `h`)
|
||||
@ -708,6 +710,8 @@ date_diff('unit', startdate, enddate, [timezone])
|
||||
- `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md).
|
||||
Возможные значения:
|
||||
|
||||
- `microsecond` (возможные сокращения: `us`, `u`)
|
||||
- `millisecond` (возможные сокращения: `ms`)
|
||||
- `second` (возможные сокращения: `ss`, `s`)
|
||||
- `minute` (возможные сокращения: `mi`, `n`)
|
||||
- `hour` (возможные сокращения: `hh`, `h`)
|
||||
|
@ -643,6 +643,8 @@ date_diff('unit', startdate, enddate, [timezone])
|
||||
- `unit` — `value`对应的时间单位。类型为[String](../../sql-reference/data-types/string.md)。
|
||||
可能的值:
|
||||
|
||||
- `microsecond`
|
||||
- `millisecond`
|
||||
- `second`
|
||||
- `minute`
|
||||
- `hour`
|
||||
|
@ -48,7 +48,11 @@ inline auto scaleMultiplier(UInt32 scale)
|
||||
|
||||
/** Components of DecimalX value:
|
||||
* whole - represents whole part of decimal, can be negative or positive.
|
||||
* fractional - for fractional part of decimal, always positive.
|
||||
* fractional - for fractional part of decimal.
|
||||
*
|
||||
* 0.123 represents 0 / 0.123
|
||||
* -0.123 represents 0 / -0.123
|
||||
* -1.123 represents -1 / 0.123
|
||||
*/
|
||||
template <typename DecimalType>
|
||||
struct DecimalComponents
|
||||
|
@ -275,7 +275,7 @@ class IColumn;
|
||||
\
|
||||
M(UInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \
|
||||
M(Bool, http_wait_end_of_query, false, "Enable HTTP response buffering on the server-side.", 0) \
|
||||
M(UInt64, http_response_buffer_size, false, "The number of bytes to buffer in the server memory before sending a HTTP response to the client or flushing to disk (when http_wait_end_of_query is enabled).", 0) \
|
||||
M(UInt64, http_response_buffer_size, 0, "The number of bytes to buffer in the server memory before sending a HTTP response to the client or flushing to disk (when http_wait_end_of_query is enabled).", 0) \
|
||||
\
|
||||
M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
|
||||
\
|
||||
|
@ -19,6 +19,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static constexpr auto microsecond_multiplier = 1000000;
|
||||
static constexpr auto millisecond_multiplier = 1000;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
@ -1377,6 +1380,36 @@ struct ToRelativeSecondNumImpl
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
template <Int64 scale_multiplier>
|
||||
struct ToRelativeSubsecondNumImpl
|
||||
{
|
||||
static constexpr auto name = "toRelativeSubsecondNumImpl";
|
||||
|
||||
static inline Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &)
|
||||
{
|
||||
static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000);
|
||||
if (scale == scale_multiplier)
|
||||
return t.value;
|
||||
if (scale > scale_multiplier)
|
||||
return t.value / (scale / scale_multiplier);
|
||||
return t.value * (scale_multiplier / scale);
|
||||
}
|
||||
static inline Int64 execute(UInt32 t, const DateLUTImpl &)
|
||||
{
|
||||
return t * scale_multiplier;
|
||||
}
|
||||
static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return static_cast<Int64>(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier;
|
||||
}
|
||||
static inline Int64 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return static_cast<Int64>(time_zone.fromDayNum(DayNum(d)) * scale_multiplier);
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToYYYYMMImpl
|
||||
{
|
||||
static constexpr auto name = "toYYYYMM";
|
||||
@ -1476,25 +1509,47 @@ struct ToYYYYMMDDhhmmssImpl
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct DateTimeComponentsWithFractionalPart : public DateLUTImpl::DateTimeComponents
|
||||
{
|
||||
UInt16 millisecond;
|
||||
UInt16 microsecond;
|
||||
};
|
||||
|
||||
struct ToDateTimeComponentsImpl
|
||||
{
|
||||
static constexpr auto name = "toDateTimeComponents";
|
||||
|
||||
static inline DateLUTImpl::DateTimeComponents execute(Int64 t, const DateLUTImpl & time_zone)
|
||||
static inline DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDateTimeComponents(t);
|
||||
auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
|
||||
|
||||
if (t.value < 0 && components.fractional)
|
||||
{
|
||||
components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional;
|
||||
--components.whole;
|
||||
}
|
||||
Int64 fractional = components.fractional;
|
||||
if (scale_multiplier > microsecond_multiplier)
|
||||
fractional = fractional / (scale_multiplier / microsecond_multiplier);
|
||||
else if (scale_multiplier < microsecond_multiplier)
|
||||
fractional = fractional * (microsecond_multiplier / scale_multiplier);
|
||||
|
||||
constexpr Int64 divider = microsecond_multiplier/ millisecond_multiplier;
|
||||
UInt16 millisecond = static_cast<UInt16>(fractional / divider);
|
||||
UInt16 microsecond = static_cast<UInt16>(fractional % divider);
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond};
|
||||
}
|
||||
static inline DateLUTImpl::DateTimeComponents execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
static inline DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDateTimeComponents(static_cast<DateLUTImpl::Time>(t));
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast<DateLUTImpl::Time>(t)), 0, 0};
|
||||
}
|
||||
static inline DateLUTImpl::DateTimeComponents execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
static inline DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDateTimeComponents(ExtendedDayNum(d));
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0};
|
||||
}
|
||||
static inline DateLUTImpl::DateTimeComponents execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
static inline DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDateTimeComponents(DayNum(d));
|
||||
return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0};
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
|
@ -5,7 +5,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Tansform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform.
|
||||
/** Transform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform.
|
||||
*
|
||||
* Depending on what overloads of Transform::execute() are available, when called with DateTime64 value,
|
||||
* invokes Transform::execute() with either:
|
||||
@ -80,7 +80,10 @@ public:
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
|
||||
auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
|
||||
if (t.value < 0 && components.fractional)
|
||||
--components.whole;
|
||||
|
||||
return wrapped_transform.execute(static_cast<Int64>(components.whole), std::forward<Args>(args)...);
|
||||
}
|
||||
}
|
||||
|
@ -174,12 +174,13 @@ public:
|
||||
{
|
||||
auto res = static_cast<Int64>(transform_y.execute(y, timezone_y))
|
||||
- static_cast<Int64>(transform_x.execute(x, timezone_x));
|
||||
DateLUTImpl::DateTimeComponents a_comp;
|
||||
DateLUTImpl::DateTimeComponents b_comp;
|
||||
DateTimeComponentsWithFractionalPart a_comp;
|
||||
DateTimeComponentsWithFractionalPart b_comp;
|
||||
Int64 adjust_value;
|
||||
auto x_seconds = TransformDateTime64<ToRelativeSecondNumImpl<ResultPrecision::Extended>>(transform_x.getScaleMultiplier()).execute(x, timezone_x);
|
||||
auto y_seconds = TransformDateTime64<ToRelativeSecondNumImpl<ResultPrecision::Extended>>(transform_y.getScaleMultiplier()).execute(y, timezone_y);
|
||||
if (x_seconds <= y_seconds)
|
||||
auto x_microseconds = TransformDateTime64<ToRelativeSubsecondNumImpl<microsecond_multiplier>>(transform_x.getScaleMultiplier()).execute(x, timezone_x);
|
||||
auto y_microseconds = TransformDateTime64<ToRelativeSubsecondNumImpl<microsecond_multiplier>>(transform_y.getScaleMultiplier()).execute(y, timezone_y);
|
||||
|
||||
if (x_microseconds <= y_microseconds)
|
||||
{
|
||||
a_comp = TransformDateTime64<ToDateTimeComponentsImpl>(transform_x.getScaleMultiplier()).execute(x, timezone_x);
|
||||
b_comp = TransformDateTime64<ToDateTimeComponentsImpl>(transform_y.getScaleMultiplier()).execute(y, timezone_y);
|
||||
@ -192,14 +193,16 @@ public:
|
||||
adjust_value = 1;
|
||||
}
|
||||
|
||||
|
||||
if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeYearNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if ((a_comp.date.month > b_comp.date.month)
|
||||
|| ((a_comp.date.month == b_comp.date.month) && ((a_comp.date.day > b_comp.date.day)
|
||||
|| ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour)
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))
|
||||
)))))
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeQuarterNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -210,8 +213,9 @@ public:
|
||||
|| ((x_month_in_quarter == y_month_in_quarter) && ((a_comp.date.day > b_comp.date.day)
|
||||
|| ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour)
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))
|
||||
)))))
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeMonthNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -219,8 +223,9 @@ public:
|
||||
if ((a_comp.date.day > b_comp.date.day)
|
||||
|| ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour)
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))
|
||||
)))
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeWeekNumImpl<ResultPrecision::Extended>>>)
|
||||
@ -230,25 +235,44 @@ public:
|
||||
if ((x_day_of_week > y_day_of_week)
|
||||
|| ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour))
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))))
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeDayNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if ((a_comp.time.hour > b_comp.time.hour)
|
||||
|| ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))))
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeHourNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if ((a_comp.time.minute > b_comp.time.minute)
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))
|
||||
|| ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeMinuteNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if (a_comp.time.second > b_comp.time.second)
|
||||
if ((a_comp.time.second > b_comp.time.second)
|
||||
|| ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeSecondNumImpl<ResultPrecision::Extended>>>)
|
||||
{
|
||||
if ((a_comp.millisecond > b_comp.millisecond)
|
||||
|| ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond)))
|
||||
res += adjust_value;
|
||||
}
|
||||
else if constexpr (std::is_same_v<TransformX, TransformDateTime64<ToRelativeSubsecondNumImpl<1000>>>)
|
||||
{
|
||||
if (a_comp.microsecond > b_comp.microsecond)
|
||||
res += adjust_value;
|
||||
}
|
||||
return res;
|
||||
@ -373,6 +397,10 @@ public:
|
||||
impl.template dispatchForColumns<ToRelativeMinuteNumImpl<ResultPrecision::Extended>>(x, y, timezone_x, timezone_y, res->getData());
|
||||
else if (unit == "second" || unit == "ss" || unit == "s")
|
||||
impl.template dispatchForColumns<ToRelativeSecondNumImpl<ResultPrecision::Extended>>(x, y, timezone_x, timezone_y, res->getData());
|
||||
else if (unit == "millisecond" || unit == "ms")
|
||||
impl.template dispatchForColumns<ToRelativeSubsecondNumImpl<millisecond_multiplier>>(x, y, timezone_x, timezone_y, res->getData());
|
||||
else if (unit == "microsecond" || unit == "us" || unit == "u")
|
||||
impl.template dispatchForColumns<ToRelativeSubsecondNumImpl<microsecond_multiplier>>(x, y, timezone_x, timezone_y, res->getData());
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Function {} does not support '{}' unit", getName(), unit);
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/TransformDateTime64.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
|
@ -904,10 +904,9 @@ try
|
||||
/// Destroy CascadeBuffer to actualize buffers' positions and reset extra references
|
||||
if (used_output.hasDelayed())
|
||||
{
|
||||
if (used_output.out_maybe_delayed_and_compressed)
|
||||
{
|
||||
used_output.out_maybe_delayed_and_compressed->finalize();
|
||||
}
|
||||
/// do not call finalize here for CascadeWriteBuffer used_output.out_maybe_delayed_and_compressed,
|
||||
/// exception is written into used_output.out_maybe_compressed later
|
||||
/// HTTPHandler::trySendExceptionToClient is called with exception context, it is Ok to destroy buffers
|
||||
used_output.out_maybe_delayed_and_compressed.reset();
|
||||
}
|
||||
|
||||
|
@ -7,13 +7,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
format="RowBinary"
|
||||
|
||||
function query {
|
||||
# bash isn't able to store \0 bytes, so use [1; 255] random range
|
||||
echo "SELECT greatest(toUInt8(1), toUInt8(intHash64(number))) FROM system.numbers LIMIT $1 FORMAT RowBinary"
|
||||
echo "SELECT greatest(toUInt8(1), toUInt8(intHash64(number))) FROM system.numbers LIMIT $1 FORMAT $format"
|
||||
}
|
||||
|
||||
function ch_url() {
|
||||
${CLICKHOUSE_CURL_COMMAND} -q -sS "${CLICKHOUSE_URL}&max_block_size=$max_block_size&$1" -d "$(query "$2")"
|
||||
${CLICKHOUSE_CURL_COMMAND} -q -sS \
|
||||
"${CLICKHOUSE_URL}${max_block_size:+"&max_block_size=$max_block_size"}&$1" \
|
||||
-d "$(query "$2")"
|
||||
}
|
||||
|
||||
|
||||
@ -24,9 +28,9 @@ exception_pattern="DB::Exception:[[:print:]]*"
|
||||
function check_only_exception() {
|
||||
local res
|
||||
res=$(ch_url "$1" "$2")
|
||||
#(echo "$res")
|
||||
#(echo "$res" | wc -l)
|
||||
#(echo "$res" | grep -c "$exception_pattern")
|
||||
# echo "$res"
|
||||
# echo "$res" | wc -l
|
||||
# echo "$res" | grep -c "$exception_pattern"
|
||||
[[ $(echo "$res" | wc -l) -eq 1 ]] || echo FAIL 1 "$@"
|
||||
[[ $(echo "$res" | grep -c "$exception_pattern") -eq 1 ]] || echo FAIL 2 "$@"
|
||||
}
|
||||
@ -34,19 +38,23 @@ function check_only_exception() {
|
||||
function check_last_line_exception() {
|
||||
local res
|
||||
res=$(ch_url "$1" "$2")
|
||||
#echo "$res" > res
|
||||
#echo "$res" | wc -c
|
||||
#echo "$res" | tail -n -2
|
||||
# echo "$res" > res
|
||||
# echo "$res" | wc -c
|
||||
# echo "$res" | tail -n -2
|
||||
[[ $(echo "$res" | tail -n -1 | grep -c "$exception_pattern") -eq 1 ]] || echo FAIL 3 "$@"
|
||||
[[ $(echo "$res" | head -n -1 | grep -c "$exception_pattern") -eq 0 ]] || echo FAIL 4 "$@"
|
||||
}
|
||||
|
||||
function check_exception_handling() {
|
||||
format=TSV \
|
||||
check_last_line_exception \
|
||||
"max_block_size=30000&max_result_rows=400000&buffer_size=1048577&wait_end_of_query=0" 111222333444
|
||||
|
||||
check_only_exception "max_result_bytes=1000" 1001
|
||||
check_only_exception "max_result_bytes=1000&wait_end_of_query=1" 1001
|
||||
|
||||
check_only_exception "max_result_bytes=1048576&buffer_size=1048576&wait_end_of_query=0" 1048577
|
||||
check_only_exception "max_result_bytes=1048576&buffer_size=1048576&wait_end_of_query=1" 1048577
|
||||
check_last_line_exception "max_result_bytes=1048576&buffer_size=1048576&wait_end_of_query=0" 1048577
|
||||
check_only_exception "max_result_bytes=1048576&buffer_size=1048576&wait_end_of_query=1" 1048577
|
||||
|
||||
check_only_exception "max_result_bytes=1500000&buffer_size=2500000&wait_end_of_query=0" 1500001
|
||||
check_only_exception "max_result_bytes=1500000&buffer_size=1500000&wait_end_of_query=1" 1500001
|
||||
|
@ -4,3 +4,4 @@
|
||||
201707
|
||||
20170721
|
||||
20170721112233
|
||||
19691231235959
|
||||
|
@ -4,3 +4,4 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21'));
|
||||
SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33'));
|
||||
SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33'));
|
||||
SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33'));
|
||||
SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3));
|
||||
|
@ -33,4 +33,14 @@ Hello
|
||||
2021-01-01
|
||||
1
|
||||
1
|
||||
86400000
|
||||
172800000
|
||||
86461000
|
||||
86401299
|
||||
701
|
||||
701
|
||||
800
|
||||
60200201
|
||||
60
|
||||
10
|
||||
1
|
||||
|
@ -41,4 +41,16 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR);
|
||||
SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01');
|
||||
SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01');
|
||||
|
||||
SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date);
|
||||
SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32);
|
||||
SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime);
|
||||
SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64);
|
||||
SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date);
|
||||
SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date);
|
||||
SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date);
|
||||
SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9));
|
||||
|
||||
SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6));
|
||||
SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6));
|
||||
|
||||
SELECT EXISTS (SELECT 1);
|
||||
|
@ -22,18 +22,27 @@ $CLICKHOUSE_CLIENT -nm -q "
|
||||
|
||||
function delete_user()
|
||||
{
|
||||
$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02242" ||:
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02242" ||:
|
||||
sleep 0.$RANDOM;
|
||||
done
|
||||
}
|
||||
|
||||
function create_and_login_user()
|
||||
{
|
||||
$CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02242" ||:
|
||||
$CLICKHOUSE_CLIENT -u "test_user_02242" -q "SELECT COUNT(*) FROM system.session_log WHERE user == 'test_user_02242'" > /dev/null ||:
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02242" ||:
|
||||
$CLICKHOUSE_CLIENT -u "test_user_02242" -q "SELECT COUNT(*) FROM system.session_log WHERE user == 'test_user_02242'" > /dev/null ||:
|
||||
sleep 0.$RANDOM;
|
||||
done
|
||||
}
|
||||
|
||||
function set_role()
|
||||
{
|
||||
$CLICKHOUSE_CLIENT -q "SET ROLE test_role_02242 TO test_user_02242" ||:
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -q "SET DEFAULT ROLE test_role_02242 TO test_user_02242" ||:
|
||||
sleep 0.$RANDOM;
|
||||
done
|
||||
}
|
||||
|
||||
export -f delete_user
|
||||
@ -42,12 +51,10 @@ export -f set_role
|
||||
|
||||
TIMEOUT=10
|
||||
|
||||
for (( i = 0 ; i < 100; ++i ))
|
||||
do
|
||||
clickhouse_client_loop_timeout $TIMEOUT create_and_login_user 2> /dev/null &
|
||||
clickhouse_client_loop_timeout $TIMEOUT delete_user 2> /dev/null &
|
||||
clickhouse_client_loop_timeout $TIMEOUT set_role 2> /dev/null &
|
||||
done
|
||||
|
||||
timeout $TIMEOUT bash -c create_and_login_user 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c delete_user 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c set_role 2> /dev/null &
|
||||
|
||||
wait
|
||||
|
||||
|
@ -19,17 +19,18 @@ $CLICKHOUSE_CLIENT -nm -q "
|
||||
|
||||
function create_drop_grant()
|
||||
{
|
||||
$CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02243 GRANTEES NONE" ||:
|
||||
$CLICKHOUSE_CLIENT -q "GRANT ALL ON *.* TO test_user_02243 WITH GRANT OPTION" ||:
|
||||
$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02243" &
|
||||
$CLICKHOUSE_CLIENT --user test_user_02243 -q "GRANT ALL ON *.* TO kek_02243" &
|
||||
wait
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02243 GRANTEES NONE" ||:
|
||||
$CLICKHOUSE_CLIENT -q "GRANT ALL ON *.* TO test_user_02243 WITH GRANT OPTION" ||:
|
||||
$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02243" &
|
||||
$CLICKHOUSE_CLIENT --user test_user_02243 -q "GRANT ALL ON *.* TO kek_02243" &
|
||||
done
|
||||
}
|
||||
|
||||
export -f create_drop_grant
|
||||
|
||||
TIMEOUT=10
|
||||
clickhouse_client_loop_timeout $TIMEOUT create_drop_grant 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c create_drop_grant 2> /dev/null &
|
||||
wait
|
||||
|
||||
$CLICKHOUSE_CLIENT --user kek_02243 -q "SELECT * FROM test" 2>&1| grep -Fa "Exception: " | grep -Eo ACCESS_DENIED | uniq
|
||||
|
@ -111,3 +111,109 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma
|
||||
1
|
||||
SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC')));
|
||||
1
|
||||
-- DateTime64 vs DateTime64 with fractional part
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC'));
|
||||
5100200
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC'));
|
||||
5100200
|
||||
SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC'));
|
||||
5100
|
||||
SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC'));
|
||||
5099
|
||||
SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC'));
|
||||
4
|
||||
SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC'));
|
||||
4
|
||||
SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC'));
|
||||
5
|
||||
SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC'));
|
||||
4
|
||||
SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC'));
|
||||
4
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
3
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC'));
|
||||
2
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC'));
|
||||
3
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC'));
|
||||
2
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC'));
|
||||
1
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC'));
|
||||
1
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC'));
|
||||
1
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC'));
|
||||
1
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC'));
|
||||
1
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC'));
|
||||
16
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC'));
|
||||
15
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC'));
|
||||
15
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC'));
|
||||
15
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC'));
|
||||
15
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC'));
|
||||
15
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC'));
|
||||
15
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC'));
|
||||
5
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC'));
|
||||
4
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC'));
|
||||
4
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC'));
|
||||
4
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC'));
|
||||
4
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC'));
|
||||
4
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC'));
|
||||
4
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC'));
|
||||
8
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC'));
|
||||
7
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC'));
|
||||
7
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC'));
|
||||
7
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC'));
|
||||
7
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC'));
|
||||
7
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC'));
|
||||
7
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC'));
|
||||
7
|
||||
-- DateTime64 vs DateTime64 with negative time
|
||||
SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC'));
|
||||
2349
|
||||
SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC'));
|
||||
2
|
||||
SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC'));
|
||||
5
|
||||
SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC'));
|
||||
4
|
||||
|
@ -75,3 +75,68 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')),
|
||||
SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC')));
|
||||
SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC')));
|
||||
SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC')));
|
||||
|
||||
-- DateTime64 vs DateTime64 with fractional part
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC'));
|
||||
SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC'));
|
||||
|
||||
SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC'));
|
||||
SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC'));
|
||||
|
||||
SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC'));
|
||||
SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC'));
|
||||
|
||||
SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC'));
|
||||
SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC'));
|
||||
SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC'));
|
||||
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC'));
|
||||
SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC'));
|
||||
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC'));
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC'));
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC'));
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC'));
|
||||
SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC'));
|
||||
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC'));
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC'));
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC'));
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC'));
|
||||
SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC'));
|
||||
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC'));
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC'));
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC'));
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC'));
|
||||
SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC'));
|
||||
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC'));
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC'));
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC'));
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC'));
|
||||
SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC'));
|
||||
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC'));
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC'));
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC'));
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC'));
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC'));
|
||||
SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC'));
|
||||
|
||||
-- DateTime64 vs DateTime64 with negative time
|
||||
SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC'));
|
||||
SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC'));
|
||||
SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC'));
|
||||
SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC'));
|
Loading…
Reference in New Issue
Block a user