mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Add integration test. [#CLICKHOUSE-3178]
This commit is contained in:
parent
d6ea9c3dcf
commit
a5e29f2166
@ -3,5 +3,6 @@
|
|||||||
<replicated_deduplication_window>999999999</replicated_deduplication_window>
|
<replicated_deduplication_window>999999999</replicated_deduplication_window>
|
||||||
<replicated_deduplication_window_seconds>1</replicated_deduplication_window_seconds>
|
<replicated_deduplication_window_seconds>1</replicated_deduplication_window_seconds>
|
||||||
<cleanup_delay_period>1</cleanup_delay_period>
|
<cleanup_delay_period>1</cleanup_delay_period>
|
||||||
|
<old_parts_lifetime>1</old_parts_lifetime>
|
||||||
</merge_tree>
|
</merge_tree>
|
||||||
</yandex>
|
</yandex>
|
||||||
|
@ -7,31 +7,19 @@ from helpers.cluster import ClickHouseCluster
|
|||||||
from helpers.network import PartitionManager
|
from helpers.network import PartitionManager
|
||||||
from helpers.test_tools import TSV
|
from helpers.test_tools import TSV
|
||||||
from helpers.client import CommandRequest
|
from helpers.client import CommandRequest
|
||||||
|
from helpers.client import QueryTimeoutExceedException
|
||||||
|
|
||||||
|
|
||||||
cluster = ClickHouseCluster(__file__)
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
|
||||||
node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True)
|
node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macroses={"layer": 0, "shard": 0, "replica": 1})
|
||||||
node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True)
|
node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macroses={"layer": 0, "shard": 0, "replica": 2})
|
||||||
nodes = [node1, node2]
|
nodes = [node1, node2]
|
||||||
|
|
||||||
@pytest.fixture(scope="module")
|
@pytest.fixture(scope="module")
|
||||||
def started_cluster():
|
def started_cluster():
|
||||||
try:
|
try:
|
||||||
cluster.start()
|
cluster.start()
|
||||||
|
|
||||||
for node in nodes:
|
|
||||||
node.query('''
|
|
||||||
CREATE TABLE simple (date Date, id UInt32)
|
|
||||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
|
|
||||||
'''.format(replica=node.name))
|
|
||||||
node.query("INSERT INTO simple VALUES (0, 0)")
|
|
||||||
|
|
||||||
node.query('''
|
|
||||||
CREATE TABLE simple2 (date Date, id UInt32)
|
|
||||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple2', '{replica}', date, id, 8192);
|
|
||||||
'''.format(replica=node.name))
|
|
||||||
|
|
||||||
yield cluster
|
yield cluster
|
||||||
|
|
||||||
finally:
|
finally:
|
||||||
@ -42,24 +30,24 @@ ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple2', '{replica}', date,
|
|||||||
def test_deduplication_window_in_seconds(started_cluster):
|
def test_deduplication_window_in_seconds(started_cluster):
|
||||||
node = node1
|
node = node1
|
||||||
|
|
||||||
node.query("INSERT INTO simple2 VALUES (0, 0)")
|
node1.query("""
|
||||||
|
CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32)
|
||||||
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, id, 8192)""")
|
||||||
|
|
||||||
|
node.query("INSERT INTO simple VALUES (0, 0)")
|
||||||
time.sleep(1)
|
time.sleep(1)
|
||||||
node.query("INSERT INTO simple2 VALUES (0, 0)") # deduplication works here
|
node.query("INSERT INTO simple VALUES (0, 0)") # deduplication works here
|
||||||
node.query("INSERT INTO simple2 VALUES (0, 1)")
|
node.query("INSERT INTO simple VALUES (0, 1)")
|
||||||
assert TSV(node.query("SELECT count() FROM simple2")) == TSV("2\n")
|
assert TSV(node.query("SELECT count() FROM simple")) == TSV("2\n")
|
||||||
|
|
||||||
# wait clean thread
|
# wait clean thread
|
||||||
time.sleep(2)
|
time.sleep(2)
|
||||||
|
|
||||||
assert TSV.toMat(node.query("SELECT count() FROM system.zookeeper WHERE path='/clickhouse/tables/0/simple2/blocks'"))[0][0] == "1"
|
assert TSV.toMat(node.query("SELECT count() FROM system.zookeeper WHERE path='/clickhouse/tables/0/simple/blocks'"))[0][0] == "1"
|
||||||
node.query("INSERT INTO simple2 VALUES (0, 0)") # deduplication doesn't works here, the first hash node was deleted
|
node.query("INSERT INTO simple VALUES (0, 0)") # deduplication doesn't works here, the first hash node was deleted
|
||||||
assert TSV.toMat(node.query("SELECT count() FROM simple2"))[0][0] == "3"
|
assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3"
|
||||||
|
|
||||||
|
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
|
||||||
def check_timeout_exception(e):
|
|
||||||
s = str(e)
|
|
||||||
#print s
|
|
||||||
assert s.find('timed out!') >= 0 or s.find('Return code: -9') >= 0
|
|
||||||
|
|
||||||
|
|
||||||
# Currently this test just reproduce incorrect behavior that sould be fixed
|
# Currently this test just reproduce incorrect behavior that sould be fixed
|
||||||
@ -67,6 +55,12 @@ def test_deduplication_works_in_case_of_intensive_inserts(started_cluster):
|
|||||||
inserters = []
|
inserters = []
|
||||||
fetchers = []
|
fetchers = []
|
||||||
|
|
||||||
|
node1.query("""
|
||||||
|
CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32)
|
||||||
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, id, 8192)""")
|
||||||
|
|
||||||
|
node1.query("INSERT INTO simple VALUES (0, 0)")
|
||||||
|
|
||||||
for node in nodes:
|
for node in nodes:
|
||||||
host = node.ip_address
|
host = node.ip_address
|
||||||
|
|
||||||
@ -81,7 +75,7 @@ done
|
|||||||
set -e
|
set -e
|
||||||
for i in `seq 1000`; do
|
for i in `seq 1000`; do
|
||||||
res=`clickhouse-client --host {} -q "SELECT count() FROM simple"`
|
res=`clickhouse-client --host {} -q "SELECT count() FROM simple"`
|
||||||
if [[ $res -ne 1 ]]; then
|
if [[ $? -ne 0 || $res -ne 1 ]]; then
|
||||||
echo "Selected $res elements! Host: {}" 1>&2
|
echo "Selected $res elements! Host: {}" 1>&2
|
||||||
exit -1
|
exit -1
|
||||||
fi;
|
fi;
|
||||||
@ -92,14 +86,16 @@ done
|
|||||||
for inserter in inserters:
|
for inserter in inserters:
|
||||||
try:
|
try:
|
||||||
inserter.get_answer()
|
inserter.get_answer()
|
||||||
except Exception as e:
|
except QueryTimeoutExceedException:
|
||||||
check_timeout_exception(e)
|
# Only timeout is accepted
|
||||||
|
pass
|
||||||
|
|
||||||
# There were not errors during SELECTs
|
# There were not errors during SELECTs
|
||||||
for fetcher in fetchers:
|
for fetcher in fetchers:
|
||||||
try:
|
try:
|
||||||
fetcher.get_answer()
|
fetcher.get_answer()
|
||||||
except Exception as e:
|
except QueryTimeoutExceedException:
|
||||||
|
# Only timeout is accepted
|
||||||
pass
|
pass
|
||||||
# Uncomment when problem will be fixed
|
|
||||||
check_timeout_exception(e)
|
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
|
||||||
|
@ -0,0 +1,8 @@
|
|||||||
|
<yandex>
|
||||||
|
<merge_tree>
|
||||||
|
<replicated_deduplication_window>999999999</replicated_deduplication_window>
|
||||||
|
<replicated_deduplication_window_seconds>999999999</replicated_deduplication_window_seconds>
|
||||||
|
<cleanup_delay_period>10</cleanup_delay_period>
|
||||||
|
<old_parts_lifetime>1</old_parts_lifetime>
|
||||||
|
</merge_tree>
|
||||||
|
</yandex>
|
@ -0,0 +1,17 @@
|
|||||||
|
<yandex>
|
||||||
|
<remote_servers>
|
||||||
|
<test_cluster>
|
||||||
|
<shard>
|
||||||
|
<internal_replication>true</internal_replication>
|
||||||
|
<replica>
|
||||||
|
<host>node1</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
<replica>
|
||||||
|
<host>node2</host>
|
||||||
|
<port>9000</port>
|
||||||
|
</replica>
|
||||||
|
</shard>
|
||||||
|
</test_cluster>
|
||||||
|
</remote_servers>
|
||||||
|
</yandex>
|
61
dbms/tests/integration/test_random_inserts/test.py
Normal file
61
dbms/tests/integration/test_random_inserts/test.py
Normal file
@ -0,0 +1,61 @@
|
|||||||
|
import time
|
||||||
|
import os
|
||||||
|
from contextlib import contextmanager
|
||||||
|
|
||||||
|
import pytest
|
||||||
|
|
||||||
|
from helpers.cluster import ClickHouseCluster
|
||||||
|
from helpers.network import PartitionManager
|
||||||
|
from helpers.test_tools import TSV
|
||||||
|
from helpers.client import CommandRequest
|
||||||
|
|
||||||
|
|
||||||
|
cluster = ClickHouseCluster(__file__)
|
||||||
|
|
||||||
|
node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macroses={"layer": 0, "shard": 0, "replica": 1})
|
||||||
|
node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macroses={"layer": 0, "shard": 0, "replica": 2})
|
||||||
|
nodes = [node1, node2]
|
||||||
|
|
||||||
|
@pytest.fixture(scope="module")
|
||||||
|
def started_cluster():
|
||||||
|
try:
|
||||||
|
cluster.start()
|
||||||
|
yield cluster
|
||||||
|
|
||||||
|
finally:
|
||||||
|
pass
|
||||||
|
cluster.shutdown()
|
||||||
|
|
||||||
|
def test_random_inserts(started_cluster):
|
||||||
|
# Duration of the test, reduce it if don't want to wait
|
||||||
|
DURATION_SECONDS = 10# * 60
|
||||||
|
|
||||||
|
node1.query("""
|
||||||
|
CREATE TABLE simple ON CLUSTER test_cluster (date Date, i UInt32, s String)
|
||||||
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}', date, i, 8192)""")
|
||||||
|
|
||||||
|
with PartitionManager() as pm_random_drops:
|
||||||
|
for sacrifice in nodes:
|
||||||
|
pass # This test doesn't work with partition problems still
|
||||||
|
#pm_random_drops._add_rule({'probability': 0.01, 'destination': sacrifice.ip_address, 'source_port': 2181, 'action': 'REJECT --reject-with tcp-reset'})
|
||||||
|
#pm_random_drops._add_rule({'probability': 0.01, 'source': sacrifice.ip_address, 'destination_port': 2181, 'action': 'REJECT --reject-with tcp-reset'})
|
||||||
|
|
||||||
|
min_timestamp = int(time.time())
|
||||||
|
max_timestamp = min_timestamp + DURATION_SECONDS
|
||||||
|
num_timestamps = max_timestamp - min_timestamp + 1
|
||||||
|
|
||||||
|
bash_script = os.path.join(os.path.dirname(__file__), "test.sh")
|
||||||
|
inserters = []
|
||||||
|
for node in nodes:
|
||||||
|
cmd = ['/bin/bash', bash_script, node.ip_address, str(min_timestamp), str(max_timestamp)]
|
||||||
|
inserters.append(CommandRequest(cmd, timeout=DURATION_SECONDS * 2, stdin=''))
|
||||||
|
print node.name, node.ip_address
|
||||||
|
|
||||||
|
for inserter in inserters:
|
||||||
|
inserter.get_answer()
|
||||||
|
|
||||||
|
answer="{}\t{}\t{}\t{}\n".format(num_timestamps, num_timestamps, min_timestamp, max_timestamp)
|
||||||
|
for node in nodes:
|
||||||
|
assert TSV(node.query("SELECT count(), uniqExact(i), min(i), max(i) FROM simple")) == TSV(answer), node.name + " : " + node.query("SELECT groupArray(_part), i, count() AS c FROM simple GROUP BY i ORDER BY c DESC LIMIT 1")
|
||||||
|
|
||||||
|
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
|
44
dbms/tests/integration/test_random_inserts/test.sh
Executable file
44
dbms/tests/integration/test_random_inserts/test.sh
Executable file
@ -0,0 +1,44 @@
|
|||||||
|
#!/bin/bash
|
||||||
|
#set -e
|
||||||
|
|
||||||
|
[[ -n "$1" ]] && host="$1" || host="127.0.0.1"
|
||||||
|
[[ -n "$2" ]] && min_timestamp="$2" || min_timestamp=$(( $(date +%s) - 10 ))
|
||||||
|
[[ -n "$3" ]] && max_timestamp="$3" || max_timestamp=$(( $(date +%s) + 10 ))
|
||||||
|
|
||||||
|
timestamps=`seq $min_timestamp $max_timestamp`
|
||||||
|
|
||||||
|
function reliable_insert {
|
||||||
|
local ts="$1"
|
||||||
|
num_tries=0
|
||||||
|
while true; do
|
||||||
|
if (( $num_tries > 20 )); then
|
||||||
|
echo "Too many retries" 1>&2
|
||||||
|
exit -1
|
||||||
|
fi
|
||||||
|
|
||||||
|
#echo clickhouse-client --host $host -q "INSERT INTO simple VALUES (0, $ts, '$ts')"
|
||||||
|
res=`clickhouse-client --host $host -q "INSERT INTO simple VALUES (0, $ts, '$ts')" 2>&1`
|
||||||
|
rt=$?
|
||||||
|
num_tries=$(($num_tries+1))
|
||||||
|
|
||||||
|
if (( $rt == 0 )); then break; fi
|
||||||
|
if [[ $res == *"Code: 319. "*"Unknown status, client must retry"* || $res == *"Code: 999. "* ]]; then
|
||||||
|
continue
|
||||||
|
else
|
||||||
|
echo FAIL "$res" 1>&2
|
||||||
|
exit -1
|
||||||
|
fi
|
||||||
|
done;
|
||||||
|
}
|
||||||
|
|
||||||
|
for i in $timestamps; do
|
||||||
|
|
||||||
|
cur_timestamp=$(date +%s)
|
||||||
|
while (( $cur_timestamp < $i )); do
|
||||||
|
ts=`shuf -i $min_timestamp-$cur_timestamp -n 1`
|
||||||
|
reliable_insert "$ts"
|
||||||
|
cur_timestamp=$(date +%s)
|
||||||
|
done
|
||||||
|
|
||||||
|
reliable_insert "$i"
|
||||||
|
done
|
Loading…
Reference in New Issue
Block a user