Skip to content

Commit fa08419

Browse files
authored
Merge pull request #833 from Altinity/backports/24.8.14/80710
24.8.14 Backport of ClickHouse#80710 -- Add support for Date32, DateTime64 in TTL
2 parents 42d1005 + 0cfc7e6 commit fa08419

File tree

12 files changed

+100
-16
lines changed

12 files changed

+100
-16
lines changed

.github/workflows/release_branches.yml

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -544,7 +544,7 @@ jobs:
544544
secrets: inherit
545545
with:
546546
runner_type: altinity-on-demand, altinity-type-cpx51, altinity-image-x86-app-docker-ce, altinity-setup-regression
547-
commit: f9e29772f4d261b82c23189d89038eb7ba027865
547+
commit: e3c00be97a045aa04e9d1a6ec50cc64f4c387b70
548548
arch: release
549549
build_sha: ${{ github.event_name == 'pull_request' && github.event.pull_request.head.sha || github.sha }}
550550
timeout_minutes: 300
@@ -555,7 +555,7 @@ jobs:
555555
secrets: inherit
556556
with:
557557
runner_type: altinity-on-demand, altinity-type-cax41, altinity-image-arm-app-docker-ce, altinity-setup-regression
558-
commit: f9e29772f4d261b82c23189d89038eb7ba027865
558+
commit: e3c00be97a045aa04e9d1a6ec50cc64f4c387b70
559559
arch: aarch64
560560
build_sha: ${{ github.event_name == 'pull_request' && github.event.pull_request.head.sha || github.sha }}
561561
timeout_minutes: 300

docs/en/engines/table-engines/mergetree-family/mergetree.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -519,7 +519,7 @@ Determines the lifetime of values.
519519

520520
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.
521521

522-
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.
522+
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.
523523

524524
**Syntax**
525525

src/Processors/TTL/ITTLAlgorithm.cpp

Lines changed: 16 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,8 @@
22
#include <Columns/ColumnVector.h>
33
#include <Columns/ColumnConst.h>
44

5+
#include <Columns/ColumnsDateTime.h>
6+
57
namespace DB
68
{
79

@@ -46,18 +48,26 @@ ColumnPtr ITTLAlgorithm::executeExpressionAndGetColumn(
4648
return block_copy.getByName(result_column).column;
4749
}
4850

49-
UInt32 ITTLAlgorithm::getTimestampByIndex(const IColumn * column, size_t index) const
51+
Int64 ITTLAlgorithm::getTimestampByIndex(const IColumn * column, size_t index) const
5052
{
5153
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
52-
return static_cast<UInt32>(date_lut.fromDayNum(DayNum(column_date->getData()[index])));
53-
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
54+
return date_lut.fromDayNum(DayNum(column_date->getData()[index]));
55+
if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
5456
return column_date_time->getData()[index];
55-
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
57+
if (const ColumnInt32 * column_date_32 = typeid_cast<const ColumnInt32 *>(column))
58+
return date_lut.fromDayNum(ExtendedDayNum(column_date_32->getData()[index]));
59+
if (const ColumnDateTime64 * column_date_time_64 = typeid_cast<const ColumnDateTime64 *>(column))
60+
return column_date_time_64->getData()[index] / intExp10OfSize<Int64>(column_date_time_64->getScale());
61+
if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
5662
{
5763
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
58-
return static_cast<UInt32>(date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>())));
59-
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
64+
return date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>()));
65+
if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
6066
return column_const->getValue<UInt32>();
67+
if (typeid_cast<const ColumnInt32 *>(&column_const->getDataColumn()))
68+
return date_lut.fromDayNum(ExtendedDayNum(column_const->getValue<Int32>()));
69+
if (const ColumnDateTime64 * column_dt64 = typeid_cast<const ColumnDateTime64 *>(&column_const->getDataColumn()))
70+
return column_const->getValue<DateTime64>() / intExp10OfSize<Int64>(column_dt64->getScale());
6171
}
6272

6373
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of result TTL column");

src/Processors/TTL/ITTLAlgorithm.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ class ITTLAlgorithm
4343

4444
protected:
4545
bool isTTLExpired(time_t ttl) const;
46-
UInt32 getTimestampByIndex(const IColumn * column, size_t index) const;
46+
Int64 getTimestampByIndex(const IColumn * column, size_t index) const;
4747

4848
const TTLExpressions ttl_expressions;
4949
const TTLDescription description;

src/Processors/TTL/TTLAggregationAlgorithm.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -86,7 +86,7 @@ void TTLAggregationAlgorithm::execute(Block & block)
8686

