Merge remote-tracking branch 'ClickHouse/master' into docs_partitionID

This commit is contained in:
Robert Schulze 2024-07-14 12:40:05 +00:00
commit 3c39c4f514
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
8 changed files with 1002 additions and 7 deletions

View File

@ -1030,7 +1030,7 @@ A table with no primary key represents the extreme case of a single equivalence
The fewer and the larger the equivalence classes are, the higher the degree of freedom when re-shuffling rows.
The heuristics applied to find the best row order within each equivalence class is suggested by D. Lemir, O. Kaser in [Reordering columns for smaller indexes](https://doi.org/10.1016/j.ins.2011.02.002) and based on sorting the rows within each equivalence class by ascending cardinality of the non-primary key columns.
The heuristics applied to find the best row order within each equivalence class is suggested by D. Lemire, O. Kaser in [Reordering columns for smaller indexes](https://doi.org/10.1016/j.ins.2011.02.002) and based on sorting the rows within each equivalence class by ascending cardinality of the non-primary key columns.
It performs three steps:
1. Find all equivalence classes based on the row values in primary key columns.
2. For each equivalence class, calculate (usually estimate) the cardinalities of the non-primary-key columns.

View File

@ -2698,6 +2698,204 @@ Like function `YYYYMMDDhhmmssToDate()` but produces a [DateTime64](../data-types
Accepts an additional, optional `precision` parameter after the `timezone` parameter.
## changeYear
Changes the year component of a date or date time.
**Syntax**
``` sql
changeYear(date_or_datetime, value)
```
**Arguments**
- `date_or_datetime` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a new value of the year. [Integer](../../sql-reference/data-types/int-uint.md).
**Returned value**
- The same type as `date_or_datetime`.
**Example**
``` sql
SELECT changeYear(toDate('1999-01-01'), 2000), changeYear(toDateTime64('1999-01-01 00:00:00.000', 3), 2000);
```
Result:
```
┌─changeYear(toDate('1999-01-01'), 2000)─┬─changeYear(toDateTime64('1999-01-01 00:00:00.000', 3), 2000)─┐
│ 2000-01-01 │ 2000-01-01 00:00:00.000 │
└────────────────────────────────────────┴──────────────────────────────────────────────────────────────┘
```
## changeMonth
Changes the month component of a date or date time.
**Syntax**
``` sql
changeMonth(date_or_datetime, value)
```
**Arguments**
- `date_or_datetime` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a new value of the month. [Integer](../../sql-reference/data-types/int-uint.md).
**Returned value**
- Returns a value of same type as `date_or_datetime`.
**Example**
``` sql
SELECT changeMonth(toDate('1999-01-01'), 2), changeMonth(toDateTime64('1999-01-01 00:00:00.000', 3), 2);
```
Result:
```
┌─changeMonth(toDate('1999-01-01'), 2)─┬─changeMonth(toDateTime64('1999-01-01 00:00:00.000', 3), 2)─┐
│ 1999-02-01 │ 1999-02-01 00:00:00.000 │
└──────────────────────────────────────┴────────────────────────────────────────────────────────────┘
```
## changeDay
Changes the day component of a date or date time.
**Syntax**
``` sql
changeDay(date_or_datetime, value)
```
**Arguments**
- `date_or_datetime` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a new value of the day. [Integer](../../sql-reference/data-types/int-uint.md).
**Returned value**
- Returns a value of same type as `date_or_datetime`.
**Example**
``` sql
SELECT changeDay(toDate('1999-01-01'), 5), changeDay(toDateTime64('1999-01-01 00:00:00.000', 3), 5);
```
Result:
```
┌─changeDay(toDate('1999-01-01'), 5)─┬─changeDay(toDateTime64('1999-01-01 00:00:00.000', 3), 5)─┐
│ 1999-01-05 │ 1999-01-05 00:00:00.000 │
└────────────────────────────────────┴──────────────────────────────────────────────────────────┘
```
## changeHour
Changes the hour component of a date or date time.
**Syntax**
``` sql
changeHour(date_or_datetime, value)
```
**Arguments**
- `date_or_datetime` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a new value of the hour. [Integer](../../sql-reference/data-types/int-uint.md).
**Returned value**
- Returns a value of same type as `date_or_datetime`. If the input is a [Date](../data-types/date.md), return [DateTime](../data-types/datetime.md). If the input is a [Date32](../data-types/date32.md), return [DateTime64](../data-types/datetime64.md).
**Example**
``` sql
SELECT changeHour(toDate('1999-01-01'), 14), changeHour(toDateTime64('1999-01-01 00:00:00.000', 3), 14);
```
Result:
```
┌─changeHour(toDate('1999-01-01'), 14)─┬─changeHour(toDateTime64('1999-01-01 00:00:00.000', 3), 14)─┐
│ 1999-01-01 14:00:00 │ 1999-01-01 14:00:00.000 │
└──────────────────────────────────────┴────────────────────────────────────────────────────────────┘
```
## changeMinute
Changes the minute component of a date or date time.
**Syntax**
``` sql
changeMinute(date_or_datetime, value)
```
**Arguments**
- `date_or_datetime` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a new value of the minute. [Integer](../../sql-reference/data-types/int-uint.md).
**Returned value**
- Returns a value of same type as `date_or_datetime`. If the input is a [Date](../data-types/date.md), return [DateTime](../data-types/datetime.md). If the input is a [Date32](../data-types/date32.md), return [DateTime64](../data-types/datetime64.md).
**Example**
``` sql
SELECT changeMinute(toDate('1999-01-01'), 15), changeMinute(toDateTime64('1999-01-01 00:00:00.000', 3), 15);
```
Result:
```
┌─changeMinute(toDate('1999-01-01'), 15)─┬─changeMinute(toDateTime64('1999-01-01 00:00:00.000', 3), 15)─┐
│ 1999-01-01 00:15:00 │ 1999-01-01 00:15:00.000 │
└────────────────────────────────────────┴──────────────────────────────────────────────────────────────┘
```
## changeSecond
Changes the second component of a date or date time.
**Syntax**
``` sql
changeSecond(date_or_datetime, value)
```
**Arguments**
- `date_or_datetime` - a [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md)
- `value` - a new value of the second. [Integer](../../sql-reference/data-types/int-uint.md).
**Returned value**
- Returns a value of same type as `date_or_datetime`. If the input is a [Date](../data-types/date.md), return [DateTime](../data-types/datetime.md). If the input is a [Date32](../data-types/date32.md), return [DateTime64](../data-types/datetime64.md).
**Example**
``` sql
SELECT changeSecond(toDate('1999-01-01'), 15), changeSecond(toDateTime64('1999-01-01 00:00:00.000', 3), 15);
```
Result:
```
┌─changeSecond(toDate('1999-01-01'), 15)─┬─changeSecond(toDateTime64('1999-01-01 00:00:00.000', 3), 15)─┐
│ 1999-01-01 00:00:15 │ 1999-01-01 00:00:15.000 │
└────────────────────────────────────────┴──────────────────────────────────────────────────────────────┘
```
## addYears
Adds a specified number of years to a date, a date with time or a string-encoded date / date with time.
@ -2714,6 +2912,7 @@ addYears(date, num)
- `num`: Number of years to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` plus `num` years. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -2751,6 +2950,7 @@ addQuarters(date, num)
- `num`: Number of quarters to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` plus `num` quarters. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -2788,6 +2988,7 @@ addMonths(date, num)
- `num`: Number of months to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` plus `num` months. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -2825,6 +3026,7 @@ addWeeks(date, num)
- `num`: Number of weeks to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` plus `num` weeks. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -2862,6 +3064,7 @@ addDays(date, num)
- `num`: Number of days to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` plus `num` days. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -2899,6 +3102,7 @@ addHours(date, num)
- `num`: Number of hours to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
o
- Returns `date` plus `num` hours. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -2936,6 +3140,7 @@ addMinutes(date, num)
- `num`: Number of minutes to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` plus `num` minutes. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -2973,6 +3178,7 @@ addSeconds(date, num)
- `num`: Number of seconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` plus `num` seconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3010,6 +3216,7 @@ addMilliseconds(date_time, num)
- `num`: Number of milliseconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date_time` plus `num` milliseconds. [DateTime64](../data-types/datetime64.md).
**Example**
@ -3045,6 +3252,7 @@ addMicroseconds(date_time, num)
- `num`: Number of microseconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date_time` plus `num` microseconds. [DateTime64](../data-types/datetime64.md).
**Example**
@ -3080,6 +3288,7 @@ addNanoseconds(date_time, num)
- `num`: Number of nanoseconds to add. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date_time` plus `num` nanoseconds. [DateTime64](../data-types/datetime64.md).
**Example**
@ -3115,6 +3324,7 @@ addInterval(interval_1, interval_2)
- `interval_2`: Second interval to be added. [interval](../data-types/special-data-types/interval.md).
**Returned value**
- Returns a tuple of intervals. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)).
:::note
@ -3161,6 +3371,7 @@ addTupleOfIntervals(interval_1, interval_2)
- `intervals`: Tuple of intervals to add to `date`. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)).
**Returned value**
- Returns `date` with added `intervals`. [date](../data-types/date.md)/[date32](../data-types/date32.md)/[datetime](../data-types/datetime.md)/[datetime64](../data-types/datetime64.md).
**Example**
@ -3195,6 +3406,7 @@ subtractYears(date, num)
- `num`: Number of years to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` years. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3232,6 +3444,7 @@ subtractQuarters(date, num)
- `num`: Number of quarters to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` quarters. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3269,6 +3482,7 @@ subtractMonths(date, num)
- `num`: Number of months to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` months. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3306,6 +3520,7 @@ subtractWeeks(date, num)
- `num`: Number of weeks to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` weeks. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3343,6 +3558,7 @@ subtractDays(date, num)
- `num`: Number of days to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` days. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3380,6 +3596,7 @@ subtractHours(date, num)
- `num`: Number of hours to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` hours. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[Datetime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3417,6 +3634,7 @@ subtractMinutes(date, num)
- `num`: Number of minutes to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` minutes. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3454,6 +3672,7 @@ subtractSeconds(date, num)
- `num`: Number of seconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date` minus `num` seconds. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**
@ -3491,6 +3710,7 @@ subtractMilliseconds(date_time, num)
- `num`: Number of milliseconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date_time` minus `num` milliseconds. [DateTime64](../data-types/datetime64.md).
**Example**
@ -3526,6 +3746,7 @@ subtractMicroseconds(date_time, num)
- `num`: Number of microseconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date_time` minus `num` microseconds. [DateTime64](../data-types/datetime64.md).
**Example**
@ -3561,6 +3782,7 @@ subtractNanoseconds(date_time, num)
- `num`: Number of nanoseconds to subtract. [(U)Int*](../data-types/int-uint.md), [Float*](../data-types/float.md).
**Returned value**
- Returns `date_time` minus `num` nanoseconds. [DateTime64](../data-types/datetime64.md).
**Example**
@ -3596,6 +3818,7 @@ subtractInterval(interval_1, interval_2)
- `interval_2`: Second interval to be negated. [interval](../data-types/special-data-types/interval.md).
**Returned value**
- Returns a tuple of intervals. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)).
:::note
@ -3642,6 +3865,7 @@ subtractTupleOfIntervals(interval_1, interval_2)
- `intervals`: Tuple of intervals to subtract from `date`. [tuple](../data-types/tuple.md)([interval](../data-types/special-data-types/interval.md)).
**Returned value**
- Returns `date` with subtracted `intervals`. [Date](../data-types/date.md)/[Date32](../data-types/date32.md)/[DateTime](../data-types/datetime.md)/[DateTime64](../data-types/datetime64.md).
**Example**

