未验证 提交 9fe987c8 编写于 作者: A Anton Popov 提交者: GitHub

Merge pull request #7920 from excitoon-favorites/datetimettl

Removed check for using `Date` or `DateTime` column from TTL expressions
......@@ -203,8 +203,15 @@ UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t i
return date_lut.fromDayNum(DayNum(column_date->getData()[ind]));
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
return column_date_time->getData()[ind];
else
throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR);
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
{
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
return date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>()));
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
return column_const->getValue<UInt32>();
}
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
}
}
......@@ -544,19 +544,6 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin
}
}
bool has_date_column = false;
for (const auto & elem : ttl_expression->getRequiredColumnsWithTypes())
{
if (typeid_cast<const DataTypeDateTime *>(elem.type.get()) || typeid_cast<const DataTypeDate *>(elem.type.get()))
{
has_date_column = true;
break;
}
}
if (!has_date_column)
throw Exception("TTL expression should use at least one Date or DateTime column", ErrorCodes::BAD_TTL_EXPRESSION);
const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name);
if (!typeid_cast<const DataTypeDateTime *>(result_column.type.get())
......
......@@ -96,8 +96,22 @@ void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, MergeTreeDataPart::TTL
for (const auto & val : column_date_time->getData())
ttl_info.update(val);
}
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
{
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
{
const auto & date_lut = DateLUT::instance();
ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>())));
}
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
{
ttl_info.update(column_const->getValue<UInt32>());
}
else
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
}
else
throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
}
......
......@@ -6,3 +6,11 @@
2000-10-10 00:00:00 0
2100-10-10 00:00:00 3
2100-10-10 2
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() - 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
1 0
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() + 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
1 1
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() - 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
1 0
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() + 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
1 1
......@@ -47,6 +47,42 @@ select sleep(0.7) format Null; -- wait if very fast merge happen
optimize table ttl_00933_1 final;
select * from ttl_00933_1 order by d;
-- const DateTime TTL positive
drop table if exists ttl_00933_1;
create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple();
show create table ttl_00933_1;
insert into ttl_00933_1 values (1, 1);
select sleep(0.7) format Null; -- wait if very fast merge happen
optimize table ttl_00933_1 final;
select * from ttl_00933_1;
-- const DateTime TTL negative
drop table if exists ttl_00933_1;
create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple();
show create table ttl_00933_1;
insert into ttl_00933_1 values (1, 1);
select sleep(0.7) format Null; -- wait if very fast merge happen
optimize table ttl_00933_1 final;
select * from ttl_00933_1;
-- const Date TTL positive
drop table if exists ttl_00933_1;
create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple();
show create table ttl_00933_1;
insert into ttl_00933_1 values (1, 1);
select sleep(0.7) format Null; -- wait if very fast merge happen
optimize table ttl_00933_1 final;
select * from ttl_00933_1;
-- const Date TTL negative
drop table if exists ttl_00933_1;
create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple();
show create table ttl_00933_1;
insert into ttl_00933_1 values (1, 1);
select sleep(0.7) format Null; -- wait if very fast merge happen
optimize table ttl_00933_1 final;
select * from ttl_00933_1;
set send_logs_level = 'none';
drop table if exists ttl_00933_1;
......@@ -54,7 +90,6 @@ drop table if exists ttl_00933_1;
create table ttl_00933_1 (d DateTime ttl d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 44}
create table ttl_00933_1 (d DateTime, a Int ttl d) engine = MergeTree order by a partition by toSecond(d); -- { serverError 44}
create table ttl_00933_1 (d DateTime, a Int ttl 2 + 2) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
create table ttl_00933_1 (d DateTime, a Int ttl toDateTime(1)) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
create table ttl_00933_1 (d DateTime, a Int ttl d - d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 day) engine = Log; -- { serverError 36 }
......
......@@ -72,7 +72,7 @@ For a description of parameters, see the [CREATE query description](../../query_
- `TTL` — An expression for setting storage time for rows.
It must depend on the `Date` or `DateTime` column and have one `Date` or `DateTime` column as a result. Example:
It must have one `Date` or `DateTime` column as a result. Example:
`TTL date + INTERVAL 1 DAY`
For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl)
......@@ -373,7 +373,7 @@ Determines the lifetime of values.
The `TTL` clause can be set for the whole table and for each individual column. If both `TTL` are set, ClickHouse uses that `TTL` which expires earlier.
The table must have the column in the [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type. To define the lifetime of data, use operations on this time column, for example:
To define the lifetime of data, use expression evaluating to [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type, for example:
```sql
TTL time_column
......
......@@ -72,7 +72,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
- `TTL` — выражение, определяющее длительность хранения строк.
Должно зависеть от столбца `Date` или `DateTime` и возвращать столбец `Date` или `DateTime`. Пример:`TTL date + INTERVAL 1 DAY`
Должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`.
Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl)
......@@ -365,7 +365,7 @@ hasToken | ✗ | ✗ | ✗ | ✔ | ✗
Секция `TTL` может быть установлена как для всей таблицы, так и для каждого отдельного столбца. Если установлены оба `TTL`, то ClickHouse использует тот, что истекает раньше.
Таблица должна иметь столбец типа [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md). Для установки времени жизни данных, следует использовать операцию со столбцом с временем, например:
Для установки времени жизни данных, следует использовать выражение, возвращающее тип [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md), например:
```sql
TTL time_column
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册