Merge pull request #12545 from ClickHouse/fix_load_part_without_checksums

Fix loading parts without checksums
This commit is contained in:
Alexander Kuzmenkov 2020-07-20 14:40:53 +03:00 committed by GitHub
commit 40ee6eba19
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 52 additions and 14 deletions

View File

@ -8,6 +8,7 @@
#include <IO/WriteHelpers.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/localBackup.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <common/JSON.h>
@ -521,7 +522,18 @@ void IMergeTreeDataPart::loadChecksums(bool require)
if (require)
throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
bytes_on_disk = calculateTotalSizeOnDisk(volume->getDisk(), getFullRelativePath());
/// If the checksums file is not present, calculate the checksums and write them to disk.
/// Check the data while we are at it.
LOG_WARNING(storage.log, "Checksums for part {} not found. Will calculate them from data on disk.", name);
checksums = checkDataPart(shared_from_this(), false);
{
auto out = volume->getDisk()->writeFile(getFullRelativePath() + "checksums.txt.tmp", 4096);
checksums.write(*out);
}
volume->getDisk()->moveFile(getFullRelativePath() + "checksums.txt.tmp", getFullRelativePath() + "checksums.txt");
bytes_on_disk = checksums.getTotalSizeOnDisk();
}
}

View File

@ -2453,19 +2453,6 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
part->loadColumnsChecksumsIndexes(false, true);
part->modification_time = disk->getLastModified(full_part_path).epochTime();
/// If the checksums file is not present, calculate the checksums and write them to disk.
/// Check the data while we are at it.
if (part->checksums.empty())
{
part->checksums = checkDataPart(part, false);
{
auto out = disk->writeFile(full_part_path + "checksums.txt.tmp", 4096);
part->checksums.write(*out);
}
disk->moveFile(full_part_path + "checksums.txt.tmp", full_part_path + "checksums.txt");
}
}
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const

View File

@ -0,0 +1,39 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1')
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_attach_without_checksums(start_cluster):
node1.query("CREATE TABLE test (date Date, key Int32, value String) Engine=MergeTree ORDER BY key PARTITION by date")
node1.query("INSERT INTO test SELECT toDate('2019-10-01'), number, toString(number) FROM numbers(100)")
assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "10\n"
node1.query("ALTER TABLE test DETACH PARTITION '2019-10-01'")
assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "0\n"
assert node1.query("SELECT COUNT() FROM test") == "0\n"
# to be sure output not empty
node1.exec_in_container(['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" | grep -e ".*" '], privileged=True, user='root')
node1.exec_in_container(['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" -delete'], privileged=True, user='root')
node1.query("ALTER TABLE test ATTACH PARTITION '2019-10-01'")
assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "10\n"
assert node1.query("SELECT COUNT() FROM test") == "100\n"