From fa6d88b3b29f9a0e852e009651c32652e0201fad Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 10 Sep 2020 21:43:02 +0300 Subject: [PATCH] fix more tests --- .../test_filesystem_layout/test.py | 2 +- .../configs/config.d/storage_conf.xml | 4 ++++ tests/integration/test_partition/test.py | 2 +- .../configs/config.d/storage_conf.xml | 4 ++++ .../test_replicated_merge_tree_s3/test.py | 22 ++++++++++++++----- tests/integration/test_ttl_replicated/test.py | 10 +++++---- ...46_clear_column_in_partition_zookeeper.sql | 6 ++--- .../0_stateless/00933_ttl_simple.reference | 8 +++---- .../queries/0_stateless/00933_ttl_simple.sql | 15 ++++++++----- 9 files changed, 48 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 83389b3d9bd..777e5ab7b9a 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -19,7 +19,7 @@ def test_file_path_escaping(started_cluster): node.query('CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary') node.query(''' CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32) - ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id`; + ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0; ''') node.query('''INSERT INTO test.`T.a_b,l-e!` VALUES (1);''') node.query('''ALTER TABLE test.`T.a_b,l-e!` FREEZE;''') diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index d097675ca63..343f248c5fb 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -25,4 +25,8 @@ + + + 0 + diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 80fbe947316..2a2bbe205b5 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -178,7 +178,7 @@ def test_attach_check_all_parts(attach_check_all_parts_table): exec_bash('cp -pr {} {}'.format(path_to_detached + '0_3_3_0', path_to_detached + 'deleting_0_7_7_0')) error = instance.client.query_and_get_error("ALTER TABLE test.attach_partition ATTACH PARTITION 0") - assert 0 <= error.find('No columns in part 0_5_5_0') + assert 0 <= error.find('No columns in part 0_5_5_0') or 0 <= error.find('No columns.txt in part 0_5_5_0') parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name") assert TSV(parts) == TSV('1_2_2_0\n1_4_4_0') diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml index b32770095fc..f3b7f959ce9 100644 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml @@ -18,4 +18,8 @@ + + + 0 + diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index a77a69b842b..612b50becf7 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -30,7 +30,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 def random_string(length): @@ -44,7 +45,7 @@ def generate_values(date_str, count, sign=1): return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data]) -def create_table(cluster): +def create_table(cluster, additional_settings=None): create_table_statement = """ CREATE TABLE s3_test ( dt Date, @@ -56,6 +57,9 @@ def create_table(cluster): ORDER BY (dt, id) SETTINGS storage_policy='s3' """ + if additional_settings: + create_table_statement += "," + create_table_statement += additional_settings for node in cluster.instances.values(): node.query(create_table_statement) @@ -72,9 +76,15 @@ def drop_table(cluster): for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): minio.remove_object(cluster.minio_bucket, obj.object_name) - -def test_insert_select_replicated(cluster): - create_table(cluster) +@pytest.mark.parametrize( + "min_rows_for_wide_part,files_per_part", + [ + (0, FILES_OVERHEAD_PER_PART_WIDE), + (8192, FILES_OVERHEAD_PER_PART_COMPACT) + ] +) +def test_insert_select_replicated(cluster, min_rows_for_wide_part, files_per_part): + create_table(cluster, additional_settings="min_rows_for_wide_part={}".format(min_rows_for_wide_part)) all_values = "" for node_idx in range(1, 4): @@ -90,4 +100,4 @@ def test_insert_select_replicated(cluster): assert node.query("SELECT * FROM s3_test order by dt, id FORMAT Values", settings={"select_sequential_consistency": 1}) == all_values minio = cluster.minio_client - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 3 * (FILES_OVERHEAD + FILES_OVERHEAD_PER_PART * 3) + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 3 * (FILES_OVERHEAD + files_per_part * 3) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 0f201f569b3..39d595662d0 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -30,7 +30,7 @@ def drop_table(nodes, table_name): node.query("DROP TABLE IF EXISTS {} NO DELAY".format(table_name)) time.sleep(1) - +# Column TTL works only with wide parts, because it's very expensive to apply it for compact parts def test_ttl_columns(started_cluster): drop_table([node1, node2], "test_ttl") for node in [node1, node2]: @@ -38,7 +38,7 @@ def test_ttl_columns(started_cluster): ''' CREATE TABLE test_ttl(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0; + ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0, min_bytes_for_wide_part=0; '''.format(replica=node.name)) node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-10 00:00:00'), 1, 1, 3)") @@ -59,7 +59,8 @@ def test_merge_with_ttl_timeout(started_cluster): ''' CREATE TABLE {table}(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{table}', '{replica}') - ORDER BY id PARTITION BY toDayOfMonth(date); + ORDER BY id PARTITION BY toDayOfMonth(date) + SETTINGS min_bytes_for_wide_part=0; '''.format(replica=node.name, table=table)) node1.query("SYSTEM STOP TTL MERGES {table}".format(table=table)) @@ -198,7 +199,7 @@ def test_ttl_double_delete_rule_returns_error(started_cluster): CREATE TABLE test_ttl(date DateTime, id UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) - TTL date + INTERVAL 1 DAY, date + INTERVAL 2 DAY SETTINGS merge_with_ttl_timeout=0; + TTL date + INTERVAL 1 DAY, date + INTERVAL 2 DAY SETTINGS merge_with_ttl_timeout=0 '''.format(replica=node1.name)) assert False except client.QueryRuntimeException: @@ -246,6 +247,7 @@ limitations under the License.""" ) ENGINE = {engine} ORDER BY tuple() TTL d1 + INTERVAL 1 DAY DELETE + SETTINGS min_bytes_for_wide_part=0 """.format(name=name, engine=engine)) node1.query("""ALTER TABLE {name} MODIFY COLUMN s1 String TTL d1 + INTERVAL 1 SECOND""".format(name=name)) diff --git a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index e6de5a91ce3..bd6c12ffce4 100644 --- a/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -3,7 +3,7 @@ SELECT '===Ordinary case==='; SET replication_alter_partitions_sync = 2; DROP TABLE IF EXISTS clear_column; -CREATE TABLE clear_column (d Date, num Int64, str String) ENGINE = MergeTree ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column (d Date, num Int64, str String) ENGINE = MergeTree ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column VALUES ('2016-12-12', 1, 'a'), ('2016-11-12', 2, 'b'); @@ -24,8 +24,8 @@ SELECT '===Replicated case==='; DROP TABLE IF EXISTS clear_column1; DROP TABLE IF EXISTS clear_column2; SELECT sleep(1) FORMAT Null; -CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; -CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by d SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '1') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; +CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test_00446/tables/clear_column', '2') ORDER BY d PARTITION by toYYYYMM(d) SETTINGS min_bytes_for_wide_part = 0; INSERT INTO clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01'); SYSTEM SYNC REPLICA clear_column2; diff --git a/tests/queries/0_stateless/00933_ttl_simple.reference b/tests/queries/0_stateless/00933_ttl_simple.reference index a4ef8033328..e3982814eab 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.reference +++ b/tests/queries/0_stateless/00933_ttl_simple.reference @@ -6,11 +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\n(\n `b` Int32,\n `a` Int32 TTL now() - 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() - 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 0 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() + 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() + 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 -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 index_granularity = 8192 +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 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 index_granularity = 8192 +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 diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index c0adcd21e62..aa8b33b2999 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -1,6 +1,8 @@ drop table if exists ttl_00933_1; -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); +-- 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; @@ -19,10 +21,11 @@ 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); +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; @@ -44,7 +47,7 @@ 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(); +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; @@ -52,7 +55,7 @@ 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(); +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; @@ -60,7 +63,7 @@ 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(); +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; @@ -68,7 +71,7 @@ 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(); +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;