diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index f297b31e0212..7fae297e16ae 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -544,7 +544,7 @@ jobs: secrets: inherit with: runner_type: altinity-on-demand, altinity-type-cpx51, altinity-image-x86-app-docker-ce, altinity-setup-regression - commit: f9e29772f4d261b82c23189d89038eb7ba027865 + commit: e3c00be97a045aa04e9d1a6ec50cc64f4c387b70 arch: release build_sha: ${{ github.event_name == 'pull_request' && github.event.pull_request.head.sha || github.sha }} timeout_minutes: 300 @@ -555,7 +555,7 @@ jobs: secrets: inherit with: runner_type: altinity-on-demand, altinity-type-cax41, altinity-image-arm-app-docker-ce, altinity-setup-regression - commit: f9e29772f4d261b82c23189d89038eb7ba027865 + commit: e3c00be97a045aa04e9d1a6ec50cc64f4c387b70 arch: aarch64 build_sha: ${{ github.event_name == 'pull_request' && github.event.pull_request.head.sha || github.sha }} timeout_minutes: 300 diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 183b94f46418..7e898d20f22c 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -519,7 +519,7 @@ Determines the lifetime of values. The `TTL` clause can be set for the whole table and for each individual column. Table-level `TTL` can also specify the logic of automatic moving data between disks and volumes, or recompressing parts where all the data has been expired. -Expressions must evaluate to [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type. +Expressions must evaluate to [Date](/docs/en/sql-reference/data-types/date.md), [Date32](/docs/en/sql-reference/data-types/date32.md), [DateTime](/docs/en/sql-reference/data-types/datetime.md) or [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) data type. **Syntax** diff --git a/src/Processors/TTL/ITTLAlgorithm.cpp b/src/Processors/TTL/ITTLAlgorithm.cpp index 761f43e2422e..9dd69a656a30 100644 --- a/src/Processors/TTL/ITTLAlgorithm.cpp +++ b/src/Processors/TTL/ITTLAlgorithm.cpp @@ -2,6 +2,8 @@ #include #include +#include + namespace DB { @@ -46,18 +48,26 @@ ColumnPtr ITTLAlgorithm::executeExpressionAndGetColumn( return block_copy.getByName(result_column).column; } -UInt32 ITTLAlgorithm::getTimestampByIndex(const IColumn * column, size_t index) const +Int64 ITTLAlgorithm::getTimestampByIndex(const IColumn * column, size_t index) const { if (const ColumnUInt16 * column_date = typeid_cast(column)) - return static_cast(date_lut.fromDayNum(DayNum(column_date->getData()[index]))); - else if (const ColumnUInt32 * column_date_time = typeid_cast(column)) + return date_lut.fromDayNum(DayNum(column_date->getData()[index])); + if (const ColumnUInt32 * column_date_time = typeid_cast(column)) return column_date_time->getData()[index]; - else if (const ColumnConst * column_const = typeid_cast(column)) + if (const ColumnInt32 * column_date_32 = typeid_cast(column)) + return date_lut.fromDayNum(ExtendedDayNum(column_date_32->getData()[index])); + if (const ColumnDateTime64 * column_date_time_64 = typeid_cast(column)) + return column_date_time_64->getData()[index] / intExp10OfSize(column_date_time_64->getScale()); + if (const ColumnConst * column_const = typeid_cast(column)) { if (typeid_cast(&column_const->getDataColumn())) - return static_cast(date_lut.fromDayNum(DayNum(column_const->getValue()))); - else if (typeid_cast(&column_const->getDataColumn())) + return date_lut.fromDayNum(DayNum(column_const->getValue())); + if (typeid_cast(&column_const->getDataColumn())) return column_const->getValue(); + if (typeid_cast(&column_const->getDataColumn())) + return date_lut.fromDayNum(ExtendedDayNum(column_const->getValue())); + if (const ColumnDateTime64 * column_dt64 = typeid_cast(&column_const->getDataColumn())) + return column_const->getValue() / intExp10OfSize(column_dt64->getScale()); } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of result TTL column"); diff --git a/src/Processors/TTL/ITTLAlgorithm.h b/src/Processors/TTL/ITTLAlgorithm.h index d79aa8a8dfcf..542f93d6ffe6 100644 --- a/src/Processors/TTL/ITTLAlgorithm.h +++ b/src/Processors/TTL/ITTLAlgorithm.h @@ -43,7 +43,7 @@ class ITTLAlgorithm protected: bool isTTLExpired(time_t ttl) const; - UInt32 getTimestampByIndex(const IColumn * column, size_t index) const; + Int64 getTimestampByIndex(const IColumn * column, size_t index) const; const TTLExpressions ttl_expressions; const TTLDescription description; diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index 2d7a37d0abea..1a4c52774033 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -86,7 +86,7 @@ void TTLAggregationAlgorithm::execute(Block & block) for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); + Int64 cur_ttl = getTimestampByIndex(ttl_column.get(), i); bool where_filter_passed = !where_column || where_column->getBool(i); bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; diff --git a/src/Processors/TTL/TTLColumnAlgorithm.cpp b/src/Processors/TTL/TTLColumnAlgorithm.cpp index e27050564cee..4b8f6b6f9ba2 100644 --- a/src/Processors/TTL/TTLColumnAlgorithm.cpp +++ b/src/Processors/TTL/TTLColumnAlgorithm.cpp @@ -59,7 +59,7 @@ void TTLColumnAlgorithm::execute(Block & block) for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); + Int64 cur_ttl = getTimestampByIndex(ttl_column.get(), i); if (isTTLExpired(cur_ttl)) { if (default_column) diff --git a/src/Processors/TTL/TTLDeleteAlgorithm.cpp b/src/Processors/TTL/TTLDeleteAlgorithm.cpp index 6f9bc315276f..fde1d7c51d54 100644 --- a/src/Processors/TTL/TTLDeleteAlgorithm.cpp +++ b/src/Processors/TTL/TTLDeleteAlgorithm.cpp @@ -34,7 +34,7 @@ void TTLDeleteAlgorithm::execute(Block & block) for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); + Int64 cur_ttl = getTimestampByIndex(ttl_column.get(), i); bool where_filter_passed = !where_column || where_column->getBool(i); if (!isTTLExpired(cur_ttl) || !where_filter_passed) diff --git a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp index 13d3030bbb87..e27698e7cb44 100644 --- a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp @@ -25,7 +25,7 @@ void TTLUpdateInfoAlgorithm::execute(Block & block) auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column); for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i); + Int64 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i); new_ttl_info.update(cur_ttl); } } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f29d715e791d..aee7bc404d61 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -1,4 +1,7 @@ #include +#include +#include +#include #include #include #include @@ -159,6 +162,17 @@ void updateTTL( for (const auto & val : column_date_time->getData()) ttl_info.update(val); } + else if (const ColumnInt32 * column_date_32 = typeid_cast(ttl_column.get())) + { + const auto & date_lut = DateLUT::serverTimezoneInstance(); + for (const auto & val : column_date_32->getData()) + ttl_info.update(date_lut.fromDayNum(ExtendedDayNum(val))); + } + else if (const ColumnDateTime64 * column_date_time_64 = typeid_cast(ttl_column.get())) + { + for (const auto & val : column_date_time_64->getData()) + ttl_info.update(val / intExp10OfSize(column_date_time_64->getScale())); + } else if (const ColumnConst * column_const = typeid_cast(ttl_column.get())) { if (typeid_cast(&column_const->getDataColumn())) @@ -170,6 +184,15 @@ void updateTTL( { ttl_info.update(column_const->getValue()); } + else if (typeid_cast(&column_const->getDataColumn())) + { + const auto & date_lut = DateLUT::serverTimezoneInstance(); + ttl_info.update(date_lut.fromDayNum(ExtendedDayNum(column_const->getValue()))); + } + else if (const ColumnDateTime64 * column_dt64 = typeid_cast(&column_const->getDataColumn())) + { + ttl_info.update(column_const->getValue() / intExp10OfSize(column_dt64->getScale())); + } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of result TTL column"); } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index d674f0546325..e0f16e0a945d 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -15,7 +15,9 @@ #include #include +#include #include +#include #include #include #include @@ -81,10 +83,12 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); if (!typeid_cast(result_column.type.get()) - && !typeid_cast(result_column.type.get())) + && !typeid_cast(result_column.type.get()) + && !typeid_cast(result_column.type.get()) + && !typeid_cast(result_column.type.get())) { throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, - "TTL expression result column should have DateTime or Date type, but has {}", + "TTL expression result column should have Date, Date32, DateTime or DateTime64 type, but has {}", result_column.type->getName()); } } diff --git a/tests/queries/0_stateless/03519_ttl_extended_data_types.reference b/tests/queries/0_stateless/03519_ttl_extended_data_types.reference new file mode 100644 index 000000000000..d098f930edc6 --- /dev/null +++ b/tests/queries/0_stateless/03519_ttl_extended_data_types.reference @@ -0,0 +1,8 @@ +"2170-01-01",2170 +"2170-01-01 12:12:12.12345",2170 +"1901-01-01","" +"2010-01-01","" +"2170-01-01","uio" +"1901-01-01 12:12:12.12345","" +"2010-01-01 12:12:12.00000","" +"2170-01-01 12:12:12.12345","uio" diff --git a/tests/queries/0_stateless/03519_ttl_extended_data_types.sql b/tests/queries/0_stateless/03519_ttl_extended_data_types.sql new file mode 100644 index 000000000000..e7c7dcebe48b --- /dev/null +++ b/tests/queries/0_stateless/03519_ttl_extended_data_types.sql @@ -0,0 +1,39 @@ +-- Row TTL with extended data types +DROP TABLE IF EXISTS ttl_03519_1 SYNC; +CREATE TABLE ttl_03519_1 (date Date32, date_key Int) ENGINE=MergeTree TTL date + INTERVAL 1 MONTH ORDER BY date; +INSERT INTO ttl_03519_1 VALUES ('2010-01-01', 2010); +INSERT INTO ttl_03519_1 VALUES ('1901-01-01', 1901); +INSERT INTO ttl_03519_1 VALUES ('2170-01-01', 2170); +OPTIMIZE TABLE ttl_03519_1 FINAL; +SELECT * FROM ttl_03519_1 ORDER BY date FORMAT CSV; +DROP TABLE ttl_03519_1 SYNC; + +DROP TABLE IF EXISTS ttl_03519_2 SYNC; +CREATE TABLE ttl_03519_2 (date DateTime64(5, 'UTC'), date_key Int) ENGINE=MergeTree TTL date + INTERVAL 1 MONTH ORDER BY date; +INSERT INTO ttl_03519_2 VALUES ('2010-01-01 12:12:12.12345', 2010); +INSERT INTO ttl_03519_2 VALUES ('1901-01-01 12:12:12.12345', 1901); +INSERT INTO ttl_03519_2 VALUES ('2170-01-01 12:12:12.12345', 2170); +OPTIMIZE TABLE ttl_03519_2 FINAL; +SELECT * FROM ttl_03519_2 ORDER BY date FORMAT CSV; +DROP TABLE ttl_03519_2 SYNC; + +-- Column TTL with extended data types + +DROP TABLE IF EXISTS ttl_03519_3 SYNC; +CREATE TABLE ttl_03519_3 (date Date32, str String TTL date + INTERVAL 1 MONTH) ENGINE=MergeTree ORDER BY date; +INSERT INTO ttl_03519_3 VALUES ('2010-01-01', 'qwe'); +INSERT INTO ttl_03519_3 VALUES ('1901-01-01', 'rty'); +INSERT INTO ttl_03519_3 VALUES ('2170-01-01', 'uio'); +OPTIMIZE TABLE ttl_03519_3 FINAL; +SELECT * FROM ttl_03519_3 ORDER BY date FORMAT CSV; +DROP TABLE ttl_03519_3 SYNC; + +DROP TABLE IF EXISTS ttl_03519_4 SYNC; +CREATE TABLE ttl_03519_4 (date DateTime64(5, 'UTC'), str String TTL date + INTERVAL 1 MONTH) ENGINE=MergeTree ORDER BY date; +INSERT INTO ttl_03519_4 VALUES ('2010-01-01 12:12:12', 'qwe'); +INSERT INTO ttl_03519_4 VALUES ('1901-01-01 12:12:12.12345', 'rty'); +INSERT INTO ttl_03519_4 VALUES ('2170-01-01 12:12:12.12345', 'uio'); +OPTIMIZE TABLE ttl_03519_4 FINAL; +SELECT * FROM ttl_03519_4 ORDER BY date FORMAT CSV; +DROP TABLE ttl_03519_4 SYNC; +