mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #12545 from ClickHouse/fix_load_part_without_checksums
Fix loading parts without checksums
This commit is contained in:
commit
40ee6eba19
@ -8,6 +8,7 @@
|
|||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Storages/MergeTree/MergeTreeData.h>
|
#include <Storages/MergeTree/MergeTreeData.h>
|
||||||
#include <Storages/MergeTree/localBackup.h>
|
#include <Storages/MergeTree/localBackup.h>
|
||||||
|
#include <Storages/MergeTree/checkDataPart.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
#include <Common/escapeForFileName.h>
|
#include <Common/escapeForFileName.h>
|
||||||
#include <common/JSON.h>
|
#include <common/JSON.h>
|
||||||
@ -521,7 +522,18 @@ void IMergeTreeDataPart::loadChecksums(bool require)
|
|||||||
if (require)
|
if (require)
|
||||||
throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2453,19 +2453,6 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
|
|||||||
|
|
||||||
part->loadColumnsChecksumsIndexes(false, true);
|
part->loadColumnsChecksumsIndexes(false, true);
|
||||||
part->modification_time = disk->getLastModified(full_part_path).epochTime();
|
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
|
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const
|
||||||
|
39
tests/integration/test_attach_without_checksums/test.py
Normal file
39
tests/integration/test_attach_without_checksums/test.py
Normal 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"
|
Loading…
Reference in New Issue
Block a user