2019-03-20 16:18:13 +00:00
|
|
|
#include <algorithm>
|
2017-12-03 00:46:34 +00:00
|
|
|
#include <optional>
|
|
|
|
|
|
|
|
#include <Poco/File.h>
|
2017-12-03 03:43:48 +00:00
|
|
|
#include <Poco/DirectoryIterator.h>
|
2017-12-03 00:46:34 +00:00
|
|
|
|
2019-03-30 13:44:23 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
|
2017-12-03 00:48:19 +00:00
|
|
|
#include <Storages/MergeTree/checkDataPart.h>
|
2020-01-13 14:53:32 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
2020-06-22 18:56:53 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
2018-12-28 18:15:26 +00:00
|
|
|
#include <Compression/CompressedReadBuffer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/HashingReadBuffer.h>
|
|
|
|
#include <Common/CurrentMetrics.h>
|
2014-07-22 08:21:16 +00:00
|
|
|
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const Metric ReplicatedChecks;
|
2016-10-24 04:06:27 +00:00
|
|
|
}
|
|
|
|
|
2014-07-22 08:21:16 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int CORRUPTED_DATA;
|
2020-01-17 12:24:27 +00:00
|
|
|
extern const int UNKNOWN_PART_TYPE;
|
2020-04-20 01:44:24 +00:00
|
|
|
extern const int MEMORY_LIMIT_EXCEEDED;
|
|
|
|
extern const int CANNOT_ALLOCATE_MEMORY;
|
|
|
|
extern const int CANNOT_MUNMAP;
|
|
|
|
extern const int CANNOT_MREMAP;
|
2021-02-10 14:12:49 +00:00
|
|
|
extern const int UNEXPECTED_FILE_IN_DATA_PART;
|
2020-04-20 01:44:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
bool isNotEnoughMemoryErrorCode(int code)
|
|
|
|
{
|
|
|
|
/// Don't count the part as broken if there is not enough memory to load it.
|
|
|
|
/// In fact, there can be many similar situations.
|
|
|
|
/// But it is OK, because there is a safety guard against deleting too many parts.
|
|
|
|
return code == ErrorCodes::MEMORY_LIMIT_EXCEEDED
|
|
|
|
|| code == ErrorCodes::CANNOT_ALLOCATE_MEMORY
|
|
|
|
|| code == ErrorCodes::CANNOT_MUNMAP
|
|
|
|
|| code == ErrorCodes::CANNOT_MREMAP;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-01-13 14:53:32 +00:00
|
|
|
IMergeTreeDataPart::Checksums checkDataPart(
|
2021-02-10 14:12:49 +00:00
|
|
|
MergeTreeData::DataPartPtr data_part,
|
2020-02-28 17:14:55 +00:00
|
|
|
const DiskPtr & disk,
|
|
|
|
const String & full_relative_path,
|
2020-01-13 14:53:32 +00:00
|
|
|
const NamesAndTypesList & columns_list,
|
|
|
|
const MergeTreeDataPartType & part_type,
|
2020-08-26 15:29:46 +00:00
|
|
|
const NameSet & files_without_checksums,
|
2017-12-03 00:46:34 +00:00
|
|
|
bool require_checksums,
|
|
|
|
std::function<bool()> is_cancelled)
|
2014-07-22 08:21:16 +00:00
|
|
|
{
|
2017-12-03 00:46:34 +00:00
|
|
|
/** Responsibility:
|
|
|
|
* - read list of columns from columns.txt;
|
|
|
|
* - read checksums if exist;
|
2020-01-13 14:53:32 +00:00
|
|
|
* - validate list of columns and checksums
|
2017-12-03 00:46:34 +00:00
|
|
|
*/
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedChecks};
|
|
|
|
|
2020-02-28 17:14:55 +00:00
|
|
|
String path = full_relative_path;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!path.empty() && path.back() != '/')
|
|
|
|
path += "/";
|
|
|
|
|
2020-01-13 14:53:32 +00:00
|
|
|
NamesAndTypesList columns_txt;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
{
|
2020-02-28 17:14:55 +00:00
|
|
|
auto buf = disk->readFile(path + "columns.txt");
|
|
|
|
columns_txt.readText(*buf);
|
|
|
|
assertEOF(*buf);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-01-13 14:53:32 +00:00
|
|
|
if (columns_txt != columns_list)
|
|
|
|
throw Exception("Columns doesn't match in part " + path
|
|
|
|
+ ". Expected: " + columns_list.toString()
|
|
|
|
+ ". Found: " + columns_txt.toString(), ErrorCodes::CORRUPTED_DATA);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-03 00:46:34 +00:00
|
|
|
/// Real checksums based on contents of data. Must correspond to checksums.txt. If not - it means the data is broken.
|
2020-01-13 14:53:32 +00:00
|
|
|
IMergeTreeDataPart::Checksums checksums_data;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-04-07 17:08:46 +00:00
|
|
|
/// This function calculates checksum for both compressed and decompressed contents of compressed file.
|
2020-02-28 17:14:55 +00:00
|
|
|
auto checksum_compressed_file = [](const DiskPtr & disk_, const String & file_path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-02-28 17:14:55 +00:00
|
|
|
auto file_buf = disk_->readFile(file_path);
|
|
|
|
HashingReadBuffer compressed_hashing_buf(*file_buf);
|
2020-01-13 14:53:32 +00:00
|
|
|
CompressedReadBuffer uncompressing_buf(compressed_hashing_buf);
|
|
|
|
HashingReadBuffer uncompressed_hashing_buf(uncompressing_buf);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-09-08 16:28:49 +00:00
|
|
|
uncompressed_hashing_buf.ignoreAll();
|
2020-01-13 14:53:32 +00:00
|
|
|
return IMergeTreeDataPart::Checksums::Checksum
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-01-13 14:53:32 +00:00
|
|
|
compressed_hashing_buf.count(), compressed_hashing_buf.getHash(),
|
|
|
|
uncompressed_hashing_buf.count(), uncompressed_hashing_buf.getHash()
|
|
|
|
};
|
|
|
|
};
|
2019-12-02 23:50:53 +00:00
|
|
|
|
2020-07-10 23:33:36 +00:00
|
|
|
/// This function calculates only checksum of file content (compressed or uncompressed).
|
2021-02-10 14:12:49 +00:00
|
|
|
/// It also calculates checksum of projections.
|
|
|
|
auto checksum_file = [&](const String & file_path, const String & file_name)
|
2020-07-10 23:33:36 +00:00
|
|
|
{
|
2021-02-10 14:12:49 +00:00
|
|
|
if (disk->isDirectory(file_path) && !startsWith(file_name, "tmp_")) // ignore projection tmp merge dir
|
|
|
|
{
|
|
|
|
auto pit = data_part->getProjectionParts().find(file_name);
|
|
|
|
if (pit == data_part->getProjectionParts().end())
|
|
|
|
{
|
|
|
|
if (require_checksums)
|
|
|
|
throw Exception("Unexpected file " + file_name + " in data part", ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART);
|
|
|
|
else
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
const auto & projection = pit->second;
|
|
|
|
IMergeTreeDataPart::Checksums projection_checksums_data;
|
|
|
|
const auto & projection_path = file_path;
|
|
|
|
const NamesAndTypesList & projection_columns_list = projection->getColumns();
|
|
|
|
for (const auto & projection_column : projection_columns_list)
|
|
|
|
{
|
|
|
|
auto serialization = IDataType::getSerialization(projection_column, [&](const String & stream_name)
|
|
|
|
{
|
|
|
|
return disk->exists(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION);
|
|
|
|
});
|
|
|
|
|
|
|
|
serialization->enumerateStreams(
|
|
|
|
[&](const ISerialization::SubstreamPath & substream_path)
|
|
|
|
{
|
|
|
|
String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin";
|
|
|
|
checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name);
|
|
|
|
},
|
|
|
|
{});
|
|
|
|
}
|
|
|
|
|
|
|
|
IMergeTreeDataPart::Checksums projection_checksums_txt;
|
|
|
|
|
|
|
|
if (require_checksums || disk->exists(projection_path + "checksums.txt"))
|
|
|
|
{
|
|
|
|
auto buf = disk->readFile(projection_path + "checksums.txt");
|
|
|
|
projection_checksums_txt.read(*buf);
|
|
|
|
assertEOF(*buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
const auto & projection_checksum_files_txt = projection_checksums_txt.files;
|
|
|
|
for (auto projection_it = disk->iterateDirectory(projection_path); projection_it->isValid(); projection_it->next())
|
|
|
|
{
|
|
|
|
const String & projection_file_name = projection_it->name();
|
|
|
|
auto projection_checksum_it = projection_checksums_data.files.find(projection_file_name);
|
|
|
|
|
|
|
|
/// Skip files that we already calculated. Also skip metadata files that are not checksummed.
|
|
|
|
if (projection_checksum_it == projection_checksums_data.files.end() && projection_file_name != "checksums.txt"
|
|
|
|
&& projection_file_name != "columns.txt")
|
|
|
|
{
|
|
|
|
auto projection_txt_checksum_it = projection_checksum_files_txt.find(file_name);
|
|
|
|
if (projection_txt_checksum_it == projection_checksum_files_txt.end()
|
|
|
|
|| projection_txt_checksum_it->second.uncompressed_size == 0)
|
|
|
|
{
|
|
|
|
auto projection_file_buf = disk->readFile(projection_it->path());
|
|
|
|
HashingReadBuffer projection_hashing_buf(*projection_file_buf);
|
|
|
|
projection_hashing_buf.ignoreAll();
|
|
|
|
projection_checksums_data.files[projection_file_name] = IMergeTreeDataPart::Checksums::Checksum(
|
|
|
|
projection_hashing_buf.count(), projection_hashing_buf.getHash());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
projection_checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_it->path());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(
|
|
|
|
projection_checksums_data.getTotalSizeOnDisk(), projection_checksums_data.getTotalChecksumUInt128());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto file_buf = disk->readFile(file_path);
|
|
|
|
HashingReadBuffer hashing_buf(*file_buf);
|
|
|
|
hashing_buf.ignoreAll();
|
|
|
|
checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash());
|
|
|
|
}
|
2020-07-10 23:33:36 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
bool check_uncompressed = true;
|
2020-04-07 17:08:46 +00:00
|
|
|
/// First calculate checksums for columns data
|
2020-01-13 14:53:32 +00:00
|
|
|
if (part_type == MergeTreeDataPartType::COMPACT)
|
|
|
|
{
|
|
|
|
const auto & file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION;
|
2021-02-10 14:12:49 +00:00
|
|
|
checksum_file(path + file_name, file_name);
|
2020-07-10 23:33:36 +00:00
|
|
|
/// Uncompressed checksums in compact parts are computed in a complex way.
|
|
|
|
/// We check only checksum of compressed file.
|
|
|
|
check_uncompressed = false;
|
2020-01-13 14:53:32 +00:00
|
|
|
}
|
|
|
|
else if (part_type == MergeTreeDataPartType::WIDE)
|
|
|
|
{
|
|
|
|
for (const auto & column : columns_list)
|
|
|
|
{
|
2021-03-09 14:46:52 +00:00
|
|
|
auto serialization = IDataType::getSerialization(column,
|
|
|
|
[&](const String & stream_name)
|
|
|
|
{
|
|
|
|
return disk->exists(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION);
|
|
|
|
});
|
|
|
|
|
|
|
|
serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
|
2020-01-13 14:53:32 +00:00
|
|
|
{
|
2021-03-09 14:46:52 +00:00
|
|
|
String file_name = ISerialization::getFileNameForStream(column, substream_path) + ".bin";
|
2020-02-28 17:14:55 +00:00
|
|
|
checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name);
|
2020-01-13 14:53:32 +00:00
|
|
|
}, {});
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-01-13 14:53:32 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
throw Exception("Unknown type in part " + path, ErrorCodes::UNKNOWN_PART_TYPE);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-04-07 17:08:46 +00:00
|
|
|
/// Checksums from the rest files listed in checksums.txt. May be absent. If present, they are subsequently compared with the actual data checksums.
|
2020-01-13 14:53:32 +00:00
|
|
|
IMergeTreeDataPart::Checksums checksums_txt;
|
2017-12-03 03:43:48 +00:00
|
|
|
|
2020-02-28 17:14:55 +00:00
|
|
|
if (require_checksums || disk->exists(path + "checksums.txt"))
|
2017-12-03 03:43:48 +00:00
|
|
|
{
|
2020-02-28 17:14:55 +00:00
|
|
|
auto buf = disk->readFile(path + "checksums.txt");
|
|
|
|
checksums_txt.read(*buf);
|
|
|
|
assertEOF(*buf);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-04-06 17:00:29 +00:00
|
|
|
const auto & checksum_files_txt = checksums_txt.files;
|
|
|
|
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
|
|
|
|
{
|
|
|
|
const String & file_name = it->name();
|
|
|
|
auto checksum_it = checksums_data.files.find(file_name);
|
2020-04-07 23:13:53 +00:00
|
|
|
|
2020-04-07 17:08:46 +00:00
|
|
|
/// Skip files that we already calculated. Also skip metadata files that are not checksummed.
|
2020-08-26 15:29:46 +00:00
|
|
|
if (checksum_it == checksums_data.files.end() && !files_without_checksums.count(file_name))
|
2020-04-06 17:00:29 +00:00
|
|
|
{
|
|
|
|
auto txt_checksum_it = checksum_files_txt.find(file_name);
|
|
|
|
if (txt_checksum_it == checksum_files_txt.end() || txt_checksum_it->second.uncompressed_size == 0)
|
|
|
|
{
|
2020-04-07 17:08:46 +00:00
|
|
|
/// The file is not compressed.
|
2021-02-10 14:12:49 +00:00
|
|
|
checksum_file(it->path(), file_name);
|
2020-04-06 17:00:29 +00:00
|
|
|
}
|
2021-02-10 14:12:49 +00:00
|
|
|
else /// If we have both compressed and uncompressed in txt, then calculate them
|
2020-04-06 17:00:29 +00:00
|
|
|
{
|
|
|
|
checksums_data.files[file_name] = checksum_compressed_file(disk, it->path());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-13 14:53:32 +00:00
|
|
|
if (is_cancelled())
|
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-03 00:46:34 +00:00
|
|
|
if (require_checksums || !checksums_txt.files.empty())
|
2020-07-10 23:33:36 +00:00
|
|
|
checksums_txt.checkEqual(checksums_data, check_uncompressed);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-03 00:46:34 +00:00
|
|
|
return checksums_data;
|
2014-07-22 08:21:16 +00:00
|
|
|
}
|
|
|
|
|
2020-06-22 18:56:53 +00:00
|
|
|
IMergeTreeDataPart::Checksums checkDataPartInMemory(const DataPartInMemoryPtr & data_part)
|
|
|
|
{
|
|
|
|
IMergeTreeDataPart::Checksums data_checksums;
|
|
|
|
data_checksums.files["data.bin"] = data_part->calculateBlockChecksum();
|
|
|
|
data_part->checksums.checkEqual(data_checksums, true);
|
|
|
|
return data_checksums;
|
|
|
|
}
|
|
|
|
|
2020-01-13 14:53:32 +00:00
|
|
|
IMergeTreeDataPart::Checksums checkDataPart(
|
2019-03-20 16:18:13 +00:00
|
|
|
MergeTreeData::DataPartPtr data_part,
|
|
|
|
bool require_checksums,
|
|
|
|
std::function<bool()> is_cancelled)
|
|
|
|
{
|
2020-06-22 18:56:53 +00:00
|
|
|
if (auto part_in_memory = asInMemoryPart(data_part))
|
|
|
|
return checkDataPartInMemory(part_in_memory);
|
2020-06-03 18:59:18 +00:00
|
|
|
|
2019-03-20 16:18:13 +00:00
|
|
|
return checkDataPart(
|
2021-02-10 14:12:49 +00:00
|
|
|
data_part,
|
2020-05-09 21:24:15 +00:00
|
|
|
data_part->volume->getDisk(),
|
2020-02-28 17:14:55 +00:00
|
|
|
data_part->getFullRelativePath(),
|
2020-01-16 16:15:01 +00:00
|
|
|
data_part->getColumns(),
|
2020-01-13 14:53:32 +00:00
|
|
|
data_part->getType(),
|
2020-08-26 15:29:46 +00:00
|
|
|
data_part->getFileNamesWithoutChecksums(),
|
2019-03-20 16:18:13 +00:00
|
|
|
require_checksums,
|
|
|
|
is_cancelled);
|
|
|
|
}
|
|
|
|
|
2014-07-22 08:21:16 +00:00
|
|
|
}
|