2017-07-26 12:31:55 +00:00
|
|
|
import time
|
|
|
|
|
|
|
|
import pytest
|
|
|
|
from helpers.client import CommandRequest
|
2017-10-06 11:29:58 +00:00
|
|
|
from helpers.client import QueryTimeoutExceedException
|
2020-09-16 04:26:10 +00:00
|
|
|
from helpers.cluster import ClickHouseCluster
|
|
|
|
from helpers.test_tools import TSV
|
2017-07-26 12:31:55 +00:00
|
|
|
|
|
|
|
cluster = ClickHouseCluster(__file__)
|
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
node1 = cluster.add_instance('node1',
|
|
|
|
main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"],
|
|
|
|
with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1})
|
|
|
|
node2 = cluster.add_instance('node2',
|
|
|
|
main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"],
|
|
|
|
with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2})
|
2017-07-26 12:31:55 +00:00
|
|
|
nodes = [node1, node2]
|
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
|
2017-07-26 12:31:55 +00:00
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def started_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
|
|
|
yield cluster
|
|
|
|
|
|
|
|
finally:
|
2017-07-26 14:15:16 +00:00
|
|
|
pass
|
2017-07-26 12:31:55 +00:00
|
|
|
cluster.shutdown()
|
|
|
|
|
|
|
|
|
|
|
|
def test_deduplication_window_in_seconds(started_cluster):
|
|
|
|
node = node1
|
|
|
|
|
2017-10-06 11:29:58 +00:00
|
|
|
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)")
|
2017-07-26 14:15:16 +00:00
|
|
|
time.sleep(1)
|
2020-09-16 04:26:10 +00:00
|
|
|
node.query("INSERT INTO simple VALUES (0, 0)") # deduplication works here
|
2017-10-06 11:29:58 +00:00
|
|
|
node.query("INSERT INTO simple VALUES (0, 1)")
|
|
|
|
assert TSV(node.query("SELECT count() FROM simple")) == TSV("2\n")
|
2017-07-26 12:31:55 +00:00
|
|
|
|
|
|
|
# wait clean thread
|
|
|
|
time.sleep(2)
|
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
assert \
|
|
|
|
TSV.toMat(node.query("SELECT count() FROM system.zookeeper WHERE path='/clickhouse/tables/0/simple/blocks'"))[
|
|
|
|
0][
|
|
|
|
0] == "1"
|
|
|
|
node.query("INSERT INTO simple VALUES (0, 0)") # deduplication doesn't works here, the first hash node was deleted
|
2017-10-06 11:29:58 +00:00
|
|
|
assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3"
|
2017-07-26 12:31:55 +00:00
|
|
|
|
2017-10-06 11:29:58 +00:00
|
|
|
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
|
2017-07-26 12:31:55 +00:00
|
|
|
|
|
|
|
|
|
|
|
# Currently this test just reproduce incorrect behavior that sould be fixed
|
2019-06-25 13:20:04 +00:00
|
|
|
@pytest.mark.skip(reason="Flapping test")
|
2017-07-26 12:31:55 +00:00
|
|
|
def test_deduplication_works_in_case_of_intensive_inserts(started_cluster):
|
|
|
|
inserters = []
|
|
|
|
fetchers = []
|
|
|
|
|
2017-10-06 11:29:58 +00:00
|
|
|
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)")
|
|
|
|
|
2017-07-26 12:31:55 +00:00
|
|
|
for node in nodes:
|
|
|
|
host = node.ip_address
|
|
|
|
|
|
|
|
inserters.append(CommandRequest(['/bin/bash'], timeout=10, stdin="""
|
|
|
|
set -e
|
|
|
|
for i in `seq 1000`; do
|
2018-09-07 11:51:51 +00:00
|
|
|
{} --host {} -q "INSERT INTO simple VALUES (0, 0)"
|
2017-07-26 12:31:55 +00:00
|
|
|
done
|
2018-09-07 11:51:51 +00:00
|
|
|
""".format(cluster.get_client_cmd(), host)))
|
2017-07-26 12:31:55 +00:00
|
|
|
|
|
|
|
fetchers.append(CommandRequest(['/bin/bash'], timeout=10, stdin="""
|
|
|
|
set -e
|
|
|
|
for i in `seq 1000`; do
|
2018-09-07 11:51:51 +00:00
|
|
|
res=`{} --host {} -q "SELECT count() FROM simple"`
|
2017-10-06 11:29:58 +00:00
|
|
|
if [[ $? -ne 0 || $res -ne 1 ]]; then
|
2017-07-26 12:31:55 +00:00
|
|
|
echo "Selected $res elements! Host: {}" 1>&2
|
|
|
|
exit -1
|
|
|
|
fi;
|
|
|
|
done
|
2018-09-07 11:51:51 +00:00
|
|
|
""".format(cluster.get_client_cmd(), host, node.name)))
|
2017-07-26 12:31:55 +00:00
|
|
|
|
|
|
|
# There were not errors during INSERTs
|
|
|
|
for inserter in inserters:
|
|
|
|
try:
|
|
|
|
inserter.get_answer()
|
2017-10-06 11:29:58 +00:00
|
|
|
except QueryTimeoutExceedException:
|
|
|
|
# Only timeout is accepted
|
|
|
|
pass
|
2017-07-26 12:31:55 +00:00
|
|
|
|
|
|
|
# There were not errors during SELECTs
|
|
|
|
for fetcher in fetchers:
|
|
|
|
try:
|
|
|
|
fetcher.get_answer()
|
2017-10-06 11:29:58 +00:00
|
|
|
except QueryTimeoutExceedException:
|
|
|
|
# Only timeout is accepted
|
2017-07-26 12:31:55 +00:00
|
|
|
pass
|
2017-10-06 11:29:58 +00:00
|
|
|
|
|
|
|
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
|