8787
for (size_t i = 0; i < block.rows(); ++i)
8888
{
89-
UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
89+
Int64 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
9090
bool where_filter_passed = !where_column || where_column->getBool(i);
9191
bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed;
9292

src/Processors/TTL/TTLColumnAlgorithm.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ void TTLColumnAlgorithm::execute(Block & block)
5959

6060
for (size_t i = 0; i < block.rows(); ++i)
6161
{
62-
UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
62+
Int64 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
6363
if (isTTLExpired(cur_ttl))
6464
{
6565
if (default_column)

src/Processors/TTL/TTLDeleteAlgorithm.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ void TTLDeleteAlgorithm::execute(Block & block)
3434

3535
for (size_t i = 0; i < block.rows(); ++i)
3636
{
37-
UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
37+
Int64 cur_ttl = getTimestampByIndex(ttl_column.get(), i);
3838
bool where_filter_passed = !where_column || where_column->getBool(i);
3939

4040
if (!isTTLExpired(cur_ttl) || !where_filter_passed)

src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ void TTLUpdateInfoAlgorithm::execute(Block & block)
2525
auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column);
2626
for (size_t i = 0; i < block.rows(); ++i)
2727
{
28-
UInt32 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i);
28+
Int64 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i);
2929
new_ttl_info.update(cur_ttl);
3030
}
3131
}

src/Storages/MergeTree/MergeTreeDataWriter.cpp

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,7 @@
11
#include <Columns/ColumnConst.h>
2+
#include <Columns/ColumnsDateTime.h>
3+
#include <Common/DateLUTImpl.h>
4+
#include <Common/intExp.h>
25
#include <DataTypes/DataTypeDate.h>
36
#include <DataTypes/DataTypeDateTime.h>
47
#include <DataTypes/ObjectUtils.h>
@@ -159,6 +162,17 @@ void updateTTL(
159162
for (const auto & val : column_date_time->getData())
160163
ttl_info.update(val);
161164
}
165+
else if (const ColumnInt32 * column_date_32 = typeid_cast<const ColumnInt32 *>(ttl_column.get()))
166+
{
167+
const auto & date_lut = DateLUT::serverTimezoneInstance();
168+
for (const auto & val : column_date_32->getData())
169+
ttl_info.update(date_lut.fromDayNum(ExtendedDayNum(val)));
170+
}
171+
else if (const ColumnDateTime64 * column_date_time_64 = typeid_cast<const ColumnDateTime64 *>(ttl_column.get()))
172+
{
173+
for (const auto & val : column_date_time_64->getData())
174+
ttl_info.update(val / intExp10OfSize<Int64>(column_date_time_64->getScale()));
175+
}
162176
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(ttl_column.get()))
163177
{
164178
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
@@ -170,6 +184,15 @@ void updateTTL(
170184
{
171185
ttl_info.update(column_const->getValue<UInt32>());
172186
}
187+
else if (typeid_cast<const ColumnInt32 *>(&column_const->getDataColumn()))
188+
{
189+
const auto & date_lut = DateLUT::serverTimezoneInstance();
190+
ttl_info.update(date_lut.fromDayNum(ExtendedDayNum(column_const->getValue<Int32>())));
191+
}
192+
else if (const ColumnDateTime64 * column_dt64 = typeid_cast<const ColumnDateTime64 *>(&column_const->getDataColumn()))
193+
{
194+
ttl_info.update(column_const->getValue<DateTime64>() / intExp10OfSize<Int64>(column_dt64->getScale()));
195+
}
173196
else
174197
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of result TTL column");
175198
}

src/Storages/TTLDescription.cpp

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,9 @@
1515
#include <Interpreters/Context.h>
1616

1717
#include <DataTypes/DataTypeDate.h>
18+
#include <DataTypes/DataTypeDate32.h>
1819
#include <DataTypes/DataTypeDateTime.h>
20+
#include <DataTypes/DataTypeDateTime64.h>
1921
#include <Interpreters/FunctionNameNormalizer.h>
2022
#include <Parsers/ExpressionListParsers.h>
2123
#include <Parsers/parseQuery.h>
@@ -81,10 +83,12 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin
8183

8284
const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name);
8385
if (!typeid_cast<const DataTypeDateTime *>(result_column.type.get())
84-
&& !typeid_cast<const DataTypeDate *>(result_column.type.get()))
86+
&& !typeid_cast<const DataTypeDate *>(result_column.type.get())
87+
&& !typeid_cast<const DataTypeDateTime64 *>(result_column.type.get())
88+
&& !typeid_cast<const DataTypeDate32 *>(result_column.type.get()))
8589
{
8690
throw Exception(ErrorCodes::BAD_TTL_EXPRESSION,
87-
"TTL expression result column should have DateTime or Date type, but has {}",
91+
"TTL expression result column should have Date, Date32, DateTime or DateTime64 type, but has {}",
8892
result_column.type->getName());
8993
}
9094
}

0 commit comments

Comments
 (0)