make tests with optimize final more stable, wait for merges in flight

This commit is contained in:
Sema Checherinda 2023-01-10 11:57:34 +01:00
parent 8f660afab3
commit 8218d59103
19 changed files with 456 additions and 348 deletions

View File

@ -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

View 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;"

View File

@ -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;

View 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;"

View File

@ -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;

View 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;"

View File

@ -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;

View File

@ -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);

View 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;"

View File

@ -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;

View 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;"

View File

@ -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;

View 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;"

View File

@ -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;

View File

@ -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 "

View 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;"

View File

@ -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;

View File

@ -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;"

View 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
}