ClickHouse/src/Storages/MergeTree/MergeTreeDataPartWide.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

304 lines
12 KiB
C++
Raw Normal View History

2019-10-10 16:30:30 +00:00
#include "MergeTreeDataPartWide.h"
#include <Storages/MergeTree/MergeTreeReaderWide.h>
2019-10-21 17:23:06 +00:00
#include <Storages/MergeTree/MergeTreeDataPartWriterWide.h>
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
2022-09-05 16:55:00 +00:00
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
2020-11-10 17:32:00 +00:00
#include <DataTypes/NestedUtils.h>
#include <Core/NamesAndTypes.h>
2019-10-10 16:30:30 +00:00
2016-02-14 04:58:47 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NO_FILE_IN_DATA_PART;
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
2020-07-25 20:56:29 +00:00
extern const int LOGICAL_ERROR;
2016-02-14 04:58:47 +00:00
}
2019-10-10 16:30:30 +00:00
MergeTreeDataPartWide::MergeTreeDataPartWide(
const MergeTreeData & storage_,
const String & name_,
const MergeTreePartInfo & info_,
2022-10-22 22:51:59 +00:00
const MutableDataPartStoragePtr & data_part_storage_,
const IMergeTreeDataPart * parent_part_)
: IMergeTreeDataPart(storage_, name_, info_, data_part_storage_, Type::Wide, parent_part_)
{
}
2019-10-10 16:30:30 +00:00
IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader(
const NamesAndTypesList & columns_to_read,
const StorageMetadataPtr & metadata_snapshot,
2019-10-10 16:30:30 +00:00
const MarkRanges & mark_ranges,
UncompressedCache * uncompressed_cache,
MarkCache * mark_cache,
const MergeTreeReaderSettings & reader_settings,
2019-10-10 16:30:30 +00:00
const ValueSizeMap & avg_value_size_hints,
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const
2019-03-22 12:56:58 +00:00
{
2022-09-05 16:55:00 +00:00
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this());
return std::make_unique<MergeTreeReaderWide>(
2022-09-05 16:55:00 +00:00
read_info, columns_to_read,
2022-07-22 23:05:11 +00:00
metadata_snapshot, 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,
2020-06-17 12:39:20 +00:00
const StorageMetadataPtr & metadata_snapshot,
2019-11-07 11:11:38 +00:00
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & writer_settings,
2022-10-22 22:51:59 +00:00
const MergeTreeIndexGranularity & computed_index_granularity)
2019-10-21 15:33:59 +00:00
{
2019-10-21 17:23:06 +00:00
return std::make_unique<MergeTreeDataPartWriterWide>(
2022-10-22 22:51:59 +00:00
shared_from_this(), columns_list,
metadata_snapshot, indices_to_recalc,
2022-09-05 05:26:58 +00:00
getMarksFileExtension(),
2021-10-29 17:21:02 +00:00
default_codec_, writer_settings, computed_index_granularity);
}
2019-10-21 15:33:59 +00:00
2019-03-22 12:56:58 +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(
const NameAndTypePair & column, std::unordered_set<String> * processed_substreams) const
{
ColumnSize size;
if (checksums.empty())
return size;
2022-07-27 14:05:16 +00:00
getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{
2021-03-09 14:46:52 +00:00
String file_name = ISerialization::getFileNameForStream(column, substream_path);
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;
}
2022-09-05 05:26:58 +00:00
auto mrk_checksum = checksums.files.find(file_name + getMarksFileExtension());
if (mrk_checksum != checksums.files.end())
size.marks += mrk_checksum->second.file_size;
});
2016-02-14 04:58:47 +00:00
return size;
}
2016-02-14 04:58:47 +00:00
2022-09-05 16:55:00 +00:00
void MergeTreeDataPartWide::loadIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_,
const IDataPartStorage & data_part_storage_, const std::string & any_column_file_name)
2016-02-14 04:58:47 +00:00
{
2022-09-05 16:55:00 +00:00
index_granularity_info_.changeGranularityIfRequired(data_part_storage_);
2018-11-15 14:06:54 +00:00
/// We can use any column, it doesn't matter
2022-09-05 16:55:00 +00:00
std::string marks_file_path = index_granularity_info_.getMarksFilePath(any_column_file_name);
if (!data_part_storage_.exists(marks_file_path))
2022-04-07 17:44:49 +00:00
throw Exception(
ErrorCodes::NO_FILE_IN_DATA_PART, "Marks file '{}' doesn't exist",
std::string(fs::path(data_part_storage_.getFullPath()) / marks_file_path));
2018-11-15 14:06:54 +00:00
size_t marks_file_size = data_part_storage_.getFileSize(marks_file_path);
2022-05-29 07:28:02 +00:00
if (!index_granularity_info_.mark_type.adaptive && !index_granularity_info_.mark_type.compressed)
2022-09-05 05:26:58 +00:00
{
/// The most easy way - no need to read the file, everything is known from its size.
2022-09-05 16:55:00 +00:00
size_t marks_count = marks_file_size / index_granularity_info_.getMarkSizeInBytes();
index_granularity_.resizeWithFixedGranularity(marks_count, index_granularity_info_.fixed_index_granularity); /// all the same
2016-02-14 04:58:47 +00:00
}
2018-11-15 14:06:54 +00:00
else
{
auto marks_file = data_part_storage_.readFile(marks_file_path, ReadSettings().adjustBufferSize(marks_file_size), marks_file_size, std::nullopt);
2022-09-05 05:26:58 +00:00
std::unique_ptr<ReadBuffer> marks_reader;
if (!index_granularity_info_.mark_type.compressed)
2022-09-05 05:26:58 +00:00
marks_reader = std::move(marks_file);
else
marks_reader = std::make_unique<CompressedReadBufferFromFile>(std::move(marks_file));
2022-05-29 07:28:02 +00:00
size_t marks_count = 0;
2022-09-05 05:26:58 +00:00
while (!marks_reader->eof())
2018-11-15 14:06:54 +00:00
{
2022-09-05 05:26:58 +00:00
MarkInCompressedFile mark;
size_t granularity;
readBinary(mark.offset_in_compressed_file, *marks_reader);
readBinary(mark.offset_in_decompressed_block, *marks_reader);
2022-05-29 07:28:02 +00:00
++marks_count;
if (index_granularity_info_.mark_type.adaptive)
2022-05-29 07:28:02 +00:00
{
2022-09-05 05:26:58 +00:00
readIntBinary(granularity, *marks_reader);
index_granularity_.appendMark(granularity);
2022-05-29 07:28:02 +00:00
}
2018-11-15 14:06:54 +00:00
}
2019-10-31 14:44:17 +00:00
if (!index_granularity_info_.mark_type.adaptive)
index_granularity_.resizeWithFixedGranularity(marks_count, index_granularity_info_.fixed_index_granularity); /// all the same
2018-11-15 14:06:54 +00:00
}
2016-02-14 04:58:47 +00:00
2022-09-05 16:55:00 +00:00
index_granularity_.setInitialized();
}
void MergeTreeDataPartWide::loadIndexGranularity()
{
if (columns.empty())
throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No columns in part {}", name);
2022-09-05 16:55:00 +00:00
loadIndexGranularityImpl(index_granularity, index_granularity_info, getDataPartStorage(), getFileNameForColumn(columns.front()));
}
2022-09-05 05:26:58 +00:00
bool MergeTreeDataPartWide::isStoredOnRemoteDisk() const
{
return getDataPartStorage().isStoredOnRemoteDisk();
}
bool MergeTreeDataPartWide::isStoredOnRemoteDiskWithZeroCopySupport() const
{
return getDataPartStorage().supportZeroCopyReplication();
}
2019-11-18 12:22:27 +00:00
MergeTreeDataPartWide::~MergeTreeDataPartWide()
{
removeIfNeeded();
}
2019-11-18 15:18:50 +00:00
void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const
{
checkConsistencyBase();
2022-09-05 05:26:58 +00:00
std::string marks_file_extension = index_granularity_info.mark_type.getFileExtension();
2019-11-18 15:18:50 +00:00
if (!checksums.empty())
{
if (require_part_metadata)
{
2022-07-27 14:05:16 +00:00
for (const auto & name_type : columns)
2019-11-18 15:18:50 +00:00
{
2022-07-27 14:05:16 +00:00
getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
2019-11-18 15:18:50 +00:00
{
2021-03-09 14:46:52 +00:00
String file_name = ISerialization::getFileNameForStream(name_type, substream_path);
2022-09-05 05:26:58 +00:00
String mrk_file_name = file_name + marks_file_extension;
2021-03-09 14:46:52 +00:00
String bin_file_name = file_name + DATA_FILE_EXTENSION;
2022-04-07 17:44:49 +00:00
if (!checksums.files.contains(mrk_file_name))
2022-04-07 17:44:49 +00:00
throw Exception(
ErrorCodes::NO_FILE_IN_DATA_PART,
"No {} file checksum for column {} in part {} ",
mrk_file_name, name_type.name, getDataPartStorage().getFullPath());
2022-04-07 17:44:49 +00:00
if (!checksums.files.contains(bin_file_name))
2022-04-07 17:44:49 +00:00
throw Exception(
ErrorCodes::NO_FILE_IN_DATA_PART,
"No {} file checksum for column {} in part {}",
bin_file_name, name_type.name, getDataPartStorage().getFullPath());
2021-03-09 14:46:52 +00:00
});
2019-11-18 15:18:50 +00:00
}
}
}
else
{
/// Check that all marks are nonempty and have the same size.
std::optional<UInt64> marks_size;
2022-07-27 14:05:16 +00:00
for (const auto & name_type : columns)
2019-11-18 15:18:50 +00:00
{
2022-07-27 14:05:16 +00:00
getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
2019-11-18 15:18:50 +00:00
{
2022-09-05 05:26:58 +00:00
auto file_path = ISerialization::getFileNameForStream(name_type, substream_path) + marks_file_extension;
2019-11-18 15:18:50 +00:00
/// Missing file is Ok for case when new column was added.
if (getDataPartStorage().exists(file_path))
2019-11-18 15:18:50 +00:00
{
UInt64 file_size = getDataPartStorage().getFileSize(file_path);
2019-11-18 15:18:50 +00:00
if (!file_size)
2022-04-07 17:44:49 +00:00
throw Exception(
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART,
"Part {} is broken: {} is empty.",
getDataPartStorage().getFullPath(),
std::string(fs::path(getDataPartStorage().getFullPath()) / file_path));
2019-11-18 15:18:50 +00:00
if (!marks_size)
marks_size = file_size;
else if (file_size != *marks_size)
2022-04-07 17:44:49 +00:00
throw Exception(
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART,
"Part {} is broken: marks have different sizes.", getDataPartStorage().getFullPath());
2019-11-18 15:18:50 +00:00
}
});
}
}
}
bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const
2019-12-09 21:21:17 +00:00
{
2022-09-05 05:26:58 +00:00
std::string marks_file_extension = index_granularity_info.mark_type.getFileExtension();
auto check_stream_exists = [this, &marks_file_extension](const String & stream_name)
2019-12-09 21:21:17 +00:00
{
2021-03-09 14:46:52 +00:00
auto bin_checksum = checksums.files.find(stream_name + DATA_FILE_EXTENSION);
2022-09-05 05:26:58 +00:00
auto mrk_checksum = checksums.files.find(stream_name + marks_file_extension);
2021-03-09 14:46:52 +00:00
return bin_checksum != checksums.files.end() && mrk_checksum != checksums.files.end();
};
2021-03-09 14:46:52 +00:00
bool res = true;
2022-07-27 14:05:16 +00:00
getSerialization(column.name)->enumerateStreams([&](const auto & substream_path)
2021-03-09 14:46:52 +00:00
{
String file_name = ISerialization::getFileNameForStream(column, substream_path);
if (!check_stream_exists(file_name))
2019-12-09 21:21:17 +00:00
res = false;
2021-03-09 14:46:52 +00:00
});
2019-12-09 21:21:17 +00:00
return res;
}
2019-12-19 13:10:57 +00:00
String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const
{
String filename;
2022-07-27 14:05:16 +00:00
getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{
2019-12-19 13:10:57 +00:00
if (filename.empty())
2021-03-09 14:46:52 +00:00
filename = ISerialization::getFileNameForStream(column, substream_path);
2019-12-19 13:10:57 +00:00
});
return filename;
}
2020-06-29 20:36:18 +00:00
void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const
2020-03-23 12:19:43 +00:00
{
std::unordered_set<String> processed_substreams;
2022-07-27 14:05:16 +00:00
for (const auto & column : columns)
2020-03-23 12:19:43 +00:00
{
ColumnSize size = getColumnSizeImpl(column, &processed_substreams);
2020-03-23 12:19:43 +00:00
each_columns_size[column.name] = size;
total_size.add(size);
2020-07-25 14:42:20 +00:00
#ifndef NDEBUG
2020-07-27 09:42:37 +00:00
/// Most trivial types
if (rows_count != 0
&& column.type->isValueRepresentedByNumber()
&& !column.type->haveSubtypes()
2022-07-27 14:05:16 +00:00
&& getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT)
2020-07-25 14:42:20 +00:00
{
size_t rows_in_column = size.data_uncompressed / column.type->getSizeOfValueInMemory();
if (rows_in_column != rows_count)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
2020-07-25 14:45:46 +00:00
"Column {} has rows count {} according to size in memory "
"and size of single value, but data part {} has {} rows",
backQuote(column.name), rows_in_column, name, rows_count);
2020-07-25 14:42:20 +00:00
}
}
#endif
2020-07-25 20:54:33 +00:00
}
2020-03-23 12:19:43 +00:00
}
2016-02-14 04:58:47 +00:00
}