View File

@ -0,0 +1,399 @@
#include "Common/DateLUTImpl.h"
#include "Common/Exception.h"
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnsDateTime.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/IColumn.h>
#include <Common/DateLUT.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/IDataType.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/castColumn.h>
#include <memory>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
enum class Component
{
Year,
Month,
Day,
Hour,
Minute,
Second
};
}
template <typename Traits>
class FunctionChangeDate : public IFunction
{
public:
static constexpr auto name = Traits::name;
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionChangeDate>(); }
String getName() const override { return Traits::name; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
size_t getNumberOfArguments() const override { return 2; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors args{
{"date_or_datetime", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isDateOrDate32OrDateTimeOrDateTime64), nullptr, "Date or date with time"},
{"value", static_cast<FunctionArgumentDescriptor::TypeValidator>(&isNativeInteger), nullptr, "Integer"}
};
validateFunctionArguments(*this, arguments, args);
const auto & input_type = arguments[0].type;
if constexpr (Traits::component == Component::Hour || Traits::component == Component::Minute || Traits::component == Component::Second)
{
if (isDate(input_type))
return std::make_shared<DataTypeDateTime>();
if (isDate32(input_type))
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale);
}
return input_type;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
const auto & input_type = arguments[0].type;
if (isDate(input_type))
{
if constexpr (Traits::component == Component::Hour || Traits::component == Component::Minute || Traits::component == Component::Second)
return execute<DataTypeDate, DataTypeDateTime>(arguments, input_type, result_type, input_rows_count);
return execute<DataTypeDate, DataTypeDate>(arguments, input_type, result_type, input_rows_count);
}
if (isDate32(input_type))
{
if constexpr (Traits::component == Component::Hour || Traits::component == Component::Minute || Traits::component == Component::Second)
return execute<DataTypeDate32, DataTypeDateTime64>(arguments, input_type, result_type, input_rows_count);
return execute<DataTypeDate32, DataTypeDate32>(arguments, input_type, result_type, input_rows_count);
}
if (isDateTime(input_type))
return execute<DataTypeDateTime, DataTypeDateTime>(arguments, input_type, result_type, input_rows_count);
if (isDateTime64(input_type))
return execute<DataTypeDateTime64, DataTypeDateTime64>(arguments, input_type, result_type, input_rows_count);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid input type");
}
template <typename InputDataType, typename ResultDataType>
ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & input_type, const DataTypePtr & result_type, size_t input_rows_count) const
{
typename ResultDataType::ColumnType::MutablePtr result_col;
if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime64>)
{
auto scale = DataTypeDateTime64::default_scale;
if constexpr (std::is_same_v<InputDataType, DateTime64>)
scale = typeid_cast<const DataTypeDateTime64 &>(*result_type).getScale();
result_col = ResultDataType::ColumnType::create(input_rows_count, scale);
}
else
result_col = ResultDataType::ColumnType::create(input_rows_count);
auto date_time_col = arguments[0].column->convertToFullIfNeeded();
const auto & date_time_col_data = typeid_cast<const typename InputDataType::ColumnType &>(*date_time_col).getData();
auto value_col = castColumn(arguments[1], std::make_shared<DataTypeFloat64>());
value_col = value_col->convertToFullIfNeeded();
const auto & value_col_data = typeid_cast<const ColumnFloat64 &>(*value_col).getData();
auto & result_col_data = result_col->getData();
if constexpr (std::is_same_v<InputDataType, DataTypeDateTime64>)
{
const auto scale = typeid_cast<const DataTypeDateTime64 &>(*result_type).getScale();
const auto & date_lut = typeid_cast<const DataTypeDateTime64 &>(*result_type).getTimeZone();
Int64 deg = 1;
for (size_t j = 0; j < scale; ++j)
deg *= 10;
for (size_t i = 0; i < input_rows_count; ++i)
{
Int64 time = date_lut.toNumYYYYMMDDhhmmss(date_time_col_data[i] / deg);
Int64 fraction = date_time_col_data[i] % deg;
result_col_data[i] = getChangedDate(time, value_col_data[i], result_type, date_lut, scale, fraction);
}
}
else if constexpr (std::is_same_v<InputDataType, DataTypeDate32> && std::is_same_v<ResultDataType, DataTypeDateTime64>)
{
const auto & date_lut = typeid_cast<const DataTypeDateTime64 &>(*result_type).getTimeZone();
for (size_t i = 0; i < input_rows_count; ++i)
{
Int64 time = static_cast<Int64>(date_lut.toNumYYYYMMDD(ExtendedDayNum(date_time_col_data[i]))) * 1'000'000;
result_col_data[i] = getChangedDate(time, value_col_data[i], result_type, date_lut, 3, 0);
}
}
else if constexpr (std::is_same_v<InputDataType, DataTypeDate> && std::is_same_v<ResultDataType, DataTypeDateTime>)
{
const auto & date_lut = typeid_cast<const DataTypeDateTime &>(*result_type).getTimeZone();
for (size_t i = 0; i < input_rows_count; ++i)
{
Int64 time = static_cast<Int64>(date_lut.toNumYYYYMMDD(ExtendedDayNum(date_time_col_data[i]))) * 1'000'000;
result_col_data[i] = static_cast<UInt32>(getChangedDate(time, value_col_data[i], result_type, date_lut));
}
}
else if constexpr (std::is_same_v<InputDataType, DataTypeDateTime>)
{
const auto & date_lut = typeid_cast<const DataTypeDateTime &>(*result_type).getTimeZone();
for (size_t i = 0; i < input_rows_count; ++i)
{
Int64 time = date_lut.toNumYYYYMMDDhhmmss(date_time_col_data[i]);
result_col_data[i] = static_cast<UInt32>(getChangedDate(time, value_col_data[i], result_type, date_lut));
}
}
else
{
const auto & date_lut = DateLUT::instance();
for (size_t i = 0; i < input_rows_count; ++i)
{
Int64 time;
if (isDate(input_type))
time = static_cast<Int64>(date_lut.toNumYYYYMMDD(DayNum(date_time_col_data[i]))) * 1'000'000;
else
time = static_cast<Int64>(date_lut.toNumYYYYMMDD(ExtendedDayNum(date_time_col_data[i]))) * 1'000'000;
if (isDate(result_type))
result_col_data[i] = static_cast<UInt16>(getChangedDate(time, value_col_data[i], result_type, date_lut));
else
result_col_data[i] = static_cast<Int32>(getChangedDate(time, value_col_data[i], result_type, date_lut));
}
}
return result_col;
}
Int64 getChangedDate(Int64 time, Float64 new_value, const DataTypePtr & result_type, const DateLUTImpl & date_lut, Int64 scale = 0, Int64 fraction = 0) const
{
auto year = time / 10'000'000'000;
auto month = (time % 10'000'000'000) / 100'000'000;
auto day = (time % 100'000'000) / 1'000'000;
auto hours = (time % 1'000'000) / 10'000;
auto minutes = (time % 10'000) / 100;
auto seconds = time % 100;
Int64 min_date = 0, max_date = 0;
Int16 min_year, max_year;
if (isDate(result_type))
{
min_date = date_lut.makeDayNum(1970, 1, 1);
max_date = date_lut.makeDayNum(2149, 6, 6);
min_year = 1970;
max_year = 2149;
}
else if (isDate32(result_type))
{
min_date = date_lut.makeDayNum(1900, 1, 1);
max_date = date_lut.makeDayNum(2299, 12, 31);
min_year = 1900;
max_year = 2299;
}
else if (isDateTime(result_type))
{
min_date = 0;
max_date = 0x0FFFFFFFFLL;
min_year = 1970;
max_year = 2106;
}
else
{
min_date = DecimalUtils::decimalFromComponents<DateTime64>(
date_lut.makeDateTime(1900, 1, 1, 0, 0, 0),
static_cast<Int64>(0),
static_cast<UInt32>(scale));
Int64 deg = 1;
for (Int64 j = 0; j < scale; ++j)
deg *= 10;
max_date = DecimalUtils::decimalFromComponents<DateTime64>(
date_lut.makeDateTime(2299, 12, 31, 23, 59, 59),
static_cast<Int64>(deg - 1),
static_cast<UInt32>(scale));
min_year = 1900;
max_year = 2299;
}
switch (Traits::component)
{
case Component::Year:
if (new_value < min_year)
return min_date;
else if (new_value > max_year)
return max_date;
year = static_cast<Int16>(new_value);
break;
case Component::Month:
if (new_value < 1 || new_value > 12)
return min_date;
month = static_cast<UInt8>(new_value);
break;
case Component::Day:
if (new_value < 1 || new_value > 31)
return min_date;
day = static_cast<UInt8>(new_value);
break;
case Component::Hour:
if (new_value < 0 || new_value > 23)
return min_date;
hours = static_cast<UInt8>(new_value);
break;
case Component::Minute:
if (new_value < 0 || new_value > 59)
return min_date;
minutes = static_cast<UInt8>(new_value);
break;
case Component::Second:
if (new_value < 0 || new_value > 59)
return min_date;
seconds = static_cast<UInt8>(new_value);
break;
}
Int64 result;
if (isDate(result_type) || isDate32(result_type))
result = date_lut.makeDayNum(year, month, day);
else if (isDateTime(result_type))
result = date_lut.makeDateTime(year, month, day, hours, minutes, seconds);
else
#ifndef __clang_analyzer__
/// ^^ This looks funny. It is the least terrible suppression of a false positive reported by clang-analyzer (a sub-class
/// of clang-tidy checks) deep down in 'decimalFromComponents'. Usual suppressions of the form NOLINT* don't work here (they
/// would only affect code in _this_ file), and suppressing the issue in 'decimalFromComponents' may suppress true positives.
result = DecimalUtils::decimalFromComponents<DateTime64>(
date_lut.makeDateTime(year, month, day, hours, minutes, seconds),
fraction,
static_cast<UInt32>(scale));
#else
{
UNUSED(fraction);
result = 0;
}
#endif
if (result < min_date)
return min_date;
if (result > max_date)
return max_date;
return result;
}
};
struct ChangeYearTraits
{
static constexpr auto name = "changeYear";
static constexpr auto component = Component::Year;
};
struct ChangeMonthTraits
{
static constexpr auto name = "changeMonth";
static constexpr auto component = Component::Month;
};
struct ChangeDayTraits
{
static constexpr auto name = "changeDay";
static constexpr auto component = Component::Day;
};
struct ChangeHourTraits
{
static constexpr auto name = "changeHour";
static constexpr auto component = Component::Hour;
};
struct ChangeMinuteTraits
{
static constexpr auto name = "changeMinute";
static constexpr auto component = Component::Minute;
};
struct ChangeSecondTraits
{
static constexpr auto name = "changeSecond";
static constexpr auto component = Component::Second;
};
REGISTER_FUNCTION(ChangeDate)
{
{
FunctionDocumentation::Description description = "Changes the year component of a date or date time.";
FunctionDocumentation::Syntax syntax = "changeYear(date_or_datetime, value);";
FunctionDocumentation::Arguments arguments = {{"date_or_datetime", "The value to change. Type: Date, Date32, DateTime, or DateTime64"}, {"value", "The new value. Type: [U]Int*"}};
FunctionDocumentation::ReturnedValue returned_value = "The same type as date_or_datetime.";
FunctionDocumentation::Categories categories = {"Dates and Times"};
FunctionDocumentation function_documentation = {.description = description, .syntax = syntax, .arguments = arguments, .returned_value = returned_value, .categories = categories};
factory.registerFunction<FunctionChangeDate<ChangeYearTraits>>(function_documentation);
}
{
FunctionDocumentation::Description description = "Changes the month component of a date or date time.";
FunctionDocumentation::Syntax syntax = "changeMonth(date_or_datetime, value);";
FunctionDocumentation::Arguments arguments = {{"date_or_datetime", "The value to change. Type: Date, Date32, DateTime, or DateTime64"}, {"value", "The new value. Type: [U]Int*"}};
FunctionDocumentation::ReturnedValue returned_value = "The same type as date_or_datetime.";
FunctionDocumentation::Categories categories = {"Dates and Times"};
FunctionDocumentation function_documentation = {.description = description, .syntax = syntax, .arguments = arguments, .returned_value = returned_value, .categories = categories};
factory.registerFunction<FunctionChangeDate<ChangeMonthTraits>>(function_documentation);
}
{
FunctionDocumentation::Description description = "Changes the day component of a date or date time.";
FunctionDocumentation::Syntax syntax = "changeDay(date_or_datetime, value);";
FunctionDocumentation::Arguments arguments = {{"date_or_datetime", "The value to change. Type: Date, Date32, DateTime, or DateTime64"}, {"value", "The new value. Type: [U]Int*"}};
FunctionDocumentation::ReturnedValue returned_value = "The same type as date_or_datetime.";
FunctionDocumentation::Categories categories = {"Dates and Times"};
FunctionDocumentation function_documentation = {.description = description, .syntax = syntax, .arguments = arguments, .returned_value = returned_value, .categories = categories};
factory.registerFunction<FunctionChangeDate<ChangeDayTraits>>(function_documentation);
}
{
FunctionDocumentation::Description description = "Changes the hour component of a date or date time.";
FunctionDocumentation::Syntax syntax = "changeHour(date_or_datetime, value);";
FunctionDocumentation::Arguments arguments = {{"date_or_datetime", "The value to change. Type: Date, Date32, DateTime, or DateTime64"}, {"value", "The new value. Type: [U]Int*"}};
FunctionDocumentation::ReturnedValue returned_value = "The same type as date_or_datetime. If the input is a Date, return DateTime. If the input is a Date32, return DateTime64.";
FunctionDocumentation::Categories categories = {"Dates and Times"};
FunctionDocumentation function_documentation = {.description = description, .syntax = syntax, .arguments = arguments, .returned_value = returned_value, .categories = categories};
factory.registerFunction<FunctionChangeDate<ChangeHourTraits>>(function_documentation);
}
{
FunctionDocumentation::Description description = "Changes the minute component of a date or date time.";
FunctionDocumentation::Syntax syntax = "changeMinute(date_or_datetime, value);";
FunctionDocumentation::Arguments arguments = {{"date_or_datetime", "The value to change. Type: Date, Date32, DateTime, or DateTime64"}, {"value", "The new value. Type: [U]Int*"}};
FunctionDocumentation::ReturnedValue returned_value = "The same type as date_or_datetime. If the input is a Date, return DateTime. If the input is a Date32, return DateTime64.";
FunctionDocumentation::Categories categories = {"Dates and Times"};
FunctionDocumentation function_documentation = {.description = description, .syntax = syntax, .arguments = arguments, .returned_value = returned_value, .categories = categories};
factory.registerFunction<FunctionChangeDate<ChangeMinuteTraits>>(function_documentation);
}
{
FunctionDocumentation::Description description = "Changes the second component of a date or date time.";
FunctionDocumentation::Syntax syntax = "changeSecond(date_or_datetime, value);";
FunctionDocumentation::Arguments arguments = {{"date_or_datetime", "The value to change. Type: Date, Date32, DateTime, or DateTime64"}, {"value", "The new value. Type: [U]Int*"}};
FunctionDocumentation::ReturnedValue returned_value = "The same type as date_or_datetime. If the input is a Date, return DateTime. If the input is a Date32, return DateTime64.";
FunctionDocumentation::Categories categories = {"Dates and Times"};
FunctionDocumentation function_documentation = {.description = description, .syntax = syntax, .arguments = arguments, .returned_value = returned_value, .categories = categories};
factory.registerFunction<FunctionChangeDate<ChangeSecondTraits>>(function_documentation);
}
}
}

