mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Allowed to use constant TTL expressions, updated tests.
This commit is contained in:
parent
fa79b7cf7d
commit
ecaa98e4e4
@ -203,8 +203,16 @@ 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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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,14 +47,41 @@ 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 (a Int ttl toDateTime(1)) engine = MergeTree order by tuple() partition by toSecond(d);
|
||||
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);
|
||||
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 order by d;
|
||||
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';
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user