mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
fix more tests
This commit is contained in:
parent
6031e6bae9
commit
fa6d88b3b2
@ -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;''')
|
||||
|
@ -25,4 +25,8 @@
|
||||
</s3>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
</merge_tree>
|
||||
</yandex>
|
||||
|
@ -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')
|
||||
|
@ -18,4 +18,8 @@
|
||||
</s3>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
</merge_tree>
|
||||
</yandex>
|
||||
|
@ -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)
|
||||
|
@ -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))
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user