View File

@ -6,6 +6,7 @@ import os
import re
import subprocess
import sys
import time
from dataclasses import dataclass
from pathlib import Path
from typing import Any, Dict, List, Optional
@ -550,7 +551,17 @@ def _update_gh_statuses_action(indata: Dict, s3: S3Helper) -> None:
except Exception as e:
raise e
print("Going to update overall CI report")
set_status_comment(commit, pr_info)
for retry in range(2):
try:
set_status_comment(commit, pr_info)
break
except Exception as e:
print(
f"WARNING: Failed to update CI Running status, attempt [{retry + 1}], exception [{e}]"
)
time.sleep(1)
else:
print("ERROR: All retry attempts failed.")
print("... CI report update - done")
@ -996,10 +1007,10 @@ def main() -> int:
args.skip_jobs,
)
ci_cache.print_status()
if IS_CI and pr_info.is_pr and not ci_settings.no_ci_cache:
ci_cache.filter_out_not_affected_jobs()
ci_cache.print_status()
ci_cache.print_status()
if IS_CI and not pr_info.is_merge_queue:
# wait for pending jobs to be finished, await_jobs is a long blocking call

View File

@ -714,7 +714,7 @@ class CiCache:
if CI.is_test_job(job_name) and job_name != CI.JobNames.BUILD_CHECK:
if job_config.reference_job_name:
reference_name = job_config.reference_job_name
reference_config = self.jobs_to_do[reference_name]
reference_config = CI.JOB_CONFIGS[reference_name]
else:
reference_name = job_name
reference_config = job_config
@ -745,7 +745,8 @@ class CiCache:
del self.jobs_to_do[job]
if job in self.jobs_to_wait:
del self.jobs_to_wait[job]
self.jobs_to_skip.append(job)
if job in self.jobs_to_skip:
self.jobs_to_skip.remove(job)
def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None:
"""

View File

@ -0,0 +1,169 @@
Negative tests
changeYear
-- Date
2001-01-01
1970-01-01
1970-01-01
2149-06-06
-- Date32
2001-01-01
1900-01-01
1900-01-01
2299-12-31
-- DateTime
2001-01-01 11:22:33
1970-01-01 01:00:00
1970-01-01 01:00:00
2106-02-07 07:28:15
-- DateTime64
2001-01-01 11:22:33.4444
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
2299-12-31 23:59:59.9999
changeMonth
-- Date
2000-01-01
2000-02-01
2000-12-01
1970-01-01
1970-01-01
1970-01-01
-- Date32
2000-01-01
2000-02-01
2000-12-01
1900-01-01
1900-01-01
1900-01-01
-- DateTime
2000-01-01 11:22:33
2000-02-01 11:22:33
2000-12-01 11:22:33
1970-01-01 01:00:00
1970-01-01 01:00:00
1970-01-01 01:00:00
-- DateTime64
2000-01-01 11:22:33.4444
2000-02-01 11:22:33.4444
2000-12-01 11:22:33.4444
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
changeDay
-- Date
2000-01-01
2000-01-02
2000-01-31
1970-01-01
1970-01-01
1970-01-01
-- Date32
2000-01-01
2000-01-02
2000-01-31
1900-01-01
1900-01-01
1900-01-01
-- DateTime
2000-01-01 11:22:33
2000-01-02 11:22:33
2000-01-31 11:22:33
1970-01-01 01:00:00
1970-01-01 01:00:00
1970-01-01 01:00:00
-- DateTime64
2000-01-01 11:22:33.4444
2000-01-02 11:22:33.4444
2000-01-31 11:22:33.4444
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
-- Special case: change to 29 Feb in a leap year
2000-02-29
2000-02-29
2000-02-29 11:22:33
2000-02-29 11:22:33.4444
changeHour
-- Date
2000-01-01 00:00:00
2000-01-01 02:00:00
2000-01-01 23:00:00
1970-01-01 01:00:00
1970-01-01 01:00:00
-- Date32
2000-01-01 00:00:00.000
2000-01-01 02:00:00.000
2000-01-01 23:00:00.000
1900-01-01 00:00:00.000
1900-01-01 00:00:00.000
-- DateTime
2000-01-01 00:22:33
2000-01-01 02:22:33
2000-01-01 23:22:33
1970-01-01 01:00:00
1970-01-01 01:00:00
-- DateTime64
2000-01-01 00:22:33.4444
2000-01-01 02:22:33.4444
2000-01-01 23:22:33.4444
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
-- With different timezone
1970-01-01 07:00:00
1970-01-01 07:00:00
changeMinute
-- Date
2000-01-01 00:00:00
2000-01-01 00:02:00
2000-01-01 00:59:00
1970-01-01 01:00:00
1970-01-01 01:00:00
-- Date32
2000-01-01 00:00:00.000
2000-01-01 00:02:00.000
2000-01-01 00:59:00.000
1900-01-01 00:00:00.000
1900-01-01 00:00:00.000
-- DateTime
2000-01-01 11:00:33
2000-01-01 11:02:33
2000-01-01 11:59:33
1970-01-01 01:00:00
1970-01-01 01:00:00
-- DateTime64
2000-01-01 11:00:33.4444
2000-01-01 11:02:33.4444
2000-01-01 11:59:33.4444
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
-- With different timezone
1970-01-01 07:00:00
1970-01-01 07:00:00
changeSecond
-- Date
2000-01-01 00:00:00
2000-01-01 00:00:02
2000-01-01 00:00:59
1970-01-01 01:00:00
1970-01-01 01:00:00
-- Date32
2000-01-01 00:00:00.000
2000-01-01 00:00:02.000
2000-01-01 00:00:59.000
1900-01-01 00:00:00.000
1900-01-01 00:00:00.000
-- DateTime
2000-01-01 11:22:00
2000-01-01 11:22:02
2000-01-01 11:22:59
1970-01-01 01:00:00
1970-01-01 01:00:00
-- DateTime64
2000-01-01 11:22:00.4444
2000-01-01 11:22:02.4444
2000-01-01 11:22:59.4444
1900-01-01 00:00:00.0000
1900-01-01 00:00:00.0000
-- With different timezone
1970-01-01 07:00:00
1970-01-01 07:00:00

View File

@ -0,0 +1,185 @@
SELECT 'Negative tests';
-- as changeYear, changeMonth, changeDay, changeMinute, changeSecond share the same implementation, just testing one of them
SELECT changeYear(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT changeYear(toDate('2000-01-01')); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT changeYear(toDate('2000-01-01'), 2000, 1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT changeYear(1999, 2000); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT changeYear(toDate('2000-01-01'), 'abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT changeYear(toDate('2000-01-01'), 1.5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Disable timezone randomization
SET session_timezone='CET';
SELECT 'changeYear';
SELECT '-- Date';
SELECT changeYear(toDate('2000-01-01'), 2001);
SELECT changeYear(toDate('2000-01-01'), 1800); -- out-of-bounds
SELECT changeYear(toDate('2000-01-01'), -5000); -- out-of-bounds
SELECT changeYear(toDate('2000-01-01'), 2500); -- out-of-bounds
SELECT '-- Date32';
SELECT changeYear(toDate32('2000-01-01'), 2001);
SELECT changeYear(toDate32('2000-01-01'), 1800); -- out-of-bounds
SELECT changeYear(toDate32('2000-01-01'), -5000); -- out-of-bounds
SELECT changeYear(toDate32('2000-01-01'), 2500); -- out-of-bounds
SELECT '-- DateTime';
SELECT changeYear(toDateTime('2000-01-01 11:22:33'), 2001);
SELECT changeYear(toDateTime('2000-01-01 11:22:33'), 1800); -- out-of-bounds
SELECT changeYear(toDateTime('2000-01-01 11:22:33'), -5000); -- out-of-bounds
SELECT changeYear(toDateTime('2000-01-01 11:22:33'), 2500); -- out-of-bounds
SELECT '-- DateTime64';
SELECT changeYear(toDateTime64('2000-01-01 11:22:33.4444', 4), 2001);
SELECT changeYear(toDateTime64('2000-01-01 11:22:33.4444', 4), 1800); -- out-of-bounds
SELECT changeYear(toDateTime64('2000-01-01 11:22:33.4444', 4), -5000); -- out-of-bounds
SELECT changeYear(toDateTime64('2000-01-01 11:22:33.4444', 4), 2500); -- out-of-bounds
SELECT 'changeMonth';
SELECT '-- Date';
SELECT changeMonth(toDate('2000-01-01'), 1);
SELECT changeMonth(toDate('2000-01-01'), 2);
SELECT changeMonth(toDate('2000-01-01'), 12);
SELECT changeMonth(toDate('2000-01-01'), 0); -- out-of-bounds
SELECT changeMonth(toDate('2000-01-01'), -1); -- out-of-bounds
SELECT changeMonth(toDate('2000-01-01'), 13); -- out-of-bounds
SELECT '-- Date32';
SELECT changeMonth(toDate32('2000-01-01'), 1);
SELECT changeMonth(toDate32('2000-01-01'), 2);
SELECT changeMonth(toDate32('2000-01-01'), 12);
SELECT changeMonth(toDate32('2000-01-01'), 0); -- out-of-bounds
SELECT changeMonth(toDate32('2000-01-01'), -1); -- out-of-bounds
SELECT changeMonth(toDate32('2000-01-01'), 13); -- out-of-bounds
SELECT '-- DateTime';
SELECT changeMonth(toDateTime('2000-01-01 11:22:33'), 1);
SELECT changeMonth(toDateTime('2000-01-01 11:22:33'), 2);
SELECT changeMonth(toDateTime('2000-01-01 11:22:33'), 12);
SELECT changeMonth(toDateTime('2000-01-01 11:22:33'), 0); -- out-of-bounds
SELECT changeMonth(toDateTime('2000-01-01 11:22:33'), -1); -- out-of-bounds
SELECT changeMonth(toDateTime('2000-01-01 11:22:33'), 13); -- out-of-bounds
SELECT '-- DateTime64';
SELECT changeMonth(toDateTime64('2000-01-01 11:22:33.4444', 4), 1);
SELECT changeMonth(toDateTime64('2000-01-01 11:22:33.4444', 4), 2);
SELECT changeMonth(toDateTime64('2000-01-01 11:22:33.4444', 4), 12);
SELECT changeMonth(toDateTime64('2000-01-01 11:22:33.4444', 4), 0); -- out-of-bounds
SELECT changeMonth(toDateTime64('2000-01-01 11:22:33.4444', 4), -1); -- out-of-bounds
SELECT changeMonth(toDateTime64('2000-01-01 11:22:33.4444', 4), 13); -- out-of-bounds
SELECT 'changeDay';
SELECT '-- Date';
SELECT changeDay(toDate('2000-01-01'), 1);
SELECT changeDay(toDate('2000-01-01'), 2);
SELECT changeDay(toDate('2000-01-01'), 31);
SELECT changeDay(toDate('2000-01-01'), 0); -- out-of-bounds
SELECT changeDay(toDate('2000-01-01'), -1); -- out-of-bounds
SELECT changeDay(toDate('2000-01-01'), 32); -- out-of-bounds
SELECT '-- Date32';
SELECT changeDay(toDate32('2000-01-01'), 1);
SELECT changeDay(toDate32('2000-01-01'), 2);
SELECT changeDay(toDate32('2000-01-01'), 31);
SELECT changeDay(toDate32('2000-01-01'), 0); -- out-of-bounds
SELECT changeDay(toDate32('2000-01-01'), -1); -- out-of-bounds
SELECT changeDay(toDate32('2000-01-01'), 32); -- out-of-bounds
SELECT '-- DateTime';
SELECT changeDay(toDateTime('2000-01-01 11:22:33'), 1);
SELECT changeDay(toDateTime('2000-01-01 11:22:33'), 2);
SELECT changeDay(toDateTime('2000-01-01 11:22:33'), 31);
SELECT changeDay(toDateTime('2000-01-01 11:22:33'), 0); -- out-of-bounds
SELECT changeDay(toDateTime('2000-01-01 11:22:33'), -1); -- out-of-bounds
SELECT changeDay(toDateTime('2000-01-01 11:22:33'), 32); -- out-of-bounds
SELECT '-- DateTime64';
SELECT changeDay(toDateTime64('2000-01-01 11:22:33.4444', 4), 1);
SELECT changeDay(toDateTime64('2000-01-01 11:22:33.4444', 4), 2);
SELECT changeDay(toDateTime64('2000-01-01 11:22:33.4444', 4), 31);
SELECT changeDay(toDateTime64('2000-01-01 11:22:33.4444', 4), 0); -- out-of-bounds
SELECT changeDay(toDateTime64('2000-01-01 11:22:33.4444', 4), -1); -- out-of-bounds
SELECT changeDay(toDateTime64('2000-01-01 11:22:33.4444', 4), 32); -- out-of-bounds
SELECT '-- Special case: change to 29 Feb in a leap year';
SELECT changeDay(toDate('2000-02-28'), 29);
SELECT changeDay(toDate32('2000-02-01'), 29);
SELECT changeDay(toDateTime('2000-02-01 11:22:33'), 29);
SELECT changeDay(toDateTime64('2000-02-01 11:22:33.4444', 4), 29);
SELECT 'changeHour';
SELECT '-- Date';
SELECT changeHour(toDate('2000-01-01'), 0);
SELECT changeHour(toDate('2000-01-01'), 2);
SELECT changeHour(toDate('2000-01-01'), 23);
SELECT changeHour(toDate('2000-01-01'), -1); -- out-of-bounds
SELECT changeHour(toDate('2000-01-01'), 24); -- out-of-bounds
SELECT '-- Date32';
SELECT changeHour(toDate32('2000-01-01'), 0);
SELECT changeHour(toDate32('2000-01-01'), 2);
SELECT changeHour(toDate32('2000-01-01'), 23);
SELECT changeHour(toDate32('2000-01-01'), -1); -- out-of-bounds
SELECT changeHour(toDate32('2000-01-01'), 24); -- out-of-bounds
SELECT '-- DateTime';
SELECT changeHour(toDateTime('2000-01-01 11:22:33'), 0);
SELECT changeHour(toDateTime('2000-01-01 11:22:33'), 2);
SELECT changeHour(toDateTime('2000-01-01 11:22:33'), 23);
SELECT changeHour(toDateTime('2000-01-01 11:22:33'), -1); -- out-of-bounds
SELECT changeHour(toDateTime('2000-01-01 11:22:33'), 24); -- out-of-bounds
SELECT '-- DateTime64';
SELECT changeHour(toDateTime64('2000-01-01 11:22:33.4444', 4), 0);
SELECT changeHour(toDateTime64('2000-01-01 11:22:33.4444', 4), 2);
SELECT changeHour(toDateTime64('2000-01-01 11:22:33.4444', 4), 23);
SELECT changeHour(toDateTime64('2000-01-01 11:22:33.4444', 4), -1); -- out-of-bounds
SELECT changeHour(toDateTime64('2000-01-01 11:22:33.4444', 4), 24); -- out-of-bounds
SELECT '-- With different timezone';
SELECT changeHour(toDate('2000-01-01'), -1) SETTINGS session_timezone = 'Asia/Novosibirsk';
SELECT changeHour(toDate('2000-01-01'), 24) SETTINGS session_timezone = 'Asia/Novosibirsk';
SELECT 'changeMinute';
SELECT '-- Date';
SELECT changeMinute(toDate('2000-01-01'), 0);
SELECT changeMinute(toDate('2000-01-01'), 2);
SELECT changeMinute(toDate('2000-01-01'), 59);
SELECT changeMinute(toDate('2000-01-01'), -1); -- out-of-bounds
SELECT changeMinute(toDate('2000-01-01'), 60); -- out-of-bounds
SELECT '-- Date32';
SELECT changeMinute(toDate32('2000-01-01'), 0);
SELECT changeMinute(toDate32('2000-01-01'), 2);
SELECT changeMinute(toDate32('2000-01-01'), 59);
SELECT changeMinute(toDate32('2000-01-01'), -1); -- out-of-bounds
SELECT changeMinute(toDate32('2000-01-01'), 60); -- out-of-bounds
SELECT '-- DateTime';
SELECT changeMinute(toDateTime('2000-01-01 11:22:33'), 0);
SELECT changeMinute(toDateTime('2000-01-01 11:22:33'), 2);
SELECT changeMinute(toDateTime('2000-01-01 11:22:33'), 59);
SELECT changeMinute(toDateTime('2000-01-01 11:22:33'), -1); -- out-of-bounds
SELECT changeMinute(toDateTime('2000-01-01 11:22:33'), 60); -- out-of-bounds
SELECT '-- DateTime64';
SELECT changeMinute(toDateTime64('2000-01-01 11:22:33.4444', 4), 0);
SELECT changeMinute(toDateTime64('2000-01-01 11:22:33.4444', 4), 2);
SELECT changeMinute(toDateTime64('2000-01-01 11:22:33.4444', 4), 59);
SELECT changeMinute(toDateTime64('2000-01-01 11:22:33.4444', 4), -1); -- out-of-bounds
SELECT changeMinute(toDateTime64('2000-01-01 11:22:33.4444', 4), 60); -- out-of-bounds
SELECT '-- With different timezone';
SELECT changeMinute(toDate('2000-01-01'), -1) SETTINGS session_timezone = 'Asia/Novosibirsk';
SELECT changeMinute(toDate('2000-01-01'), 60) SETTINGS session_timezone = 'Asia/Novosibirsk';
SELECT 'changeSecond';
SELECT '-- Date';
SELECT changeSecond(toDate('2000-01-01'), 0);
SELECT changeSecond(toDate('2000-01-01'), 2);
SELECT changeSecond(toDate('2000-01-01'), 59);
SELECT changeSecond(toDate('2000-01-01'), -1); -- out-of-bounds
SELECT changeSecond(toDate('2000-01-01'), 60); -- out-of-bounds
SELECT '-- Date32';
SELECT changeSecond(toDate32('2000-01-01'), 0);
SELECT changeSecond(toDate32('2000-01-01'), 2);
SELECT changeSecond(toDate32('2000-01-01'), 59);
SELECT changeSecond(toDate32('2000-01-01'), -1); -- out-of-bounds
SELECT changeSecond(toDate32('2000-01-01'), 60); -- out-of-bounds
SELECT '-- DateTime';
SELECT changeSecond(toDateTime('2000-01-01 11:22:33'), 0);
SELECT changeSecond(toDateTime('2000-01-01 11:22:33'), 2);
SELECT changeSecond(toDateTime('2000-01-01 11:22:33'), 59);
SELECT changeSecond(toDateTime('2000-01-01 11:22:33'), -1); -- out-of-bounds
SELECT changeSecond(toDateTime('2000-01-01 11:22:33'), 60); -- out-of-bounds
SELECT '-- DateTime64';
SELECT changeSecond(toDateTime64('2000-01-01 11:22:33.4444', 4), 0);
SELECT changeSecond(toDateTime64('2000-01-01 11:22:33.4444', 4), 2);
SELECT changeSecond(toDateTime64('2000-01-01 11:22:33.4444', 4), 59);
SELECT changeSecond(toDateTime64('2000-01-01 11:22:33.4444', 4), -1); -- out-of-bounds
SELECT changeSecond(toDateTime64('2000-01-01 11:22:33.4444', 4), 60); -- out-of-bounds
SELECT '-- With different timezone';
SELECT changeSecond(toDate('2000-01-01'), -1) SETTINGS session_timezone = 'Asia/Novosibirsk';
SELECT changeSecond(toDate('2000-01-01'), 60) SETTINGS session_timezone = 'Asia/Novosibirsk';

View File

@ -467,7 +467,7 @@ LOCALTIME
LOCALTIMESTAMP
LONGLONG
LOONGARCH
Lemir
Lemire
Levenshtein
Liao
LibFuzzer
@ -1319,6 +1319,12 @@ cfg
cgroup
cgroups
chadmin
changeDay
changeHour
changeMinute
changeMonth
changeSecond
changeYear
changelog
changelogs
charset