mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-14 03:25:15 +00:00
make tests with optimize final more stable, wait for merges in flight
This commit is contained in:
parent
8f660afab3
commit
8218d59103
@ -14,3 +14,9 @@ CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() -
|
||||
1 0
|
||||
CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() + 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
|
||||
1 1
|
||||
ILLEGAL_COLUMN
|
||||
ILLEGAL_COLUMN
|
||||
BAD_TTL_EXPRESSION
|
||||
BAD_TTL_EXPRESSION
|
||||
BAD_ARGUMENTS
|
||||
BAD_ARGUMENTS
|
||||
|
136
tests/queries/0_stateless/00933_ttl_simple.sh
Executable file
136
tests/queries/0_stateless/00933_ttl_simple.sh
Executable file
@ -0,0 +1,136 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./merges.lib
|
||||
. "$CURDIR"/merges.lib
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
# Column TTL works only with wide parts, because it's very expensive to apply it for compact parts
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 second, b Int ttl d + interval 1 second) engine = MergeTree order by tuple() partition by toMinute(d) settings min_bytes_for_wide_part = 0;
|
||||
insert into ttl_00933_1 values (now(), 1, 2);
|
||||
insert into ttl_00933_1 values (now(), 3, 4);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select a, b from ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (d DateTime, a Int, b Int)
|
||||
engine = MergeTree order by toDate(d) partition by tuple() ttl d + interval 1 second
|
||||
settings remove_empty_parts = 0;
|
||||
insert into ttl_00933_1 values (now(), 1, 2);
|
||||
insert into ttl_00933_1 values (now(), 3, 4);
|
||||
insert into ttl_00933_1 values (now() + 1000, 5, 6);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;" # check ttl merge for part with both expired and unexpired values
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select a, b from ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d) settings min_bytes_for_wide_part = 0;
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1);
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2);
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 3);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl_00933_1 order by d;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (d DateTime, a Int)
|
||||
engine = MergeTree order by tuple() partition by tuple() ttl d + interval 1 day
|
||||
settings remove_empty_parts = 0;
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1);
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2);
|
||||
insert into ttl_00933_1 values (toDateTime('2100-10-10 00:00:00'), 3);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl_00933_1 order by d;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (d Date, a Int)
|
||||
engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day
|
||||
settings remove_empty_parts = 0;
|
||||
insert into ttl_00933_1 values (toDate('2000-10-10'), 1);
|
||||
insert into ttl_00933_1 values (toDate('2100-10-10'), 2);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl_00933_1 order by d;"
|
||||
|
||||
# const DateTime TTL positive
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl_00933_1;"
|
||||
|
||||
# const DateTime TTL negative
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl_00933_1;"
|
||||
|
||||
# const Date TTL positive
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl_00933_1;"
|
||||
|
||||
#const Date TTL negative
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl_00933_1 final;"
|
||||
wait_for_merges_done ttl_00933_1
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "set send_logs_level = 'fatal';"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "create table ttl_00933_1 (d DateTime ttl d) engine = MergeTree order by tuple() partition by toSecond(d);" 2>&1 | grep -o ILLEGAL_COLUMN | uniq || echo "expected error ILLEGAL_COLUMN"
|
||||
$CLICKHOUSE_CLIENT --query "create table ttl_00933_1 (d DateTime, a Int ttl d) engine = MergeTree order by a partition by toSecond(d);" 2>&1 | grep -o ILLEGAL_COLUMN | uniq || echo "expected error ILLEGAL_COLUMN"
|
||||
$CLICKHOUSE_CLIENT --query "create table ttl_00933_1 (d DateTime, a Int ttl 2 + 2) engine = MergeTree order by tuple() partition by toSecond(d);" 2>&1 | grep -o BAD_TTL_EXPRESSION | uniq || echo "expected error BAD_TTL_EXPRESSION"
|
||||
$CLICKHOUSE_CLIENT --query "create table ttl_00933_1 (d DateTime, a Int ttl d - d) engine = MergeTree order by tuple() partition by toSecond(d);" 2>&1 | grep -o BAD_TTL_EXPRESSION | uniq || echo "expected error BAD_TTL_EXPRESSION"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 day) engine = Log;" 2>&1 | grep -o BAD_ARGUMENTS | uniq || echo "expected error BAD_ARGUMENTS"
|
||||
$CLICKHOUSE_CLIENT --query "create table ttl_00933_1 (d DateTime, a Int) engine = Log ttl d + interval 1 day;" 2>&1 | grep -o BAD_ARGUMENTS | uniq || echo "expected error BAD_ARGUMENTS"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl_00933_1;"
|
@ -1,98 +0,0 @@
|
||||
drop table if exists ttl_00933_1;
|
||||
|
||||
-- Column TTL works only with wide parts, because it's very expensive to apply it for compact parts
|
||||
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 second, b Int ttl d + interval 1 second) engine = MergeTree order by tuple() partition by toMinute(d) settings min_bytes_for_wide_part = 0;
|
||||
insert into ttl_00933_1 values (now(), 1, 2);
|
||||
insert into ttl_00933_1 values (now(), 3, 4);
|
||||
select sleep(1.1) format Null;
|
||||
optimize table ttl_00933_1 final;
|
||||
select a, b from ttl_00933_1;
|
||||
|
||||
drop table if exists ttl_00933_1;
|
||||
|
||||
create table ttl_00933_1 (d DateTime, a Int, b Int)
|
||||
engine = MergeTree order by toDate(d) partition by tuple() ttl d + interval 1 second
|
||||
settings remove_empty_parts = 0;
|
||||
insert into ttl_00933_1 values (now(), 1, 2);
|
||||
insert into ttl_00933_1 values (now(), 3, 4);
|
||||
insert into ttl_00933_1 values (now() + 1000, 5, 6);
|
||||
select sleep(1.1) format Null;
|
||||
optimize table ttl_00933_1 final; -- check ttl merge for part with both expired and unexpired values
|
||||
select a, b from ttl_00933_1;
|
||||
|
||||
drop table if exists ttl_00933_1;
|
||||
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d) settings min_bytes_for_wide_part = 0;
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1);
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2);
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 3);
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1 order by d;
|
||||
|
||||
drop table if exists ttl_00933_1;
|
||||
|
||||
create table ttl_00933_1 (d DateTime, a Int)
|
||||
engine = MergeTree order by tuple() partition by tuple() ttl d + interval 1 day
|
||||
settings remove_empty_parts = 0;
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 1);
|
||||
insert into ttl_00933_1 values (toDateTime('2000-10-10 00:00:00'), 2);
|
||||
insert into ttl_00933_1 values (toDateTime('2100-10-10 00:00:00'), 3);
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1 order by d;
|
||||
|
||||
drop table if exists ttl_00933_1;
|
||||
|
||||
create table ttl_00933_1 (d Date, a Int)
|
||||
engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day
|
||||
settings remove_empty_parts = 0;
|
||||
insert into ttl_00933_1 values (toDate('2000-10-10'), 1);
|
||||
insert into ttl_00933_1 values (toDate('2100-10-10'), 2);
|
||||
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() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
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() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
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() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
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() settings min_bytes_for_wide_part = 0;
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1;
|
||||
|
||||
set send_logs_level = 'fatal';
|
||||
|
||||
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 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 }
|
||||
create table ttl_00933_1 (d DateTime, a Int) engine = Log ttl d + interval 1 day; -- { serverError 36 }
|
||||
|
||||
drop table if exists ttl_00933_1;
|
29
tests/queries/0_stateless/00976_system_stop_ttl_merges.sh
Executable file
29
tests/queries/0_stateless/00976_system_stop_ttl_merges.sh
Executable file
@ -0,0 +1,29 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./merges.lib
|
||||
. "$CURDIR"/merges.lib
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "system stop ttl merges ttl;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1), (toDateTime('2000-10-10 00:00:00'), 2);"
|
||||
$CLICKHOUSE_CLIENT --query "insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3), (toDateTime('2100-10-10 00:00:00'), 4);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl partition 10 final;"
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl order by d, a;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "system start ttl merges ttl;"
|
||||
$CLICKHOUSE_CLIENT --query "optimize table ttl partition 10 final;"
|
||||
|
||||
wait_for_merges_done ttl
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "select * from ttl order by d, a;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "drop table if exists ttl;"
|
@ -1,18 +0,0 @@
|
||||
drop table if exists ttl;
|
||||
|
||||
create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day;
|
||||
|
||||
system stop ttl merges ttl;
|
||||
|
||||
insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1), (toDateTime('2000-10-10 00:00:00'), 2)
|
||||
insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3), (toDateTime('2100-10-10 00:00:00'), 4);
|
||||
|
||||
select sleep(1) format Null; -- wait if very fast merge happen
|
||||
optimize table ttl partition 10 final;
|
||||
select * from ttl order by d, a;
|
||||
|
||||
system start ttl merges ttl;
|
||||
optimize table ttl partition 10 final;
|
||||
select * from ttl order by d, a;
|
||||
|
||||
drop table if exists ttl;
|
19
tests/queries/0_stateless/01282_system_parts_ttl_info.sh
Executable file
19
tests/queries/0_stateless/01282_system_parts_ttl_info.sh
Executable file
@ -0,0 +1,19 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./merges.lib
|
||||
. "$CURDIR"/merges.lib
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl;"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE ttl (d DateTime) ENGINE = MergeTree ORDER BY tuple() TTL d + INTERVAL 10 DAY SETTINGS remove_empty_parts=0;"
|
||||
$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES ttl;"
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO ttl VALUES ('2000-01-01 01:02:03'), ('2000-02-03 04:05:06');"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT rows, delete_ttl_info_min, delete_ttl_info_max, move_ttl_info.expression, move_ttl_info.min, move_ttl_info.max FROM system.parts WHERE database = currentDatabase() AND table = 'ttl';"
|
||||
$CLICKHOUSE_CLIENT --query "SYSTEM START MERGES ttl;"
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl FINAL;"
|
||||
wait_for_merges_done ttl
|
||||
$CLICKHOUSE_CLIENT --query "SELECT rows, toTimeZone(delete_ttl_info_min, 'UTC'), toTimeZone(delete_ttl_info_max, 'UTC'), move_ttl_info.expression, move_ttl_info.min, move_ttl_info.max FROM system.parts WHERE database = currentDatabase() AND table = 'ttl' AND active;"
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE ttl;"
|
@ -1,9 +0,0 @@
|
||||
DROP TABLE IF EXISTS ttl;
|
||||
CREATE TABLE ttl (d DateTime) ENGINE = MergeTree ORDER BY tuple() TTL d + INTERVAL 10 DAY SETTINGS remove_empty_parts=0;
|
||||
SYSTEM STOP MERGES ttl;
|
||||
INSERT INTO ttl VALUES ('2000-01-01 01:02:03'), ('2000-02-03 04:05:06');
|
||||
SELECT rows, delete_ttl_info_min, delete_ttl_info_max, move_ttl_info.expression, move_ttl_info.min, move_ttl_info.max FROM system.parts WHERE database = currentDatabase() AND table = 'ttl';
|
||||
SYSTEM START MERGES ttl;
|
||||
OPTIMIZE TABLE ttl FINAL;
|
||||
SELECT rows, toTimeZone(delete_ttl_info_min, 'UTC'), toTimeZone(delete_ttl_info_max, 'UTC'), move_ttl_info.expression, move_ttl_info.min, move_ttl_info.max FROM system.parts WHERE database = currentDatabase() AND table = 'ttl' AND active;
|
||||
DROP TABLE ttl;
|
@ -1,6 +1,6 @@
|
||||
DROP TABLE IF EXISTS mt_01451;
|
||||
|
||||
CREATE TABLE mt_01451 (v UInt8) ENGINE = MergeTree() order by tuple();
|
||||
CREATE TABLE mt_01451 (v UInt8) ENGINE = MergeTree() order by tuple() SETTINGS old_parts_lifetime=0;
|
||||
SYSTEM STOP MERGES mt_01451;
|
||||
|
||||
INSERT INTO mt_01451 VALUES (0);
|
||||
|
91
tests/queries/0_stateless/01465_ttl_recompression.sh
Executable file
91
tests/queries/0_stateless/01465_ttl_recompression.sh
Executable file
@ -0,0 +1,91 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
set -euo pipefail
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./merges.lib
|
||||
. "$CURDIR"/merges.lib
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS recompression_table;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE recompression_table
|
||||
(
|
||||
dt DateTime,
|
||||
key UInt64,
|
||||
value String
|
||||
|
||||
) ENGINE MergeTree()
|
||||
ORDER BY tuple()
|
||||
PARTITION BY key
|
||||
TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10))
|
||||
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SHOW CREATE TABLE recompression_table;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM STOP TTL MERGES recompression_table;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO recompression_table SELECT now(), 1, toString(number) from numbers(1000);"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO recompression_table SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000);"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO recompression_table SELECT now() - INTERVAL 2 YEAR, 3, toString(number) from numbers(2000, 1000);"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT COUNT() FROM recompression_table;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT substring(name, 1, length(name) - 2), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE recompression_table FINAL;"
|
||||
|
||||
# merge level and mutation in part name is not important
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT substring(name, 1, length(name) - 2), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 DAY RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SHOW CREATE TABLE recompression_table;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT substring(name, 1, length(name) - 4), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM START TTL MERGES recompression_table;"
|
||||
|
||||
# Additional merge can happen here
|
||||
${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE recompression_table FINAL;"
|
||||
wait_for_merges_done recompression_table
|
||||
|
||||
# merge level and mutation in part name is not important
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT substring(name, 1, length(name) - 4), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT substring(name, 1, length(name) - 4), recompression_ttl_info.expression FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS recompression_table;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "CREATE TABLE recompression_table_compact
|
||||
(
|
||||
dt DateTime,
|
||||
key UInt64,
|
||||
value String
|
||||
|
||||
) ENGINE MergeTree()
|
||||
ORDER BY tuple()
|
||||
PARTITION BY key
|
||||
TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10))
|
||||
SETTINGS min_rows_for_wide_part = 10000;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SYSTEM STOP TTL MERGES recompression_table_compact;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO recompression_table_compact SELECT now(), 1, toString(number) from numbers(1000);"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO recompression_table_compact SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000);"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "INSERT INTO recompression_table_compact SELECT now() - INTERVAL 2 YEAR, 3, toString(number) from numbers(2000, 1000);"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT substring(name, 1, length(name) - 2), default_compression_codec FROM system.parts WHERE table = 'recompression_table_compact' and active = 1 and database = currentDatabase() ORDER BY name;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "ALTER TABLE recompression_table_compact MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; -- mutation affect all columns, so codec changes"
|
||||
|
||||
# merge level and mutation in part name is not important
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT substring(name, 1, length(name) - 4), default_compression_codec FROM system.parts WHERE table = 'recompression_table_compact' and active = 1 and database = currentDatabase() ORDER BY name;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "DROP TABLE recompression_table_compact;"
|
@ -1,80 +0,0 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP TABLE IF EXISTS recompression_table;
|
||||
|
||||
CREATE TABLE recompression_table
|
||||
(
|
||||
dt DateTime,
|
||||
key UInt64,
|
||||
value String
|
||||
|
||||
) ENGINE MergeTree()
|
||||
ORDER BY tuple()
|
||||
PARTITION BY key
|
||||
TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10))
|
||||
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0;
|
||||
|
||||
SHOW CREATE TABLE recompression_table;
|
||||
|
||||
SYSTEM STOP TTL MERGES recompression_table;
|
||||
|
||||
INSERT INTO recompression_table SELECT now(), 1, toString(number) from numbers(1000);
|
||||
|
||||
INSERT INTO recompression_table SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000);
|
||||
|
||||
INSERT INTO recompression_table SELECT now() - INTERVAL 2 YEAR, 3, toString(number) from numbers(2000, 1000);
|
||||
|
||||
SELECT COUNT() FROM recompression_table;
|
||||
|
||||
SELECT substring(name, 1, length(name) - 2), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;
|
||||
|
||||
OPTIMIZE TABLE recompression_table FINAL;
|
||||
|
||||
-- merge level and mutation in part name is not important
|
||||
SELECT substring(name, 1, length(name) - 2), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;
|
||||
|
||||
ALTER TABLE recompression_table MODIFY TTL dt + INTERVAL 1 DAY RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2;
|
||||
|
||||
SHOW CREATE TABLE recompression_table;
|
||||
|
||||
SELECT substring(name, 1, length(name) - 4), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;
|
||||
|
||||
SYSTEM START TTL MERGES recompression_table;
|
||||
-- Additional merge can happen here
|
||||
OPTIMIZE TABLE recompression_table FINAL;
|
||||
|
||||
-- merge level and mutation in part name is not important
|
||||
SELECT substring(name, 1, length(name) - 4), default_compression_codec FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;
|
||||
|
||||
SELECT substring(name, 1, length(name) - 4), recompression_ttl_info.expression FROM system.parts WHERE table = 'recompression_table' and active = 1 and database = currentDatabase() ORDER BY name;
|
||||
|
||||
DROP TABLE IF EXISTS recompression_table;
|
||||
|
||||
CREATE TABLE recompression_table_compact
|
||||
(
|
||||
dt DateTime,
|
||||
key UInt64,
|
||||
value String
|
||||
|
||||
) ENGINE MergeTree()
|
||||
ORDER BY tuple()
|
||||
PARTITION BY key
|
||||
TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(17)), dt + INTERVAL 1 YEAR RECOMPRESS CODEC(LZ4HC(10))
|
||||
SETTINGS min_rows_for_wide_part = 10000;
|
||||
|
||||
SYSTEM STOP TTL MERGES recompression_table_compact;
|
||||
|
||||
INSERT INTO recompression_table_compact SELECT now(), 1, toString(number) from numbers(1000);
|
||||
|
||||
INSERT INTO recompression_table_compact SELECT now() - INTERVAL 2 MONTH, 2, toString(number) from numbers(1000, 1000);
|
||||
|
||||
INSERT INTO recompression_table_compact SELECT now() - INTERVAL 2 YEAR, 3, toString(number) from numbers(2000, 1000);
|
||||
|
||||
SELECT substring(name, 1, length(name) - 2), default_compression_codec FROM system.parts WHERE table = 'recompression_table_compact' and active = 1 and database = currentDatabase() ORDER BY name;
|
||||
|
||||
ALTER TABLE recompression_table_compact MODIFY TTL dt + INTERVAL 1 MONTH RECOMPRESS CODEC(ZSTD(12)) SETTINGS mutations_sync = 2; -- mutation affect all columns, so codec changes
|
||||
|
||||
-- merge level and mutation in part name is not important
|
||||
SELECT substring(name, 1, length(name) - 4), default_compression_codec FROM system.parts WHERE table = 'recompression_table_compact' and active = 1 and database = currentDatabase() ORDER BY name;
|
||||
|
||||
DROP TABLE recompression_table_compact;
|
42
tests/queries/0_stateless/01603_remove_column_ttl.sh
Executable file
42
tests/queries/0_stateless/01603_remove_column_ttl.sh
Executable file
@ -0,0 +1,42 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./merges.lib
|
||||
. "$CURDIR"/merges.lib
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_with_column_ttl;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE table_with_column_ttl
|
||||
(
|
||||
EventTime DateTime,
|
||||
UserID UInt64,
|
||||
Age UInt8 TTL EventTime + INTERVAL 3 MONTH
|
||||
)
|
||||
ENGINE MergeTree()
|
||||
ORDER BY tuple()
|
||||
SETTINGS min_bytes_for_wide_part = 0;" # column TTL doesn't work for compact parts
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_column_ttl VALUES (now(), 1, 32);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_column_ttl VALUES (now() - INTERVAL 4 MONTH, 2, 45);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE table_with_column_ttl FINAL;"
|
||||
|
||||
wait_for_merges_done table_with_column_ttl
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT UserID, Age FROM table_with_column_ttl ORDER BY UserID;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_column_ttl MODIFY COLUMN Age REMOVE TTL;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_with_column_ttl;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_column_ttl VALUES (now() - INTERVAL 10 MONTH, 3, 27);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE table_with_column_ttl FINAL;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT UserID, Age FROM table_with_column_ttl ORDER BY UserID;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE table_with_column_ttl;"
|
@ -1,30 +0,0 @@
|
||||
DROP TABLE IF EXISTS table_with_column_ttl;
|
||||
CREATE TABLE table_with_column_ttl
|
||||
(
|
||||
EventTime DateTime,
|
||||
UserID UInt64,
|
||||
Age UInt8 TTL EventTime + INTERVAL 3 MONTH
|
||||
)
|
||||
ENGINE MergeTree()
|
||||
ORDER BY tuple()
|
||||
SETTINGS min_bytes_for_wide_part = 0; -- column TTL doesn't work for compact parts
|
||||
|
||||
INSERT INTO table_with_column_ttl VALUES (now(), 1, 32);
|
||||
|
||||
INSERT INTO table_with_column_ttl VALUES (now() - INTERVAL 4 MONTH, 2, 45);
|
||||
|
||||
OPTIMIZE TABLE table_with_column_ttl FINAL;
|
||||
|
||||
SELECT UserID, Age FROM table_with_column_ttl ORDER BY UserID;
|
||||
|
||||
ALTER TABLE table_with_column_ttl MODIFY COLUMN Age REMOVE TTL;
|
||||
|
||||
SHOW CREATE TABLE table_with_column_ttl;
|
||||
|
||||
INSERT INTO table_with_column_ttl VALUES (now() - INTERVAL 10 MONTH, 3, 27);
|
||||
|
||||
OPTIMIZE TABLE table_with_column_ttl FINAL;
|
||||
|
||||
SELECT UserID, Age FROM table_with_column_ttl ORDER BY UserID;
|
||||
|
||||
DROP TABLE table_with_column_ttl;
|
64
tests/queries/0_stateless/01622_multiple_ttls.sh
Executable file
64
tests/queries/0_stateless/01622_multiple_ttls.sh
Executable file
@ -0,0 +1,64 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./merges.lib
|
||||
. "$CURDIR"/merges.lib
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT 'TTL WHERE';"
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_where;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query '
|
||||
CREATE TABLE ttl_where
|
||||
(
|
||||
d Date,
|
||||
i UInt32
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
TTL d + toIntervalYear(10) DELETE WHERE i % 3 = 0,
|
||||
d + toIntervalYear(40) DELETE WHERE i % 3 = 1;
|
||||
'
|
||||
|
||||
# This test will fail at 2040-10-10
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
INSERT INTO ttl_where SELECT toDate('2000-10-10'), number FROM numbers(10);
|
||||
INSERT INTO ttl_where SELECT toDate('1970-10-10'), number FROM numbers(10);
|
||||
"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl_where FINAL;"
|
||||
wait_for_merges_done ttl_where
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM ttl_where ORDER BY d, i;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE ttl_where;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT 'TTL GROUP BY';"
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_group_by;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query '
|
||||
CREATE TABLE ttl_group_by
|
||||
(
|
||||
d Date,
|
||||
i UInt32,
|
||||
v UInt64
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY (toStartOfMonth(d), i % 10)
|
||||
TTL d + toIntervalYear(10) GROUP BY toStartOfMonth(d), i % 10 SET d = any(toStartOfMonth(d)), i = any(i % 10), v = sum(v),
|
||||
d + toIntervalYear(40) GROUP BY toStartOfMonth(d) SET d = any(toStartOfMonth(d)), v = sum(v);
|
||||
'
|
||||
|
||||
$CLICKHOUSE_CLIENT --multiquery --query "
|
||||
INSERT INTO ttl_group_by SELECT toDate('2000-10-10'), number, number FROM numbers(100);
|
||||
INSERT INTO ttl_group_by SELECT toDate('1970-10-10'), number, number FROM numbers(100);
|
||||
"
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl_group_by FINAL;"
|
||||
wait_for_merges_done ttl_group_by
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM ttl_group_by ORDER BY d, i;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE ttl_group_by;"
|
@ -1,44 +0,0 @@
|
||||
SELECT 'TTL WHERE';
|
||||
DROP TABLE IF EXISTS ttl_where;
|
||||
|
||||
CREATE TABLE ttl_where
|
||||
(
|
||||
`d` Date,
|
||||
`i` UInt32
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
TTL d + toIntervalYear(10) DELETE WHERE i % 3 = 0,
|
||||
d + toIntervalYear(40) DELETE WHERE i % 3 = 1;
|
||||
|
||||
-- This test will fail at 2040-10-10
|
||||
|
||||
INSERT INTO ttl_where SELECT toDate('2000-10-10'), number FROM numbers(10);
|
||||
INSERT INTO ttl_where SELECT toDate('1970-10-10'), number FROM numbers(10);
|
||||
OPTIMIZE TABLE ttl_where FINAL;
|
||||
|
||||
SELECT * FROM ttl_where ORDER BY d, i;
|
||||
|
||||
DROP TABLE ttl_where;
|
||||
|
||||
SELECT 'TTL GROUP BY';
|
||||
DROP TABLE IF EXISTS ttl_group_by;
|
||||
|
||||
CREATE TABLE ttl_group_by
|
||||
(
|
||||
`d` Date,
|
||||
`i` UInt32,
|
||||
`v` UInt64
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY (toStartOfMonth(d), i % 10)
|
||||
TTL d + toIntervalYear(10) GROUP BY toStartOfMonth(d), i % 10 SET d = any(toStartOfMonth(d)), i = any(i % 10), v = sum(v),
|
||||
d + toIntervalYear(40) GROUP BY toStartOfMonth(d) SET d = any(toStartOfMonth(d)), v = sum(v);
|
||||
|
||||
INSERT INTO ttl_group_by SELECT toDate('2000-10-10'), number, number FROM numbers(100);
|
||||
INSERT INTO ttl_group_by SELECT toDate('1970-10-10'), number, number FROM numbers(100);
|
||||
OPTIMIZE TABLE ttl_group_by FINAL;
|
||||
|
||||
SELECT * FROM ttl_group_by ORDER BY d, i;
|
||||
|
||||
DROP TABLE ttl_group_by;
|
@ -4,6 +4,8 @@ set -euo pipefail
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./parts.lib
|
||||
. "$CURDIR"/parts.lib
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q 'DROP TABLE IF EXISTS table_with_single_pk'
|
||||
|
||||
@ -15,6 +17,7 @@ ${CLICKHOUSE_CLIENT} -q '
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY key
|
||||
SETTINGS old_parts_lifetime=0
|
||||
'
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q 'INSERT INTO table_with_single_pk SELECT number, toString(number % 10) FROM numbers(1000000)'
|
||||
@ -33,42 +36,7 @@ ${CLICKHOUSE_CLIENT} -q "
|
||||
|
||||
# Now let's check RemovePart
|
||||
${CLICKHOUSE_CLIENT} -q 'TRUNCATE TABLE table_with_single_pk'
|
||||
|
||||
# Wait until parts are removed
|
||||
function get_inactive_parts_count() {
|
||||
table_name=$1
|
||||
${CLICKHOUSE_CLIENT} -q "
|
||||
SELECT
|
||||
count()
|
||||
FROM
|
||||
system.parts
|
||||
WHERE
|
||||
table = 'table_with_single_pk'
|
||||
AND
|
||||
active = 0
|
||||
AND
|
||||
database = '${CLICKHOUSE_DATABASE}'
|
||||
"
|
||||
}
|
||||
|
||||
function wait_table_inactive_parts_are_gone() {
|
||||
table_name=$1
|
||||
|
||||
while true
|
||||
do
|
||||
count=$(get_inactive_parts_count $table_name)
|
||||
if [[ count -gt 0 ]]
|
||||
then
|
||||
sleep 1
|
||||
else
|
||||
break
|
||||
fi
|
||||
done
|
||||
}
|
||||
|
||||
export -f get_inactive_parts_count
|
||||
export -f wait_table_inactive_parts_are_gone
|
||||
timeout 60 bash -c 'wait_table_inactive_parts_are_gone table_with_single_pk'
|
||||
wait_for_delete_inactive_parts table_with_single_pk
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q 'SYSTEM FLUSH LOGS;'
|
||||
${CLICKHOUSE_CLIENT} -q "
|
||||
|
39
tests/queries/0_stateless/02403_ttl_column_multiple_times.sh
Executable file
39
tests/queries/0_stateless/02403_ttl_column_multiple_times.sh
Executable file
@ -0,0 +1,39 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./merges.lib
|
||||
. "$CURDIR"/merges.lib
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_table;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "CREATE TABLE ttl_table
|
||||
(
|
||||
EventDate Date,
|
||||
Longitude Float64 TTL EventDate + toIntervalWeek(2)
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY EventDate
|
||||
SETTINGS vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES ttl_table;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "INSERT INTO ttl_table VALUES(toDate('2020-10-01'), 144);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM ttl_table;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SYSTEM START MERGES ttl_table;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl_table FINAL;"
|
||||
|
||||
wait_for_merges_done ttl_table
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM ttl_table;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl_table FINAL;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM ttl_table;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ttl_table;"
|
@ -1,28 +0,0 @@
|
||||
DROP TABLE IF EXISTS ttl_table;
|
||||
|
||||
CREATE TABLE ttl_table
|
||||
(
|
||||
EventDate Date,
|
||||
Longitude Float64 TTL EventDate + toIntervalWeek(2)
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY EventDate
|
||||
SETTINGS vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;
|
||||
|
||||
SYSTEM STOP MERGES ttl_table;
|
||||
|
||||
INSERT INTO ttl_table VALUES(toDate('2020-10-01'), 144);
|
||||
|
||||
SELECT * FROM ttl_table;
|
||||
|
||||
SYSTEM START MERGES ttl_table;
|
||||
|
||||
OPTIMIZE TABLE ttl_table FINAL;
|
||||
|
||||
SELECT * FROM ttl_table;
|
||||
|
||||
OPTIMIZE TABLE ttl_table FINAL;
|
||||
|
||||
SELECT * FROM ttl_table;
|
||||
|
||||
DROP TABLE IF EXISTS ttl_table;
|
@ -4,10 +4,6 @@
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
# shellcheck source=./transactions.lib
|
||||
. "$CURDIR"/transactions.lib
|
||||
# shellcheck source=./parts.lib
|
||||
. "$CURDIR"/parts.lib
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_with_gap;"
|
||||
|
25
tests/queries/0_stateless/merges.lib
Normal file
25
tests/queries/0_stateless/merges.lib
Normal file
@ -0,0 +1,25 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
function merges_count {
|
||||
local table=$1
|
||||
local database=${2:-$CLICKHOUSE_DATABASE}
|
||||
$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.merges WHERE table='$table' AND database='$database'"
|
||||
}
|
||||
|
||||
function wait_for_merges_done {
|
||||
local table=$1
|
||||
local database=${2:-$CLICKHOUSE_DATABASE}
|
||||
local timeout=${3:-20}
|
||||
|
||||
while [[ timeout -gt 0 ]]
|
||||
do
|
||||
res=$(merges_count "$table" "$database")
|
||||
[[ $res -eq 0 ]] && return 0
|
||||
|
||||
sleep 2
|
||||
timeout=$((timeout - 2))
|
||||
done
|
||||
|
||||
echo "Timed out while waiting for merges done!" >&2
|
||||
return 2
|
||||
}
|
Loading…
Reference in New Issue
Block a user