2019-10-10 16:30:30 +00:00
|
|
|
#include "MergeTreeDataPartWide.h"
|
2018-03-14 03:19:23 +00:00
|
|
|
|
2019-05-31 04:03:46 +00:00
|
|
|
#include <optional>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2018-12-28 18:15:26 +00:00
|
|
|
#include <Compression/CompressedReadBuffer.h>
|
|
|
|
#include <Compression/CompressedWriteBuffer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadBufferFromString.h>
|
|
|
|
#include <IO/WriteBufferFromString.h>
|
|
|
|
#include <IO/ReadBufferFromFile.h>
|
2017-08-31 15:40:34 +00:00
|
|
|
#include <IO/HashingWriteBuffer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Defines.h>
|
|
|
|
#include <Common/SipHash.h>
|
|
|
|
#include <Common/escapeForFileName.h>
|
2018-01-15 19:07:47 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2018-05-21 13:49:54 +00:00
|
|
|
#include <Common/localBackup.h>
|
2018-12-21 13:25:39 +00:00
|
|
|
#include <Compression/CompressionInfo.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2016-02-14 04:58:47 +00:00
|
|
|
#include <Poco/File.h>
|
2017-05-16 15:40:32 +00:00
|
|
|
#include <Poco/Path.h>
|
|
|
|
#include <Poco/DirectoryIterator.h>
|
|
|
|
#include <common/logger_useful.h>
|
2019-04-15 09:30:45 +00:00
|
|
|
#include <common/JSON.h>
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeReaderWide.h>
|
|
|
|
#include <Storages/MergeTree/IMergeTreeReader.h>
|
|
|
|
|
|
|
|
|
2016-02-14 04:58:47 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// namespace
|
|
|
|
// {
|
|
|
|
// }
|
|
|
|
|
2016-02-14 04:58:47 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int FILE_DOESNT_EXIST;
|
|
|
|
extern const int NO_FILE_IN_DATA_PART;
|
|
|
|
extern const int EXPECTED_END_OF_FILE;
|
2017-09-11 17:55:41 +00:00
|
|
|
extern const int CORRUPTED_DATA;
|
2017-11-20 19:33:12 +00:00
|
|
|
extern const int NOT_FOUND_EXPECTED_DATA_PART;
|
2018-03-21 20:21:34 +00:00
|
|
|
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
|
2019-04-15 09:30:45 +00:00
|
|
|
extern const int BAD_TTL_FILE;
|
2019-06-16 19:32:52 +00:00
|
|
|
extern const int CANNOT_UNLINK;
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-08-31 15:40:34 +00:00
|
|
|
static ReadBufferFromFile openForReading(const String & path)
|
|
|
|
{
|
|
|
|
return ReadBufferFromFile(path, std::min(static_cast<Poco::File::FileSize>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize()));
|
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
MergeTreeDataPartWide::MergeTreeDataPartWide(
|
|
|
|
MergeTreeData & storage_,
|
|
|
|
const String & name_,
|
2019-10-16 18:27:53 +00:00
|
|
|
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
2019-10-10 16:30:30 +00:00
|
|
|
const DiskSpace::DiskPtr & disk_,
|
|
|
|
const std::optional<String> & relative_path_)
|
2019-10-16 18:27:53 +00:00
|
|
|
: IMergeTreeDataPart(storage_, name_, index_granularity_info_, disk_, relative_path_)
|
2017-08-16 19:24:50 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
MergeTreeDataPartWide::MergeTreeDataPartWide(
|
|
|
|
const MergeTreeData & storage_,
|
|
|
|
const String & name_,
|
|
|
|
const MergeTreePartInfo & info_,
|
2019-10-16 18:27:53 +00:00
|
|
|
const MergeTreeIndexGranularityInfo & index_granularity_info_,
|
2019-10-10 16:30:30 +00:00
|
|
|
const DiskSpace::DiskPtr & disk_,
|
|
|
|
const std::optional<String> & relative_path_)
|
2019-10-16 18:27:53 +00:00
|
|
|
: IMergeTreeDataPart(storage_, name_, info_, index_granularity_info_, disk_, relative_path_)
|
2017-08-25 20:41:45 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader(
|
|
|
|
const NamesAndTypesList & columns_to_read,
|
|
|
|
const MarkRanges & mark_ranges,
|
|
|
|
UncompressedCache * uncompressed_cache,
|
|
|
|
MarkCache * mark_cache,
|
|
|
|
const ReaderSettings & reader_settings,
|
|
|
|
const ValueSizeMap & avg_value_size_hints,
|
|
|
|
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const
|
2019-03-22 12:56:58 +00:00
|
|
|
{
|
2019-10-10 16:30:30 +00:00
|
|
|
return std::make_unique<MergeTreeReaderWide>(shared_from_this(), columns_to_read, uncompressed_cache,
|
|
|
|
mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback);
|
2019-03-22 12:56:58 +00:00
|
|
|
}
|
|
|
|
|
2019-10-21 15:33:59 +00:00
|
|
|
IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter(
|
|
|
|
const NamesAndTypesList & columns_list,
|
|
|
|
const IColumn::Permutation * permutation,
|
|
|
|
const CompressionCodecPtr & default_codec,
|
|
|
|
const WriterSettings & writer_settings) const
|
|
|
|
{
|
|
|
|
UNUSED(columns_list);
|
|
|
|
UNUSED(permutation);
|
|
|
|
UNUSED(default_codec);
|
|
|
|
UNUSED(writer_settings);
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2019-03-22 12:56:58 +00:00
|
|
|
|
2018-03-26 14:18:04 +00:00
|
|
|
/// Takes into account the fact that several columns can e.g. share their .size substreams.
|
|
|
|
/// When calculating totals these should be counted only once.
|
2019-10-10 16:30:30 +00:00
|
|
|
ColumnSize MergeTreeDataPartWide::getColumnSizeImpl(
|
2019-01-04 12:10:00 +00:00
|
|
|
const String & column_name, const IDataType & type, std::unordered_set<String> * processed_substreams) const
|
2017-03-24 13:52:50 +00:00
|
|
|
{
|
2018-03-26 14:18:04 +00:00
|
|
|
ColumnSize size;
|
|
|
|
if (checksums.empty())
|
|
|
|
return size;
|
|
|
|
|
|
|
|
type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
|
|
|
{
|
2019-01-04 12:10:00 +00:00
|
|
|
String file_name = IDataType::getFileNameForStream(column_name, substream_path);
|
2018-03-26 14:18:04 +00:00
|
|
|
|
|
|
|
if (processed_substreams && !processed_substreams->insert(file_name).second)
|
|
|
|
return;
|
|
|
|
|
|
|
|
auto bin_checksum = checksums.files.find(file_name + ".bin");
|
|
|
|
if (bin_checksum != checksums.files.end())
|
|
|
|
{
|
|
|
|
size.data_compressed += bin_checksum->second.file_size;
|
|
|
|
size.data_uncompressed += bin_checksum->second.uncompressed_size;
|
|
|
|
}
|
|
|
|
|
2019-06-19 10:07:56 +00:00
|
|
|
auto mrk_checksum = checksums.files.find(file_name + index_granularity_info.marks_file_extension);
|
2018-03-26 14:18:04 +00:00
|
|
|
if (mrk_checksum != checksums.files.end())
|
|
|
|
size.marks += mrk_checksum->second.file_size;
|
|
|
|
}, {});
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2018-03-26 14:18:04 +00:00
|
|
|
return size;
|
2017-03-24 13:52:50 +00:00
|
|
|
}
|
2016-02-14 04:58:47 +00:00
|
|
|
|
|
|
|
/** Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
|
2017-06-05 20:43:23 +00:00
|
|
|
* If no checksums are present returns the name of the first physically existing column.
|
|
|
|
*/
|
2019-10-10 16:30:30 +00:00
|
|
|
String MergeTreeDataPartWide::getColumnNameWithMinumumCompressedSize() const
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2019-01-04 12:10:00 +00:00
|
|
|
const auto & storage_columns = storage.getColumns().getAllPhysical();
|
2017-04-01 07:20:54 +00:00
|
|
|
const std::string * minimum_size_column = nullptr;
|
2018-03-03 15:36:20 +00:00
|
|
|
UInt64 minimum_size = std::numeric_limits<UInt64>::max();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
for (const auto & column : storage_columns)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-06-16 19:32:52 +00:00
|
|
|
if (!hasColumnFiles(column.name, *column.type))
|
2017-04-01 07:20:54 +00:00
|
|
|
continue;
|
|
|
|
|
2019-10-16 18:27:53 +00:00
|
|
|
const auto size = getColumnSizeImpl(column.name, *column.type, nullptr).data_compressed;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (size < minimum_size)
|
|
|
|
{
|
|
|
|
minimum_size = size;
|
|
|
|
minimum_size_column = &column.name;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!minimum_size_column)
|
2017-08-17 10:28:57 +00:00
|
|
|
throw Exception("Could not find a column of minimum size in MergeTree, part " + getFullPath(), ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return *minimum_size_column;
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
MergeTreeDataPartWide::~MergeTreeDataPartWide()
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2019-09-04 18:26:18 +00:00
|
|
|
if (state == State::DeleteOnDestroy || is_temp)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2017-03-24 13:52:50 +00:00
|
|
|
std::string path = getFullPath();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Poco::File dir(path);
|
|
|
|
if (!dir.exists())
|
|
|
|
return;
|
|
|
|
|
2019-06-19 17:56:41 +00:00
|
|
|
if (is_temp)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-10-10 16:30:30 +00:00
|
|
|
String file_name = Poco::Path(relative_path).getFileName();
|
|
|
|
|
|
|
|
if (file_name.empty())
|
|
|
|
throw Exception("relative_path " + relative_path + " of part " + name + " is invalid or not set", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
if (!startsWith(file_name, "tmp"))
|
2019-06-19 17:56:41 +00:00
|
|
|
{
|
|
|
|
LOG_ERROR(storage.log, "~DataPart() should remove part " << path
|
|
|
|
<< " but its name doesn't start with tmp. Too suspicious, keeping the part.");
|
|
|
|
return;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
dir.remove(true);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
UInt64 MergeTreeDataPartWide::calculateTotalSizeOnDisk(const String & from)
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::File cur(from);
|
|
|
|
if (cur.isFile())
|
|
|
|
return cur.getSize();
|
|
|
|
std::vector<std::string> files;
|
|
|
|
cur.list(files);
|
2018-03-03 15:36:20 +00:00
|
|
|
UInt64 res = 0;
|
2017-10-03 19:04:56 +00:00
|
|
|
for (const auto & file : files)
|
2018-03-26 14:18:04 +00:00
|
|
|
res += calculateTotalSizeOnDisk(from + file);
|
2017-04-01 07:20:54 +00:00
|
|
|
return res;
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::remove() const
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2017-07-28 21:25:24 +00:00
|
|
|
if (relative_path.empty())
|
|
|
|
throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2019-02-13 12:08:51 +00:00
|
|
|
/** Atomic directory removal:
|
|
|
|
* - rename directory to temporary name;
|
|
|
|
* - remove it recursive.
|
|
|
|
*
|
|
|
|
* For temporary name we use "delete_tmp_" prefix.
|
|
|
|
*
|
|
|
|
* NOTE: We cannot use "tmp_delete_" prefix, because there is a second thread,
|
|
|
|
* that calls "clearOldTemporaryDirectories" and removes all directories, that begin with "tmp_" and are old enough.
|
|
|
|
* But when we removing data part, it can be old enough. And rename doesn't change mtime.
|
|
|
|
* And a race condition can happen that will lead to "File not found" error here.
|
|
|
|
*/
|
|
|
|
|
2019-04-21 18:38:44 +00:00
|
|
|
String full_path = storage.getFullPathOnDisk(disk);
|
2019-04-06 15:21:29 +00:00
|
|
|
String from = full_path + relative_path;
|
|
|
|
String to = full_path + "delete_tmp_" + name;
|
2019-07-30 17:24:40 +00:00
|
|
|
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Poco::File from_dir{from};
|
|
|
|
Poco::File to_dir{to};
|
|
|
|
|
|
|
|
if (to_dir.exists())
|
|
|
|
{
|
|
|
|
LOG_WARNING(storage.log, "Directory " << to << " (to which part must be renamed before removing) already exists."
|
|
|
|
" Most likely this is due to unclean restart. Removing it.");
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
to_dir.remove(true);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
LOG_ERROR(storage.log, "Cannot remove directory " << to << ". Check owner and access rights.");
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
from_dir.renameTo(to);
|
|
|
|
}
|
2018-08-10 04:02:56 +00:00
|
|
|
catch (const Poco::FileNotFoundException &)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-11-02 14:34:22 +00:00
|
|
|
LOG_ERROR(storage.log, "Directory " << from << " (part to remove) doesn't exist or one of nested files has gone."
|
2017-04-01 07:20:54 +00:00
|
|
|
" Most likely this is due to manual removing. This should be discouraged. Ignoring.");
|
|
|
|
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2019-06-16 19:32:52 +00:00
|
|
|
try
|
|
|
|
{
|
2019-06-16 19:59:30 +00:00
|
|
|
/// Remove each expected file in directory, then remove directory itself.
|
2019-06-16 19:32:52 +00:00
|
|
|
|
2019-06-17 16:50:31 +00:00
|
|
|
#if !__clang__
|
|
|
|
#pragma GCC diagnostic push
|
|
|
|
#pragma GCC diagnostic ignored "-Wunused-variable"
|
|
|
|
#endif
|
2019-08-16 00:49:33 +00:00
|
|
|
std::shared_lock<std::shared_mutex> lock(columns_lock);
|
|
|
|
|
2019-06-16 19:32:52 +00:00
|
|
|
for (const auto & [file, _] : checksums.files)
|
|
|
|
{
|
2019-06-16 19:42:05 +00:00
|
|
|
String path_to_remove = to + "/" + file;
|
|
|
|
if (0 != unlink(path_to_remove.c_str()))
|
2019-08-07 12:52:47 +00:00
|
|
|
throwFromErrnoWithPath("Cannot unlink file " + path_to_remove, path_to_remove,
|
|
|
|
ErrorCodes::CANNOT_UNLINK);
|
2019-06-16 19:32:52 +00:00
|
|
|
}
|
2019-06-17 16:50:31 +00:00
|
|
|
#if !__clang__
|
|
|
|
#pragma GCC diagnostic pop
|
|
|
|
#endif
|
2019-06-16 19:32:52 +00:00
|
|
|
|
2019-06-16 19:53:05 +00:00
|
|
|
for (const auto & file : {"checksums.txt", "columns.txt"})
|
2019-06-16 19:42:05 +00:00
|
|
|
{
|
2019-06-16 19:53:05 +00:00
|
|
|
String path_to_remove = to + "/" + file;
|
2019-06-16 19:42:05 +00:00
|
|
|
if (0 != unlink(path_to_remove.c_str()))
|
2019-08-07 12:52:47 +00:00
|
|
|
throwFromErrnoWithPath("Cannot unlink file " + path_to_remove, path_to_remove,
|
|
|
|
ErrorCodes::CANNOT_UNLINK);
|
2019-06-16 19:42:05 +00:00
|
|
|
}
|
2019-06-16 19:32:52 +00:00
|
|
|
|
|
|
|
if (0 != rmdir(to.c_str()))
|
2019-08-07 12:52:47 +00:00
|
|
|
throwFromErrnoWithPath("Cannot rmdir file " + to, to, ErrorCodes::CANNOT_UNLINK);
|
2019-06-16 19:32:52 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2019-06-16 19:59:30 +00:00
|
|
|
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
|
|
|
|
|
2019-06-16 19:49:12 +00:00
|
|
|
LOG_ERROR(storage.log, "Cannot quickly remove directory " << to << " by removing files; fallback to recursive removal. Reason: "
|
|
|
|
<< getCurrentExceptionMessage(false));
|
2019-06-16 19:59:30 +00:00
|
|
|
|
2019-06-16 19:32:52 +00:00
|
|
|
to_dir.remove(true);
|
|
|
|
}
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool /* check_consistency */)
|
2017-08-16 19:22:49 +00:00
|
|
|
{
|
2019-02-14 13:05:44 +00:00
|
|
|
/// Memory should not be limited during ATTACH TABLE query.
|
|
|
|
/// This is already true at the server startup but must be also ensured for manual table ATTACH.
|
|
|
|
/// Motivation: memory for index is shared between queries - not belong to the query itself.
|
|
|
|
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
|
|
|
|
|
2017-08-16 19:22:49 +00:00
|
|
|
loadColumns(require_columns_checksums);
|
|
|
|
loadChecksums(require_columns_checksums);
|
2019-03-25 13:55:24 +00:00
|
|
|
loadIndexGranularity();
|
|
|
|
loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
|
|
|
|
loadRowsCount(); /// Must be called after loadIndex() as it uses the value of `index_granularity`.
|
2017-08-18 19:46:26 +00:00
|
|
|
loadPartitionAndMinMaxIndex();
|
2019-04-15 09:30:45 +00:00
|
|
|
loadTTLInfos();
|
2019-10-10 16:30:30 +00:00
|
|
|
// if (check_consistency)
|
|
|
|
// checkConsistency(require_columns_checksums);
|
2017-08-16 19:22:49 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadIndexGranularity()
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2019-06-19 10:07:56 +00:00
|
|
|
String full_path = getFullPath();
|
2019-10-16 18:27:53 +00:00
|
|
|
index_granularity_info.changeGranularityIfRequired(full_path);
|
2019-06-19 10:07:56 +00:00
|
|
|
|
2018-11-15 14:06:54 +00:00
|
|
|
if (columns.empty())
|
|
|
|
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-15 14:06:54 +00:00
|
|
|
/// We can use any column, it doesn't matter
|
2019-06-19 12:30:56 +00:00
|
|
|
std::string marks_file_path = index_granularity_info.getMarksFilePath(full_path + escapeForFileName(columns.front().name));
|
2019-03-25 13:55:24 +00:00
|
|
|
if (!Poco::File(marks_file_path).exists())
|
|
|
|
throw Exception("Marks file '" + marks_file_path + "' doesn't exist", ErrorCodes::NO_FILE_IN_DATA_PART);
|
2018-11-15 14:06:54 +00:00
|
|
|
|
|
|
|
size_t marks_file_size = Poco::File(marks_file_path).getSize();
|
2018-11-12 17:44:43 +00:00
|
|
|
|
2019-06-19 10:07:56 +00:00
|
|
|
if (!index_granularity_info.is_adaptive)
|
2018-11-15 14:06:54 +00:00
|
|
|
{
|
2019-06-19 10:07:56 +00:00
|
|
|
size_t marks_count = marks_file_size / index_granularity_info.mark_size_in_bytes;
|
|
|
|
index_granularity.resizeWithFixedGranularity(marks_count, index_granularity_info.fixed_index_granularity); /// all the same
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-11-15 14:06:54 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
ReadBufferFromFile buffer(marks_file_path, marks_file_size, -1);
|
|
|
|
while (!buffer.eof())
|
|
|
|
{
|
|
|
|
buffer.seek(sizeof(size_t) * 2, SEEK_CUR); /// skip offset_in_compressed file and offset_in_decompressed_block
|
2019-03-25 13:55:24 +00:00
|
|
|
size_t granularity;
|
|
|
|
readIntBinary(granularity, buffer);
|
|
|
|
index_granularity.appendMark(granularity);
|
2018-11-15 14:06:54 +00:00
|
|
|
}
|
2019-06-19 10:07:56 +00:00
|
|
|
if (index_granularity.getMarksCount() * index_granularity_info.mark_size_in_bytes != marks_file_size)
|
2018-11-15 14:06:54 +00:00
|
|
|
throw Exception("Cannot read all marks from file " + marks_file_path, ErrorCodes::CANNOT_READ_ALL_DATA);
|
|
|
|
}
|
2019-03-26 09:12:48 +00:00
|
|
|
index_granularity.setInitialized();
|
2018-11-15 14:06:54 +00:00
|
|
|
}
|
2017-08-16 19:22:49 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadIndex()
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2019-03-26 09:12:48 +00:00
|
|
|
/// It can be empty in case of mutations
|
|
|
|
if (!index_granularity.isInitialized())
|
2019-03-25 13:55:24 +00:00
|
|
|
throw Exception("Index granularity is not loaded before index loading", ErrorCodes::LOGICAL_ERROR);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-11 14:53:23 +00:00
|
|
|
size_t key_size = storage.primary_key_columns.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (key_size)
|
|
|
|
{
|
2017-12-15 20:48:46 +00:00
|
|
|
MutableColumns loaded_index;
|
|
|
|
loaded_index.resize(key_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < key_size; ++i)
|
|
|
|
{
|
2017-12-15 20:48:46 +00:00
|
|
|
loaded_index[i] = storage.primary_key_data_types[i]->createColumn();
|
2019-03-25 13:55:24 +00:00
|
|
|
loaded_index[i]->reserve(index_granularity.getMarksCount());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-03-24 13:52:50 +00:00
|
|
|
String index_path = getFullPath() + "primary.idx";
|
2017-08-30 20:23:29 +00:00
|
|
|
ReadBufferFromFile index_file = openForReading(index_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-03-25 13:55:24 +00:00
|
|
|
for (size_t i = 0; i < index_granularity.getMarksCount(); ++i) //-V756
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t j = 0; j < key_size; ++j)
|
2019-03-10 03:16:51 +00:00
|
|
|
storage.primary_key_data_types[j]->deserializeBinary(*loaded_index[j], index_file);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < key_size; ++i)
|
2019-03-10 03:16:51 +00:00
|
|
|
{
|
|
|
|
loaded_index[i]->protect();
|
2019-03-25 13:55:24 +00:00
|
|
|
if (loaded_index[i]->size() != index_granularity.getMarksCount())
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Cannot read all data from index file " + index_path
|
2019-03-25 13:55:24 +00:00
|
|
|
+ "(expected size: " + toString(index_granularity.getMarksCount()) + ", read: " + toString(loaded_index[i]->size()) + ")",
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::CANNOT_READ_ALL_DATA);
|
2019-03-10 03:16:51 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!index_file.eof())
|
|
|
|
throw Exception("Index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE);
|
2017-12-15 20:48:46 +00:00
|
|
|
|
|
|
|
index.assign(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end()));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadPartitionAndMinMaxIndex()
|
2017-08-16 19:24:50 +00:00
|
|
|
{
|
2017-09-07 16:21:06 +00:00
|
|
|
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
2017-08-25 20:41:45 +00:00
|
|
|
{
|
2018-05-25 13:29:15 +00:00
|
|
|
DayNum min_date;
|
|
|
|
DayNum max_date;
|
2017-08-25 20:41:45 +00:00
|
|
|
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
|
2017-08-21 15:35:29 +00:00
|
|
|
|
2017-08-25 20:41:45 +00:00
|
|
|
const auto & date_lut = DateLUT::instance();
|
2017-09-11 17:55:41 +00:00
|
|
|
partition = MergeTreePartition(date_lut.toNumYYYYMM(min_date));
|
2017-08-31 15:40:34 +00:00
|
|
|
minmax_idx = MinMaxIndex(min_date, max_date);
|
2017-08-25 20:41:45 +00:00
|
|
|
}
|
|
|
|
else
|
2017-08-30 20:23:29 +00:00
|
|
|
{
|
2019-04-06 15:21:29 +00:00
|
|
|
String path = getFullPath();
|
|
|
|
partition.load(storage, path);
|
2018-05-23 19:34:37 +00:00
|
|
|
if (!isEmpty())
|
2019-04-06 15:21:29 +00:00
|
|
|
minmax_idx.load(storage, path);
|
2017-08-30 20:23:29 +00:00
|
|
|
}
|
2017-09-11 17:55:41 +00:00
|
|
|
|
2019-01-25 15:17:12 +00:00
|
|
|
String calculated_partition_id = partition.getID(storage.partition_key_sample);
|
2017-09-11 17:55:41 +00:00
|
|
|
if (calculated_partition_id != info.partition_id)
|
|
|
|
throw Exception(
|
2018-11-26 00:56:50 +00:00
|
|
|
"While loading part " + getFullPath() + ": calculated partition ID: " + calculated_partition_id
|
2017-09-11 17:55:41 +00:00
|
|
|
+ " differs from partition ID in part name: " + info.partition_id,
|
|
|
|
ErrorCodes::CORRUPTED_DATA);
|
2017-08-16 19:24:50 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadChecksums(bool require)
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2017-03-24 13:52:50 +00:00
|
|
|
String path = getFullPath() + "checksums.txt";
|
2018-07-08 03:56:24 +00:00
|
|
|
Poco::File checksums_file(path);
|
|
|
|
if (checksums_file.exists())
|
|
|
|
{
|
|
|
|
ReadBufferFromFile file = openForReading(path);
|
|
|
|
if (checksums.read(file))
|
|
|
|
{
|
|
|
|
assertEOF(file);
|
|
|
|
bytes_on_disk = checksums.getTotalSizeOnDisk();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
bytes_on_disk = calculateTotalSizeOnDisk(getFullPath());
|
|
|
|
}
|
|
|
|
else
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (require)
|
|
|
|
throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
|
2018-07-08 03:56:24 +00:00
|
|
|
bytes_on_disk = calculateTotalSizeOnDisk(getFullPath());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadRowsCount()
|
2017-10-24 14:11:53 +00:00
|
|
|
{
|
2019-03-25 13:55:24 +00:00
|
|
|
if (index_granularity.empty())
|
2018-05-23 19:34:37 +00:00
|
|
|
{
|
|
|
|
rows_count = 0;
|
|
|
|
}
|
|
|
|
else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
2017-10-24 14:11:53 +00:00
|
|
|
{
|
|
|
|
String path = getFullPath() + "count.txt";
|
|
|
|
if (!Poco::File(path).exists())
|
|
|
|
throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
|
|
|
|
ReadBufferFromFile file = openForReading(path);
|
|
|
|
readIntText(rows_count, file);
|
|
|
|
assertEOF(file);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (const NameAndTypePair & column : columns)
|
|
|
|
{
|
|
|
|
ColumnPtr column_col = column.type->createColumn();
|
2018-12-10 17:47:18 +00:00
|
|
|
if (!column_col->isFixedAndContiguous() || column_col->lowCardinality())
|
2018-03-26 14:18:04 +00:00
|
|
|
continue;
|
2017-10-24 14:11:53 +00:00
|
|
|
|
2019-10-16 18:27:53 +00:00
|
|
|
size_t column_size = getColumnSizeImpl(column.name, *column.type, nullptr).data_uncompressed;
|
2018-03-26 14:18:04 +00:00
|
|
|
if (!column_size)
|
2017-10-24 14:11:53 +00:00
|
|
|
continue;
|
|
|
|
|
2017-12-09 10:14:45 +00:00
|
|
|
size_t sizeof_field = column_col->sizeOfValueIfFixed();
|
2018-03-26 14:18:04 +00:00
|
|
|
rows_count = column_size / sizeof_field;
|
2017-10-24 14:11:53 +00:00
|
|
|
|
2018-03-26 14:18:04 +00:00
|
|
|
if (column_size % sizeof_field != 0)
|
2017-10-24 14:11:53 +00:00
|
|
|
{
|
|
|
|
throw Exception(
|
2018-03-26 14:18:04 +00:00
|
|
|
"Uncompressed size of column " + column.name + "(" + toString(column_size)
|
|
|
|
+ ") is not divisible by the size of value (" + toString(sizeof_field) + ")",
|
2017-10-24 14:11:53 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
2019-06-19 12:30:56 +00:00
|
|
|
size_t last_mark_index_granularity = index_granularity.getLastNonFinalMarkRows();
|
2019-03-25 13:55:24 +00:00
|
|
|
size_t rows_approx = index_granularity.getTotalRows();
|
2018-11-15 14:06:54 +00:00
|
|
|
if (!(rows_count <= rows_approx && rows_approx < rows_count + last_mark_index_granularity))
|
2017-10-24 14:11:53 +00:00
|
|
|
throw Exception(
|
2018-01-15 17:30:16 +00:00
|
|
|
"Unexpected size of column " + column.name + ": " + toString(rows_count) + " rows, expected "
|
2018-11-15 14:06:54 +00:00
|
|
|
+ toString(rows_approx) + "+-" + toString(last_mark_index_granularity) + " rows according to the index",
|
2017-10-24 14:11:53 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
throw Exception("Data part doesn't contain fixed size column (even Date column)", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadTTLInfos()
|
2019-04-15 09:30:45 +00:00
|
|
|
{
|
|
|
|
String path = getFullPath() + "ttl.txt";
|
|
|
|
if (Poco::File(path).exists())
|
|
|
|
{
|
|
|
|
ReadBufferFromFile in = openForReading(path);
|
|
|
|
assertString("ttl format version: ", in);
|
|
|
|
size_t format_version;
|
|
|
|
readText(format_version, in);
|
|
|
|
assertChar('\n', in);
|
|
|
|
|
|
|
|
if (format_version == 1)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
ttl_infos.read(in);
|
|
|
|
}
|
|
|
|
catch (const JSONException &)
|
|
|
|
{
|
|
|
|
throw Exception("Error while parsing file ttl.txt in part: " + name, ErrorCodes::BAD_TTL_FILE);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
throw Exception("Unknown ttl format version: " + toString(format_version), ErrorCodes::BAD_TTL_FILE);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// void MergeTreeDataPartWide::accumulateColumnSizes(ColumnToSize & column_to_size) const
|
|
|
|
// {
|
|
|
|
// std::shared_lock<std::shared_mutex> part_lock(columns_lock);
|
2017-12-02 22:12:27 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// for (const NameAndTypePair & name_type : storage.getColumns().getAllPhysical())
|
|
|
|
// {
|
|
|
|
// IDataType::SubstreamPath path;
|
|
|
|
// name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
|
|
|
// {
|
|
|
|
// Poco::File bin_file(getFullPath() + IDataType::getFileNameForStream(name_type.name, substream_path) + ".bin");
|
|
|
|
// if (bin_file.exists())
|
|
|
|
// column_to_size[name_type.name] += bin_file.getSize();
|
|
|
|
// }, path);
|
|
|
|
// }
|
|
|
|
// }
|
2016-02-14 04:58:47 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::loadColumns(bool require)
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2017-03-24 13:52:50 +00:00
|
|
|
String path = getFullPath() + "columns.txt";
|
2019-05-31 04:03:46 +00:00
|
|
|
Poco::File poco_file_path{path};
|
|
|
|
if (!poco_file_path.exists())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (require)
|
|
|
|
throw Exception("No columns.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
|
|
|
|
/// If there is no file with a list of columns, write it down.
|
2018-03-13 15:00:28 +00:00
|
|
|
for (const NameAndTypePair & column : storage.getColumns().getAllPhysical())
|
2017-03-24 13:52:50 +00:00
|
|
|
if (Poco::File(getFullPath() + escapeForFileName(column.name) + ".bin").exists())
|
2017-04-01 07:20:54 +00:00
|
|
|
columns.push_back(column);
|
|
|
|
|
|
|
|
if (columns.empty())
|
|
|
|
throw Exception("No columns in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
|
|
|
|
{
|
|
|
|
WriteBufferFromFile out(path + ".tmp", 4096);
|
|
|
|
columns.writeText(out);
|
|
|
|
}
|
|
|
|
Poco::File(path + ".tmp").renameTo(path);
|
|
|
|
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2019-05-31 04:03:46 +00:00
|
|
|
is_frozen = !poco_file_path.canWrite();
|
|
|
|
|
2017-08-30 20:23:29 +00:00
|
|
|
ReadBufferFromFile file = openForReading(path);
|
2017-04-01 07:20:54 +00:00
|
|
|
columns.readText(file);
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata)
|
2016-02-14 04:58:47 +00:00
|
|
|
{
|
2017-03-24 13:52:50 +00:00
|
|
|
String path = getFullPath();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!checksums.empty())
|
|
|
|
{
|
2018-10-11 14:53:23 +00:00
|
|
|
if (!storage.primary_key_columns.empty() && !checksums.files.count("primary.idx"))
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
|
|
|
|
if (require_part_metadata)
|
|
|
|
{
|
2017-12-02 22:12:27 +00:00
|
|
|
for (const NameAndTypePair & name_type : columns)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-06-07 18:14:37 +00:00
|
|
|
IDataType::SubstreamPath stream_path;
|
2017-12-02 22:12:27 +00:00
|
|
|
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
|
|
|
{
|
|
|
|
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
|
2019-06-19 10:07:56 +00:00
|
|
|
String mrk_file_name = file_name + index_granularity_info.marks_file_extension;
|
2017-12-02 22:12:27 +00:00
|
|
|
String bin_file_name = file_name + ".bin";
|
|
|
|
if (!checksums.files.count(mrk_file_name))
|
2018-09-19 13:04:05 +00:00
|
|
|
throw Exception("No " + mrk_file_name + " file checksum for column " + name_type.name + " in part " + path,
|
2017-12-02 22:12:27 +00:00
|
|
|
ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
if (!checksums.files.count(bin_file_name))
|
2018-09-19 13:04:05 +00:00
|
|
|
throw Exception("No " + bin_file_name + " file checksum for column " + name_type.name + " in part " + path,
|
2017-12-02 22:12:27 +00:00
|
|
|
ErrorCodes::NO_FILE_IN_DATA_PART);
|
2018-06-07 18:14:37 +00:00
|
|
|
}, stream_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-09-07 16:21:06 +00:00
|
|
|
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
2017-08-30 20:23:29 +00:00
|
|
|
{
|
2017-10-24 14:11:53 +00:00
|
|
|
if (!checksums.files.count("count.txt"))
|
|
|
|
throw Exception("No checksum for count.txt", ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
|
2018-11-06 18:25:36 +00:00
|
|
|
if (storage.partition_key_expr && !checksums.files.count("partition.dat"))
|
2017-08-30 20:23:29 +00:00
|
|
|
throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
|
2018-05-23 19:34:37 +00:00
|
|
|
if (!isEmpty())
|
2017-08-30 20:23:29 +00:00
|
|
|
{
|
2018-05-23 19:34:37 +00:00
|
|
|
for (const String & col_name : storage.minmax_idx_columns)
|
|
|
|
{
|
|
|
|
if (!checksums.files.count("minmax_" + escapeForFileName(col_name) + ".idx"))
|
|
|
|
throw Exception("No minmax idx file checksum for column " + col_name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
|
|
}
|
2017-08-30 20:23:29 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-03-24 13:52:50 +00:00
|
|
|
checksums.checkSizes(path);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2017-08-30 20:23:29 +00:00
|
|
|
auto check_file_not_empty = [&path](const String & file_path)
|
|
|
|
{
|
|
|
|
Poco::File file(file_path);
|
|
|
|
if (!file.exists() || file.getSize() == 0)
|
|
|
|
throw Exception("Part " + path + " is broken: " + file_path + " is empty", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
|
|
|
return file.getSize();
|
|
|
|
};
|
|
|
|
|
|
|
|
/// Check that the primary key index is not empty.
|
2018-10-11 14:53:23 +00:00
|
|
|
if (!storage.primary_key_columns.empty())
|
2017-08-30 20:23:29 +00:00
|
|
|
check_file_not_empty(path + "primary.idx");
|
|
|
|
|
2017-09-07 16:21:06 +00:00
|
|
|
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-10-24 14:11:53 +00:00
|
|
|
check_file_not_empty(path + "count.txt");
|
|
|
|
|
2018-11-06 18:25:36 +00:00
|
|
|
if (storage.partition_key_expr)
|
2017-08-30 20:23:29 +00:00
|
|
|
check_file_not_empty(path + "partition.dat");
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-30 20:23:29 +00:00
|
|
|
for (const String & col_name : storage.minmax_idx_columns)
|
|
|
|
check_file_not_empty(path + "minmax_" + escapeForFileName(col_name) + ".idx");
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Check that all marks are nonempty and have the same size.
|
2017-12-02 22:12:27 +00:00
|
|
|
|
2018-03-03 15:36:20 +00:00
|
|
|
std::optional<UInt64> marks_size;
|
2017-12-02 22:12:27 +00:00
|
|
|
for (const NameAndTypePair & name_type : columns)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-02 22:12:27 +00:00
|
|
|
name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-06-19 10:07:56 +00:00
|
|
|
Poco::File file(IDataType::getFileNameForStream(name_type.name, substream_path) + index_granularity_info.marks_file_extension);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-02 22:12:27 +00:00
|
|
|
/// Missing file is Ok for case when new column was added.
|
|
|
|
if (file.exists())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-03-03 15:36:20 +00:00
|
|
|
UInt64 file_size = file.getSize();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-02 22:12:27 +00:00
|
|
|
if (!file_size)
|
|
|
|
throw Exception("Part " + path + " is broken: " + file.path() + " is empty.",
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
2017-12-02 22:12:27 +00:00
|
|
|
|
|
|
|
if (!marks_size)
|
|
|
|
marks_size = file_size;
|
|
|
|
else if (file_size != *marks_size)
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Part " + path + " is broken: marks have different sizes.",
|
|
|
|
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
|
|
|
}
|
2018-06-07 18:14:37 +00:00
|
|
|
});
|
2017-12-02 22:12:27 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// bool MergeTreeDataPartWide::hasColumnFiles(const String & column_name, const IDataType & type) const
|
|
|
|
// {
|
|
|
|
// bool res = true;
|
2019-01-05 03:33:22 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
|
|
|
// {
|
|
|
|
// String file_name = IDataType::getFileNameForStream(column_name, substream_path);
|
2017-09-11 22:40:51 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// auto bin_checksum = checksums.files.find(file_name + ".bin");
|
|
|
|
// auto mrk_checksum = checksums.files.find(file_name + index_granularity_info.marks_file_extension);
|
2017-09-11 22:40:51 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// if (bin_checksum == checksums.files.end() || mrk_checksum == checksums.files.end())
|
|
|
|
// res = false;
|
|
|
|
// }, {});
|
2017-11-20 19:33:12 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
// return res;
|
|
|
|
// }
|
2017-11-20 19:33:12 +00:00
|
|
|
|
2016-02-14 04:58:47 +00:00
|
|
|
}
|