mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fix test and add logical error
This commit is contained in:
parent
6fc39b10d3
commit
c74631c650
@ -18,6 +18,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_DIFFERS_TOO_MUCH;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000;
|
||||
@ -194,7 +195,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPartAndFetchIfPossible(
|
||||
if (!storage.queue.remove(zookeeper, part_name))
|
||||
{
|
||||
/// The part was not in our queue. Why did it happen?
|
||||
LOG_ERROR(log, "Missing part {} is not in our queue.", part_name);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing part {} is not in our queue.", part_name);
|
||||
}
|
||||
|
||||
/** This situation is possible if on all the replicas where the part was, it deteriorated.
|
||||
|
@ -13,18 +13,6 @@ def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
for node in [node1, node2]:
|
||||
node.query('''
|
||||
CREATE TABLE replicated_mt(date Date, id UInt32, value Int32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
|
||||
'''.format(replica=node.name))
|
||||
|
||||
node1.query('''
|
||||
CREATE TABLE non_replicated_mt(date Date, id UInt32, value Int32)
|
||||
ENGINE = MergeTree() PARTITION BY toYYYYMM(date) ORDER BY id
|
||||
SETTINGS min_bytes_for_wide_part=0;
|
||||
''')
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
@ -54,6 +42,14 @@ def remove_part_from_disk(node, table, part_name):
|
||||
|
||||
|
||||
def test_check_normal_table_corruption(started_cluster):
|
||||
node1.query("DROP TABLE IF EXISTS non_replicated_mt")
|
||||
|
||||
node1.query('''
|
||||
CREATE TABLE non_replicated_mt(date Date, id UInt32, value Int32)
|
||||
ENGINE = MergeTree() PARTITION BY toYYYYMM(date) ORDER BY id
|
||||
SETTINGS min_bytes_for_wide_part=0;
|
||||
''')
|
||||
|
||||
node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-02-01'), 1, 10), (toDate('2019-02-01'), 2, 12)")
|
||||
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902",
|
||||
settings={"check_query_single_value_result": 0}) == "201902_1_1_0\t1\t\n"
|
||||
@ -94,8 +90,14 @@ def test_check_normal_table_corruption(started_cluster):
|
||||
|
||||
|
||||
def test_check_replicated_table_simple(started_cluster):
|
||||
node1.query("TRUNCATE TABLE replicated_mt")
|
||||
node2.query("SYSTEM SYNC REPLICA replicated_mt")
|
||||
for node in [node1, node2]:
|
||||
node.query("DROP TABLE IF EXISTS replicated_mt")
|
||||
|
||||
node.query('''
|
||||
CREATE TABLE replicated_mt(date Date, id UInt32, value Int32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
|
||||
'''.format(replica=node.name))
|
||||
|
||||
node1.query("INSERT INTO replicated_mt VALUES (toDate('2019-02-01'), 1, 10), (toDate('2019-02-01'), 2, 12)")
|
||||
node2.query("SYSTEM SYNC REPLICA replicated_mt")
|
||||
|
||||
@ -119,34 +121,40 @@ def test_check_replicated_table_simple(started_cluster):
|
||||
|
||||
|
||||
def test_check_replicated_table_corruption(started_cluster):
|
||||
node1.query("TRUNCATE TABLE replicated_mt")
|
||||
node2.query("SYSTEM SYNC REPLICA replicated_mt")
|
||||
node1.query("INSERT INTO replicated_mt VALUES (toDate('2019-02-01'), 1, 10), (toDate('2019-02-01'), 2, 12)")
|
||||
node1.query("INSERT INTO replicated_mt VALUES (toDate('2019-01-02'), 3, 10), (toDate('2019-01-02'), 4, 12)")
|
||||
node2.query("SYSTEM SYNC REPLICA replicated_mt")
|
||||
for node in [node1, node2]:
|
||||
node.query("DROP TABLE IF EXISTS replicated_mt_1")
|
||||
|
||||
assert node1.query("SELECT count() from replicated_mt") == "4\n"
|
||||
assert node2.query("SELECT count() from replicated_mt") == "4\n"
|
||||
node.query('''
|
||||
CREATE TABLE replicated_mt_1(date Date, id UInt32, value Int32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt_1', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id;
|
||||
'''.format(replica=node.name))
|
||||
|
||||
node1.query("INSERT INTO replicated_mt_1 VALUES (toDate('2019-02-01'), 1, 10), (toDate('2019-02-01'), 2, 12)")
|
||||
node1.query("INSERT INTO replicated_mt_1 VALUES (toDate('2019-01-02'), 3, 10), (toDate('2019-01-02'), 4, 12)")
|
||||
node2.query("SYSTEM SYNC REPLICA replicated_mt_1")
|
||||
|
||||
assert node1.query("SELECT count() from replicated_mt_1") == "4\n"
|
||||
assert node2.query("SELECT count() from replicated_mt_1") == "4\n"
|
||||
|
||||
part_name = node1.query(
|
||||
"SELECT name from system.parts where table = 'replicated_mt' and partition_id = '201901' and active = 1").strip()
|
||||
"SELECT name from system.parts where table = 'replicated_mt_1' and partition_id = '201901' and active = 1").strip()
|
||||
|
||||
corrupt_data_part_on_disk(node1, "replicated_mt", part_name)
|
||||
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={
|
||||
"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(
|
||||
corrupt_data_part_on_disk(node1, "replicated_mt_1", part_name)
|
||||
assert node1.query("CHECK TABLE replicated_mt_1 PARTITION 201901", settings={
|
||||
"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and will try to fetch.\n".format(
|
||||
p=part_name)
|
||||
|
||||
node1.query("SYSTEM SYNC REPLICA replicated_mt")
|
||||
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901",
|
||||
node1.query("SYSTEM SYNC REPLICA replicated_mt_1")
|
||||
assert node1.query("CHECK TABLE replicated_mt_1 PARTITION 201901",
|
||||
settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
|
||||
assert node1.query("SELECT count() from replicated_mt") == "4\n"
|
||||
assert node1.query("SELECT count() from replicated_mt_1") == "4\n"
|
||||
|
||||
remove_part_from_disk(node2, "replicated_mt", part_name)
|
||||
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901", settings={
|
||||
"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(
|
||||
remove_part_from_disk(node2, "replicated_mt_1", part_name)
|
||||
assert node2.query("CHECK TABLE replicated_mt_1 PARTITION 201901", settings={
|
||||
"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and will try to fetch.\n".format(
|
||||
p=part_name)
|
||||
|
||||
node1.query("SYSTEM SYNC REPLICA replicated_mt")
|
||||
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901",
|
||||
node1.query("SYSTEM SYNC REPLICA replicated_mt_1")
|
||||
assert node1.query("CHECK TABLE replicated_mt_1 PARTITION 201901",
|
||||
settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
|
||||
assert node1.query("SELECT count() from replicated_mt") == "4\n"
|
||||
assert node1.query("SELECT count() from replicated_mt_1") == "4\n"
|
||||
|
Loading…
Reference in New Issue
Block a user