Merge pull request #28707 from azat/max_suspicious_broken_parts_bytes

Implement max_suspicious_broken_parts_bytes
This commit is contained in:
Anton Popov 2021-09-20 19:55:07 +03:00 committed by GitHub
commit 55cf857aba
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 145 additions and 11 deletions

View File

@ -978,6 +978,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
DataPartsVector broken_parts_to_detach;
size_t suspicious_broken_parts = 0;
size_t suspicious_broken_parts_bytes = 0;
std::atomic<bool> has_adaptive_parts = false;
std::atomic<bool> has_non_adaptive_parts = false;
@ -1004,17 +1005,18 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (part_disk_ptr->exists(marker_path))
{
/// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist
size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath());
LOG_WARNING(log,
"Detaching stale part {}{}, which should have been deleted after a move. That can only happen "
"after unclean restart of ClickHouse after move of a part having an operation blocking that "
"stale copy of part.",
getFullPathOnDisk(part_disk_ptr), part_name);
"Detaching stale part {}{} (size: {}), which should have been deleted after a move. "
"That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.",
getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part));
std::lock_guard loading_lock(mutex);
broken_parts_to_detach.push_back(part);
++suspicious_broken_parts;
suspicious_broken_parts_bytes += size_of_part;
return;
}
@ -1043,16 +1045,20 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
/// Ignore broken parts that can appear as a result of hard server restart.
if (broken)
{
LOG_ERROR(log,
"Detaching broken part {}{}. If it happened after update, it is likely because of backward "
"incompatibility. You need to resolve this manually",
getFullPathOnDisk(part_disk_ptr), part_name);
/// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist
size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath());
LOG_ERROR(log,
"Detaching broken part {}{} (size: {}). "
"If it happened after update, it is likely because of backward incompability. "
"You need to resolve this manually",
getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part));
std::lock_guard loading_lock(mutex);
broken_parts_to_detach.push_back(part);
++suspicious_broken_parts;
suspicious_broken_parts_bytes += size_of_part;
return;
}
@ -1099,8 +1105,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
has_non_adaptive_index_granularity_parts = has_non_adaptive_parts;
if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks)
throw Exception("Suspiciously many (" + toString(suspicious_broken_parts) + ") broken parts to remove.",
ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS,
"Suspiciously many ({}) broken parts to remove.",
suspicious_broken_parts);
if (suspicious_broken_parts_bytes > settings->max_suspicious_broken_parts_bytes && !skip_sanity_checks)
throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS,
"Suspiciously big size ({}) of all broken parts to remove.",
formatReadableSizeWithBinarySuffix(suspicious_broken_parts_bytes));
for (auto & part : broken_parts_to_detach)
part->renameToDetached("broken-on-start"); /// detached parts must not have '_' in prefixes

View File

@ -79,6 +79,7 @@ struct Settings;
M(Seconds, try_fetch_recompressed_part_timeout, 7200, "Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.", 0) \
M(Bool, always_fetch_merged_part, false, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \
M(UInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \
M(UInt64, max_suspicious_broken_parts_bytes, 1ULL * 1024 * 1024 * 1024, "Max size of all broken parts, if more - deny automatic deletion.", 0) \
M(UInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \
M(UInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \
M(Float, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \

View File

@ -0,0 +1,121 @@
# pylint: disable=unused-argument
# pylint: disable=redefined-outer-name
# pylint: disable=line-too-long
import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', stay_alive=True)
@pytest.fixture(scope='module', autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def break_part(table, part_name):
node.exec_in_container(['bash', '-c', f'rm /var/lib/clickhouse/data/default/{table}/{part_name}/columns.txt'])
def remove_part(table, part_name):
node.exec_in_container(['bash', '-c', f'rm -r /var/lib/clickhouse/data/default/{table}/{part_name}'])
def get_count(table):
return int(node.query(f'SELECT count() FROM {table}').strip())
def detach_table(table):
node.query(f'DETACH TABLE {table}')
def attach_table(table):
node.query(f'ATTACH TABLE {table}')
def check_table(table):
rows = 900
per_part_rows = 90
node.query(f'INSERT INTO {table} SELECT * FROM numbers(900)')
assert get_count(table) == rows
# break one part, and check that clickhouse will be alive
break_part(table, '0_1_1_0')
rows -= per_part_rows
detach_table(table)
attach_table(table)
assert get_count(table) == rows
# break two parts, and check that clickhouse will not start
break_part(table, '1_2_2_0')
break_part(table, '2_3_3_0')
rows -= per_part_rows*2
detach_table(table)
with pytest.raises(QueryRuntimeException):
attach_table(table)
# now remove one part, and check
remove_part(table, '1_2_2_0')
attach_table(table)
assert get_count(table) == rows
node.query(f'DROP TABLE {table}')
def test_max_suspicious_broken_parts():
node.query("""
CREATE TABLE test_max_suspicious_broken_parts (
key Int
)
ENGINE=MergeTree
ORDER BY key
PARTITION BY key%10
SETTINGS
max_suspicious_broken_parts = 1;
""")
check_table('test_max_suspicious_broken_parts')
def test_max_suspicious_broken_parts_bytes():
node.query("""
CREATE TABLE test_max_suspicious_broken_parts_bytes (
key Int
)
ENGINE=MergeTree
ORDER BY key
PARTITION BY key%10
SETTINGS
max_suspicious_broken_parts = 10,
/* one part takes ~751 byte, so we allow failure of one part with these limit */
max_suspicious_broken_parts_bytes = 1000;
""")
check_table('test_max_suspicious_broken_parts_bytes')
def test_max_suspicious_broken_parts__wide():
node.query("""
CREATE TABLE test_max_suspicious_broken_parts__wide (
key Int
)
ENGINE=MergeTree
ORDER BY key
PARTITION BY key%10
SETTINGS
min_bytes_for_wide_part = 0,
max_suspicious_broken_parts = 1;
""")
check_table('test_max_suspicious_broken_parts__wide')
def test_max_suspicious_broken_parts_bytes__wide():
node.query("""
CREATE TABLE test_max_suspicious_broken_parts_bytes__wide (
key Int
)
ENGINE=MergeTree
ORDER BY key
PARTITION BY key%10
SETTINGS
min_bytes_for_wide_part = 0,
max_suspicious_broken_parts = 10,
/* one part takes ~750 byte, so we allow failure of one part with these limit */
max_suspicious_broken_parts_bytes = 1000;
""")
check_table('test_max_suspicious_broken_parts_bytes__wide')