added tests for synchronous insert into distributed table [#CLICKHOUSE-3033]

This commit is contained in:
Nikolai Kochetov 2017-07-26 17:41:21 +03:00
parent b3157aebb2
commit 19d3c36871
4 changed files with 112 additions and 4 deletions

View File

@ -19,6 +19,14 @@ class Client:
return QueryRequest(self, sql, stdin, timeout)
class QueryTimeoutExceedException(Exception):
pass
class QueryRuntimeException(Exception):
pass
class QueryRequest:
def __init__(self, client, sql, stdin=None, timeout=None):
self.client = client
@ -61,11 +69,11 @@ class QueryRequest:
stdout = self.stdout_file.read()
stderr = self.stderr_file.read()
if self.process.returncode != 0 or stderr:
raise Exception('Client failed! Return code: {}, stderr: {}'.format(self.process.returncode, stderr))
if self.timer is not None and not self.process_finished_before_timeout:
raise Exception('Client timed out!')
raise QueryTimeoutExceedException('Client timed out!')
if self.process.returncode != 0 or stderr:
raise QueryRuntimeException('Client failed! Return code: {}, stderr: {}'.format(self.process.returncode, stderr))
return stdout

View File

@ -0,0 +1,16 @@
<yandex>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,84 @@
from contextlib import contextmanager
from helpers.network import PartitionManager
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException, QueryTimeoutExceedException
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'])
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'])
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for node in (node1, node2):
node.query('''
CREATE TABLE local_table(date Date, val UInt64) ENGINE = MergeTree(date, (date, val), 8192);
''')
node1.query('''
CREATE TABLE distributed_table(date Date, val UInt64) ENGINE = Distributed(test_cluster, default, local_table)
''')
yield cluster
finally:
cluster.shutdown()
def test_insertion_sync(started_cluster):
node1.query('''SET insert_distributed_sync = 1, insert_distributed_timeout = 0;
INSERT INTO distributed_table SELECT today() as date, number as val FROM system.numbers LIMIT 10000''')
assert node2.query("SELECT count() FROM local_table").rstrip() == '10000'
node1.query('''
SET insert_distributed_sync = 1, insert_distributed_timeout = 1;
INSERT INTO distributed_table SELECT today() - 1 as date, number as val FROM system.numbers LIMIT 10000''')
assert node2.query("SELECT count() FROM local_table").rstrip() == '20000'
"""
def test_insertion_sync_fails_on_error(started_cluster):
with PartitionManager() as pm:
pm.partition_instances(node2, node1, action='REJECT --reject-with tcp-reset')
with pytest.raises(QueryRuntimeException):
node1.query('''
SET insert_distributed_sync = 1, insert_distributed_timeout = 0;
INSERT INTO distributed_table SELECT today() as date, number as val FROM system.numbers''', timeout=2)
"""
def test_insertion_sync_fails_with_timeout(started_cluster):
with pytest.raises(QueryRuntimeException):
node1.query('''
SET insert_distributed_sync = 1, insert_distributed_timeout = 1;
INSERT INTO distributed_table SELECT today() as date, number as val FROM system.numbers''', timeout=1.5)
def test_insertion_without_sync_ignores_timeout(started_cluster):
with pytest.raises(QueryTimeoutExceedException):
node1.query('''
SET insert_distributed_sync = 0, insert_distributed_timeout = 1;
INSERT INTO distributed_table SELECT today() as date, number as val FROM system.numbers''', timeout=1.5)
def test_insertion_sync_with_disabled_timeout(started_cluster):
with pytest.raises(QueryTimeoutExceedException):
node1.query('''
SET insert_distributed_sync = 1, insert_distributed_timeout = 0;
INSERT INTO distributed_table SELECT today() as date, number as val FROM system.numbers''', timeout=1)
if __name__ == '__main__':
with contextmanager(started_cluster)() as cluster:
for name, instance in cluster.instances.items():
print name, instance.ip_address
raw_input("Cluster created, press any key to destroy...")