tests for compact parts

This commit is contained in:
CurtizJ 2020-02-19 17:55:03 +03:00
parent 23592991e6
commit 6bc5d988d5
6 changed files with 149 additions and 81 deletions

View File

@ -0,0 +1,5 @@
<yandex>
<merge_tree>
<min_rows_for_wide_part>512</min_rows_for_wide_part>
</merge_tree>
</yandex>

View File

@ -38,9 +38,15 @@ def create_tables(name, nodes, node_settings, shard):
ORDER BY id
SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes},
min_rows_for_wide_part = {min_rows_for_wide_part}, min_bytes_for_wide_part = {min_bytes_for_wide_part}
'''.format(name=name, shard=shard, repl=i, **settings)
)
'''.format(name=name, shard=shard, repl=i, **settings))
def create_tables_old_format(name, nodes, shard):
for i, node in enumerate(nodes):
node.query(
'''
CREATE TABLE {name}(date Date, id UInt32, s String, arr Array(Int32))
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}', date, id, 64)
'''.format(name=name, shard=shard, repl=i))
node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True)
@ -54,6 +60,8 @@ node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['confi
settings_compact = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0}
settings_wide = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_bytes_for_wide_part' : 0}
node5 = cluster.add_instance('node5', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True)
node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
@ -64,64 +72,71 @@ def start_cluster():
create_tables('non_adaptive_table', [node1, node2], [settings_not_adaptive, settings_default], "shard1")
create_tables('polymorphic_table_compact', [node3, node4], [settings_compact, settings_wide], "shard2")
create_tables('polymorphic_table_wide', [node3, node4], [settings_wide, settings_compact], "shard2")
create_tables_old_format('polymorphic_table', [node5, node6], "shard3")
yield cluster
finally:
cluster.shutdown()
def test_polymorphic_parts_basics(start_cluster):
node1.query("SYSTEM STOP MERGES")
node2.query("SYSTEM STOP MERGES")
@pytest.mark.parametrize(
('first_node', 'second_node'),
[
(node1, node2),
(node5, node6)
]
)
def test_polymorphic_parts_basics(start_cluster, first_node, second_node):
first_node.query("SYSTEM STOP MERGES")
second_node.query("SYSTEM STOP MERGES")
for size in [300, 300, 600]:
insert_random_data('polymorphic_table', node1, size)
node2.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
insert_random_data('polymorphic_table', first_node, size)
second_node.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
assert node1.query("SELECT count() FROM polymorphic_table") == "1200\n"
assert node2.query("SELECT count() FROM polymorphic_table") == "1200\n"
assert first_node.query("SELECT count() FROM polymorphic_table") == "1200\n"
assert second_node.query("SELECT count() FROM polymorphic_table") == "1200\n"
expected = "Compact\t2\nWide\t1\n"
assert TSV(node1.query("SELECT part_type, count() FROM system.parts " \
assert TSV(first_node.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'polymorphic_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected)
assert TSV(node2.query("SELECT part_type, count() FROM system.parts " \
assert TSV(second_node.query("SELECT part_type, count() FROM system.parts " \
"WHERE table = 'polymorphic_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected)
node1.query("SYSTEM START MERGES")
node2.query("SYSTEM START MERGES")
first_node.query("SYSTEM START MERGES")
second_node.query("SYSTEM START MERGES")
for _ in range(40):
insert_random_data('polymorphic_table', node1, 10)
insert_random_data('polymorphic_table', node2, 10)
insert_random_data('polymorphic_table', first_node, 10)
insert_random_data('polymorphic_table', second_node, 10)
node1.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
node2.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
first_node.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
second_node.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
assert node1.query("SELECT count() FROM polymorphic_table") == "2000\n"
assert node2.query("SELECT count() FROM polymorphic_table") == "2000\n"
assert first_node.query("SELECT count() FROM polymorphic_table") == "2000\n"
assert second_node.query("SELECT count() FROM polymorphic_table") == "2000\n"
node1.query("OPTIMIZE TABLE polymorphic_table FINAL")
node2.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
first_node.query("OPTIMIZE TABLE polymorphic_table FINAL")
second_node.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
assert node1.query("SELECT count() FROM polymorphic_table") == "2000\n"
assert node2.query("SELECT count() FROM polymorphic_table") == "2000\n"
assert first_node.query("SELECT count() FROM polymorphic_table") == "2000\n"
assert second_node.query("SELECT count() FROM polymorphic_table") == "2000\n"
assert node1.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' AND active") == "Wide\n"
assert node2.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' AND active") == "Wide\n"
assert first_node.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' AND active") == "Wide\n"
assert second_node.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' AND active") == "Wide\n"
# Check alters and mutations also work
node1.query("ALTER TABLE polymorphic_table ADD COLUMN ss String")
node1.query("ALTER TABLE polymorphic_table UPDATE ss = toString(id) WHERE 1")
first_node.query("ALTER TABLE polymorphic_table ADD COLUMN ss String")
first_node.query("ALTER TABLE polymorphic_table UPDATE ss = toString(id) WHERE 1")
node2.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
second_node.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
node1.query("SELECT count(ss) FROM polymorphic_table") == "2000\n"
node1.query("SELECT uniqExact(ss) FROM polymorphic_table") == "600\n"
first_node.query("SELECT count(ss) FROM polymorphic_table") == "2000\n"
first_node.query("SELECT uniqExact(ss) FROM polymorphic_table") == "600\n"
node2.query("SELECT count(ss) FROM polymorphic_table") == "2000\n"
node2.query("SELECT uniqExact(ss) FROM polymorphic_table") == "600\n"
second_node.query("SELECT count(ss) FROM polymorphic_table") == "2000\n"
second_node.query("SELECT uniqExact(ss) FROM polymorphic_table") == "600\n"
# Check that follower replicas create parts of the same type, which leader has chosen at merge.
@ -167,7 +182,7 @@ def start_cluster_diff_versions():
node7.query(
'''
CREATE TABLE {name}(date Date, id UInt32, s String, arr Array(Int32))
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard4/{name}', '1')
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard5/{name}', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes}
@ -177,7 +192,7 @@ def start_cluster_diff_versions():
node8.query(
'''
CREATE TABLE {name}(date Date, id UInt32, s String, arr Array(Int32))
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard4/{name}', '2')
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard5/{name}', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes},

View File

@ -1,46 +0,0 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
. $CURDIR/mergetree_mutations.lib
# Testing basic functionality with compact parts
${CLICKHOUSE_CLIENT} --query="drop table if exists mt_compact;"
${CLICKHOUSE_CLIENT} --query="create table mt_compact(a UInt64, b UInt64 DEFAULT a * a, s String, n Nested(x UInt32, y String), lc LowCardinality(String)) \
engine = MergeTree \
order by a partition by a % 10 \
settings index_granularity = 8, \
min_rows_for_wide_part = 10;"
${CLICKHOUSE_CLIENT} --query="insert into mt_compact (a, s, n.y, lc) select number, toString((number * 2132214234 + 5434543) % 2133443), ['a', 'b', 'c'], number % 2 ? 'bar' : 'baz' from numbers(90);"
${CLICKHOUSE_CLIENT} --query="select * from mt_compact order by a limit 10;"
${CLICKHOUSE_CLIENT} --query="select '=====================';"
${CLICKHOUSE_CLIENT} --query="select distinct part_type from system.parts where database = currentDatabase() and table = 'mt_compact' and active;"
${CLICKHOUSE_CLIENT} --query="insert into mt_compact (a, s, n.x, lc) select number % 3, toString((number * 75434535 + 645645) % 2133443), [1, 2], toString(number) from numbers(5);"
${CLICKHOUSE_CLIENT} --query="optimize table mt_compact final;"
${CLICKHOUSE_CLIENT} --query="select part_type, count() from system.parts where database = currentDatabase() and table = 'mt_compact' and active group by part_type order by part_type; "
${CLICKHOUSE_CLIENT} --query="select * from mt_compact order by a, s limit 10;"
${CLICKHOUSE_CLIENT} --query="select '=====================';"
${CLICKHOUSE_CLIENT} --query="alter table mt_compact drop column n.y;"
${CLICKHOUSE_CLIENT} --query="alter table mt_compact add column n.y Array(String) DEFAULT ['qwqw'] after n.x;"
${CLICKHOUSE_CLIENT} --query="select * from mt_compact order by a, s limit 10;"
${CLICKHOUSE_CLIENT} --query="select '=====================';"
${CLICKHOUSE_CLIENT} --query="alter table mt_compact update b = 42 where 1;"
sleep 0.5
mutation_id=`${CLICKHOUSE_CLIENT} --query="SELECT max(mutation_id) FROM system.mutations WHERE table='mt_compact'"`
wait_for_mutation "mt_compact" "$mutation_id"
${CLICKHOUSE_CLIENT} --query="select * from mt_compact where a > 1 order by a, s limit 10;"
${CLICKHOUSE_CLIENT} --query="select '=====================';"
${CLICKHOUSE_CLIENT} --query="drop table if exists mt_compact;"

View File

@ -0,0 +1,36 @@
-- Testing basic functionality with compact parts
set mutations_sync = 2;
drop table if exists mt_compact;
create table mt_compact(a UInt64, b UInt64 DEFAULT a * a, s String, n Nested(x UInt32, y String), lc LowCardinality(String))
engine = MergeTree
order by a partition by a % 10
settings index_granularity = 8,
min_rows_for_wide_part = 10;
insert into mt_compact (a, s, n.y, lc) select number, toString((number * 2132214234 + 5434543) % 2133443), ['a', 'b', 'c'], number % 2 ? 'bar' : 'baz' from numbers(90);
select * from mt_compact order by a limit 10;
select '=====================';
select distinct part_type from system.parts where database = currentDatabase() and table = 'mt_compact' and active;
insert into mt_compact (a, s, n.x, lc) select number % 3, toString((number * 75434535 + 645645) % 2133443), [1, 2], toString(number) from numbers(5);
optimize table mt_compact final;
select part_type, count() from system.parts where database = currentDatabase() and table = 'mt_compact' and active group by part_type order by part_type;
select * from mt_compact order by a, s limit 10;
select '=====================';
alter table mt_compact drop column n.y;
alter table mt_compact add column n.y Array(String) DEFAULT ['qwqw'] after n.x;
select * from mt_compact order by a, s limit 10;
select '=====================';
alter table mt_compact update b = 42 where 1;
select * from mt_compact where a > 1 order by a, s limit 10;
select '=====================';
drop table if exists mt_compact;

View File

@ -0,0 +1,6 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
11 5 4.7 6.50 cba b 2014-06-11
12 5 4.7 6.50 cba b 2015-01-01
"rows_read": 4,
"rows_read": 2,

View File

@ -0,0 +1,52 @@
#!/usr/bin/env bash
# It's test 00837_minmax_index, buts with compact part.
# Checks, that basic functionality with compact parts and skip indices is not broken.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;"
$CLICKHOUSE_CLIENT -n --query="
CREATE TABLE minmax_idx
(
u64 UInt64,
i32 Int32,
f64 Float64,
d Decimal(10, 2),
s String,
e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date,
INDEX idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1,
INDEX idx_all2 (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 2,
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = MergeTree()
ORDER BY u64
SETTINGS index_granularity = 2, min_rows_for_wide_part = 1000000"
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
(0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04'),
(1, 5, 4.7, 6.5, 'cba', 'b', '2014-03-11'),
(2, 2, 4.5, 2.5, 'abc', 'a', '2014-01-01'),
(3, 5, 6.9, 1.57, 'bac', 'c', '2017-01-01'),
(4, 2, 4.5, 2.5, 'abc', 'a', '2016-01-01'),
(5, 5, 6.9, 1.57, 'bac', 'c', '2014-11-11'),
(6, 2, 4.5, 2.5, 'abc', 'a', '2014-02-11'),
(7, 5, 6.9, 1.57, 'bac', 'c', '2014-04-11'),
(8, 2, 4.5, 2.5, 'abc', 'a', '2014-05-11'),
(9, 5, 6.9, 1.57, 'bac', 'c', '2014-07-11'),
(11, 5, 4.7, 6.5, 'cba', 'b', '2014-06-11'),
(12, 5, 4.7, 6.5, 'cba', 'b', '2015-01-01')"
# simple select
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt FORMAT JSON" | grep "rows_read"
# select with hole made by primary key
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"