ClickHouse/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp

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

426 lines
17 KiB
C++
Raw Normal View History

2020-04-14 01:26:34 +00:00
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <utility>
#include "IO/WriteBufferFromFileDecorator.h"
2019-10-19 16:49:36 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2019-10-21 15:33:59 +00:00
void MergeTreeDataPartWriterOnDisk::Stream::preFinalize()
2019-10-19 16:49:36 +00:00
{
2022-09-05 04:31:19 +00:00
compressed_hashing.next();
compressor.next();
plain_hashing.next();
2022-05-29 07:28:02 +00:00
2022-09-05 02:47:32 +00:00
if (compress_marks)
2022-09-05 04:31:19 +00:00
{
marks_compressed_hashing.next();
marks_compressor.next();
}
2022-05-29 07:28:02 +00:00
marks_hashing.next();
plain_file->preFinalize();
marks_file->preFinalize();
is_prefinalized = true;
}
void MergeTreeDataPartWriterOnDisk::Stream::finalize()
{
if (!is_prefinalized)
preFinalize();
plain_file->finalize();
marks_file->finalize();
2019-10-19 16:49:36 +00:00
}
void MergeTreeDataPartWriterOnDisk::Stream::sync() const
2019-10-19 16:49:36 +00:00
{
plain_file->sync();
marks_file->sync();
2019-10-19 16:49:36 +00:00
}
2020-04-14 01:26:34 +00:00
MergeTreeDataPartWriterOnDisk::Stream::Stream(
2019-10-19 16:49:36 +00:00
const String & escaped_column_name_,
2022-10-22 22:51:59 +00:00
const MutableDataPartStoragePtr & data_part_storage,
2019-10-19 16:49:36 +00:00
const String & data_path_,
const std::string & data_file_extension_,
const std::string & marks_path_,
const std::string & marks_file_extension_,
const CompressionCodecPtr & compression_codec_,
2022-03-21 13:56:38 +00:00
size_t max_compress_block_size_,
2022-05-29 07:28:02 +00:00
const CompressionCodecPtr & marks_compression_codec_,
size_t marks_compress_block_size_,
2022-03-21 13:56:38 +00:00
const WriteSettings & query_write_settings) :
2019-10-19 16:49:36 +00:00
escaped_column_name(escaped_column_name_),
data_file_extension{data_file_extension_},
marks_file_extension{marks_file_extension_},
2022-10-22 22:51:59 +00:00
plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)),
plain_hashing(*plain_file),
2022-09-05 04:31:19 +00:00
compressor(plain_hashing, compression_codec_, max_compress_block_size_),
compressed_hashing(compressor),
2022-10-22 22:51:59 +00:00
marks_file(data_part_storage->writeFile(marks_path_ + marks_file_extension, 4096, query_write_settings)),
2022-05-29 07:28:02 +00:00
marks_hashing(*marks_file),
2022-09-05 04:31:19 +00:00
marks_compressor(marks_hashing, marks_compression_codec_, marks_compress_block_size_),
marks_compressed_hashing(marks_compressor),
compress_marks(MarkType(marks_file_extension).compressed)
2019-10-19 16:49:36 +00:00
{
}
2020-04-14 01:26:34 +00:00
void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums)
2019-10-21 15:33:59 +00:00
{
String name = escaped_column_name;
2019-10-19 16:49:36 +00:00
2019-10-21 15:33:59 +00:00
checksums.files[name + data_file_extension].is_compressed = true;
2022-09-05 04:31:19 +00:00
checksums.files[name + data_file_extension].uncompressed_size = compressed_hashing.count();
checksums.files[name + data_file_extension].uncompressed_hash = compressed_hashing.getHash();
2019-10-21 15:33:59 +00:00
checksums.files[name + data_file_extension].file_size = plain_hashing.count();
checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash();
2019-10-19 16:49:36 +00:00
2022-09-05 02:47:32 +00:00
if (compress_marks)
2022-05-29 07:28:02 +00:00
{
checksums.files[name + marks_file_extension].is_compressed = true;
2022-09-05 04:31:19 +00:00
checksums.files[name + marks_file_extension].uncompressed_size = marks_compressed_hashing.count();
checksums.files[name + marks_file_extension].uncompressed_hash = marks_compressed_hashing.getHash();
2022-05-29 07:28:02 +00:00
}
2022-09-05 04:31:19 +00:00
2022-05-29 07:28:02 +00:00
checksums.files[name + marks_file_extension].file_size = marks_hashing.count();
checksums.files[name + marks_file_extension].file_hash = marks_hashing.getHash();
2019-10-21 15:33:59 +00:00
}
2019-10-19 16:49:36 +00:00
2019-11-05 11:53:22 +00:00
2020-04-14 01:26:34 +00:00
MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk(
2022-10-22 22:51:59 +00:00
const MergeTreeMutableDataPartPtr & data_part_,
2019-11-05 11:53:22 +00:00
const NamesAndTypesList & columns_list_,
2020-06-26 11:30:23 +00:00
const StorageMetadataPtr & metadata_snapshot_,
2020-12-10 09:22:43 +00:00
const MergeTreeIndices & indices_to_recalc_,
2019-11-05 11:53:22 +00:00
const String & marks_file_extension_,
const CompressionCodecPtr & default_codec_,
const MergeTreeWriterSettings & settings_,
const MergeTreeIndexGranularity & index_granularity_)
2022-10-22 22:51:59 +00:00
: IMergeTreeDataPartWriter(data_part_, columns_list_, metadata_snapshot_, settings_, index_granularity_)
2020-12-10 09:22:43 +00:00
, skip_indices(indices_to_recalc_)
2019-11-05 11:53:22 +00:00
, marks_file_extension(marks_file_extension_)
, default_codec(default_codec_)
2019-11-07 11:11:38 +00:00
, compute_granularity(index_granularity.empty())
, compress_primary_key(settings.compress_primary_key)
2019-11-05 11:53:22 +00:00
{
if (settings.blocks_are_granules_size && !index_granularity.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Can't take information about index granularity from blocks, when non empty index_granularity array specified");
if (!data_part->getDataPartStorage().exists())
data_part->getDataPartStorage().createDirectories();
2020-12-09 18:10:09 +00:00
2020-12-09 18:19:49 +00:00
if (settings.rewrite_primary_key)
initPrimaryIndex();
2020-12-09 18:10:09 +00:00
initSkipIndices();
2019-11-05 11:53:22 +00:00
}
2020-08-08 01:01:47 +00:00
// Implementation is split into static functions for ability
/// of making unit tests without creation instance of IMergeTreeDataPartWriter,
/// which requires a lot of dependencies and access to filesystem.
static size_t computeIndexGranularityImpl(
2019-11-05 11:53:22 +00:00
const Block & block,
size_t index_granularity_bytes,
size_t fixed_index_granularity_rows,
bool blocks_are_granules,
bool can_use_adaptive_index_granularity)
2019-11-05 11:53:22 +00:00
{
size_t rows_in_block = block.rows();
size_t index_granularity_for_block;
2019-11-05 11:53:22 +00:00
if (!can_use_adaptive_index_granularity)
2022-12-25 15:06:34 +00:00
{
2019-11-05 11:53:22 +00:00
index_granularity_for_block = fixed_index_granularity_rows;
2022-12-25 15:06:34 +00:00
}
2019-11-05 11:53:22 +00:00
else
{
size_t block_size_in_memory = block.bytes();
if (blocks_are_granules)
{
2019-11-05 11:53:22 +00:00
index_granularity_for_block = rows_in_block;
}
2019-11-05 11:53:22 +00:00
else if (block_size_in_memory >= index_granularity_bytes)
{
size_t granules_in_block = block_size_in_memory / index_granularity_bytes;
index_granularity_for_block = rows_in_block / granules_in_block;
}
else
{
2021-03-12 16:33:41 +00:00
size_t size_of_row_in_bytes = std::max(block_size_in_memory / rows_in_block, 1UL);
2019-11-05 11:53:22 +00:00
index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes;
}
}
2022-12-25 15:06:34 +00:00
/// We should be less or equal than fixed index granularity.
/// But if block size is a granule size then do not adjust it.
/// Granularity greater than fixed granularity might come from compact part.
if (!blocks_are_granules)
index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block);
/// Very rare case when index granularity bytes less than single row.
2022-12-25 15:06:34 +00:00
if (index_granularity_for_block == 0)
index_granularity_for_block = 1;
return index_granularity_for_block;
}
2020-12-09 18:10:09 +00:00
size_t MergeTreeDataPartWriterOnDisk::computeIndexGranularity(const Block & block) const
2019-11-05 11:53:22 +00:00
{
const auto storage_settings = storage.getSettings();
return computeIndexGranularityImpl(
block,
storage_settings->index_granularity_bytes,
storage_settings->index_granularity,
settings.blocks_are_granules_size,
settings.can_use_adaptive_granularity);
}
2020-04-14 01:26:34 +00:00
void MergeTreeDataPartWriterOnDisk::initPrimaryIndex()
2019-11-05 11:53:22 +00:00
{
2020-06-26 11:30:23 +00:00
if (metadata_snapshot->hasPrimaryKey())
2019-11-05 11:53:22 +00:00
{
String index_name = "primary" + getIndexExtension(compress_primary_key);
index_file_stream = data_part->getDataPartStorage().writeFile(index_name, DBMS_DEFAULT_BUFFER_SIZE, settings.query_write_settings);
2022-09-05 04:31:19 +00:00
index_file_hashing_stream = std::make_unique<HashingWriteBuffer>(*index_file_stream);
2022-05-29 07:28:02 +00:00
if (compress_primary_key)
2022-05-29 07:28:02 +00:00
{
ParserCodec codec_parser;
auto ast = parseQuery(codec_parser, "(" + Poco::toUpper(settings.primary_key_compression_codec) + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
CompressionCodecPtr primary_key_compression_codec = CompressionCodecFactory::instance().get(ast, nullptr);
2022-09-05 04:31:19 +00:00
index_compressor_stream = std::make_unique<CompressedWriteBuffer>(*index_file_hashing_stream, primary_key_compression_codec, settings.primary_key_compress_block_size);
index_source_hashing_stream = std::make_unique<HashingWriteBuffer>(*index_compressor_stream);
2022-05-29 07:28:02 +00:00
}
2019-11-05 11:53:22 +00:00
}
}
2020-04-14 01:26:34 +00:00
void MergeTreeDataPartWriterOnDisk::initSkipIndices()
2019-11-05 11:53:22 +00:00
{
2022-05-29 07:28:02 +00:00
ParserCodec codec_parser;
auto ast = parseQuery(codec_parser, "(" + Poco::toUpper(settings.marks_compression_codec) + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
CompressionCodecPtr marks_compression_codec = CompressionCodecFactory::instance().get(ast, nullptr);
for (const auto & index_helper : skip_indices)
2019-11-05 11:53:22 +00:00
{
String stream_name = index_helper->getFileName();
2019-11-05 11:53:22 +00:00
skip_indices_streams.emplace_back(
2020-04-14 01:26:34 +00:00
std::make_unique<MergeTreeDataPartWriterOnDisk::Stream>(
2019-11-05 11:53:22 +00:00
stream_name,
data_part->getDataPartStoragePtr(),
2022-04-12 18:59:49 +00:00
stream_name, index_helper->getSerializedFileExtension(),
stream_name, marks_file_extension,
2022-05-29 07:28:02 +00:00
default_codec, settings.max_compress_block_size,
marks_compression_codec, settings.marks_compress_block_size,
settings.query_write_settings));
2022-06-24 01:56:15 +00:00
GinIndexStorePtr store = nullptr;
2023-01-20 12:01:53 +00:00
if (dynamic_cast<const MergeTreeIndexInverted *>(&*index_helper) != nullptr)
2022-06-24 01:56:15 +00:00
{
store = std::make_shared<GinIndexStore>(stream_name, data_part->getDataPartStoragePtr(), data_part->getDataPartStoragePtr(), storage.getSettings()->max_digestion_size_per_segment);
2022-06-24 01:56:15 +00:00
gin_index_stores[stream_name] = store;
}
skip_indices_aggregators.push_back(index_helper->createIndexAggregatorForPart(store));
2020-12-15 10:34:28 +00:00
skip_index_accumulated_marks.push_back(0);
2019-11-05 11:53:22 +00:00
}
}
2020-12-11 13:20:19 +00:00
void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Block & primary_index_block, const Granules & granules_to_write)
2019-11-05 11:53:22 +00:00
{
2020-03-09 01:50:33 +00:00
size_t primary_columns_num = primary_index_block.columns();
2019-11-05 11:53:22 +00:00
if (index_columns.empty())
{
2020-03-09 01:50:33 +00:00
index_types = primary_index_block.getDataTypes();
2019-11-05 11:53:22 +00:00
index_columns.resize(primary_columns_num);
2020-11-02 12:19:37 +00:00
last_block_index_columns.resize(primary_columns_num);
2019-11-05 11:53:22 +00:00
for (size_t i = 0; i < primary_columns_num; ++i)
2020-03-09 01:50:33 +00:00
index_columns[i] = primary_index_block.getByPosition(i).column->cloneEmpty();
2019-11-05 11:53:22 +00:00
}
{
/** While filling index (index_columns), disable memory tracker.
* Because memory is allocated here (maybe in context of INSERT query),
* but then freed in completely different place (while merging parts), where query memory_tracker is not available.
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
MemoryTrackerBlockerInThread temporarily_disable_memory_tracker;
/// Write index. The index contains Primary Key value for each `index_granularity` row.
for (const auto & granule : granules_to_write)
2019-11-05 11:53:22 +00:00
{
if (metadata_snapshot->hasPrimaryKey() && granule.mark_on_start)
2019-11-05 11:53:22 +00:00
{
for (size_t j = 0; j < primary_columns_num; ++j)
{
const auto & primary_column = primary_index_block.getByPosition(j);
index_columns[j]->insertFrom(*primary_column.column, granule.start_row);
2022-05-29 07:28:02 +00:00
primary_column.type->getDefaultSerialization()->serializeBinary(
*primary_column.column, granule.start_row, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {});
}
2019-11-05 11:53:22 +00:00
}
}
}
2019-11-05 11:53:22 +00:00
/// store last index row to write final mark at the end of column
for (size_t j = 0; j < primary_columns_num; ++j)
2020-11-02 12:19:37 +00:00
last_block_index_columns[j] = primary_index_block.getByPosition(j).column;
2019-11-05 11:53:22 +00:00
}
2020-12-11 13:20:19 +00:00
void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block & skip_indexes_block, const Granules & granules_to_write)
2019-11-05 11:53:22 +00:00
{
2019-12-27 21:17:53 +00:00
/// Filling and writing skip indices like in MergeTreeDataPartWriterWide::writeColumn
2019-11-05 11:53:22 +00:00
for (size_t i = 0; i < skip_indices.size(); ++i)
{
const auto index_helper = skip_indices[i];
2019-11-05 11:53:22 +00:00
auto & stream = *skip_indices_streams[i];
2022-09-05 04:31:19 +00:00
WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing;
2023-01-20 11:27:49 +00:00
GinIndexStorePtr store;
2023-01-20 12:01:53 +00:00
if (dynamic_cast<const MergeTreeIndexInverted *>(&*index_helper) != nullptr)
2022-06-24 01:56:15 +00:00
{
String stream_name = index_helper->getFileName();
auto it = gin_index_stores.find(stream_name);
if (it == gin_index_stores.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Index '{}' does not exist", stream_name);
2022-06-24 01:56:15 +00:00
store = it->second;
}
2022-06-30 17:48:07 +00:00
2020-12-11 13:20:19 +00:00
for (const auto & granule : granules_to_write)
2019-11-05 11:53:22 +00:00
{
2020-12-18 13:49:45 +00:00
if (skip_index_accumulated_marks[i] == index_helper->index.granularity)
{
2022-09-05 04:31:19 +00:00
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing);
2020-12-18 13:49:45 +00:00
skip_index_accumulated_marks[i] = 0;
}
2020-12-11 13:20:19 +00:00
if (skip_indices_aggregators[i]->empty() && granule.mark_on_start)
2019-11-05 11:53:22 +00:00
{
2022-06-24 01:56:15 +00:00
skip_indices_aggregators[i] = index_helper->createIndexAggregatorForPart(store);
2020-12-11 13:20:19 +00:00
2022-09-05 04:31:19 +00:00
if (stream.compressed_hashing.offset() >= settings.min_compress_block_size)
stream.compressed_hashing.next();
writeIntBinary(stream.plain_hashing.count(), marks_out);
writeIntBinary(stream.compressed_hashing.offset(), marks_out);
2020-12-11 13:20:19 +00:00
/// Actually this numbers is redundant, but we have to store them
2022-09-05 04:31:19 +00:00
/// to be compatible with the normal .mrk2 file format
2020-12-11 13:20:19 +00:00
if (settings.can_use_adaptive_granularity)
2022-09-05 04:31:19 +00:00
writeIntBinary(1UL, marks_out);
2019-11-05 11:53:22 +00:00
}
2020-12-15 08:04:37 +00:00
size_t pos = granule.start_row;
2020-12-18 14:44:31 +00:00
skip_indices_aggregators[i]->update(skip_indexes_block, &pos, granule.rows_to_write);
if (granule.is_complete)
2020-12-15 10:34:28 +00:00
++skip_index_accumulated_marks[i];
2019-11-05 11:53:22 +00:00
}
}
}
void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::DataPart::Checksums & checksums)
2019-11-05 11:53:22 +00:00
{
2019-11-20 13:33:41 +00:00
bool write_final_mark = (with_final_mark && data_written);
if (write_final_mark && compute_granularity)
index_granularity.appendMark(0);
2022-09-05 04:31:19 +00:00
if (index_file_hashing_stream)
2019-11-05 11:53:22 +00:00
{
2019-11-20 13:33:41 +00:00
if (write_final_mark)
2019-11-05 11:53:22 +00:00
{
for (size_t j = 0; j < index_columns.size(); ++j)
{
2020-11-02 12:19:37 +00:00
const auto & column = *last_block_index_columns[j];
size_t last_row_number = column.size() - 1;
index_columns[j]->insertFrom(column, last_row_number);
2022-05-29 07:28:02 +00:00
index_types[j]->getDefaultSerialization()->serializeBinary(
column, last_row_number, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {});
2019-11-05 11:53:22 +00:00
}
2020-11-02 12:19:37 +00:00
last_block_index_columns.clear();
2019-11-05 11:53:22 +00:00
}
if (compress_primary_key)
2022-09-05 04:31:19 +00:00
index_source_hashing_stream->next();
2022-05-29 07:28:02 +00:00
2022-09-05 04:31:19 +00:00
index_file_hashing_stream->next();
2022-05-29 07:28:02 +00:00
String index_name = "primary" + getIndexExtension(compress_primary_key);
if (compress_primary_key)
2022-05-29 07:28:02 +00:00
{
checksums.files[index_name].is_compressed = true;
2022-09-05 04:31:19 +00:00
checksums.files[index_name].uncompressed_size = index_source_hashing_stream->count();
checksums.files[index_name].uncompressed_hash = index_source_hashing_stream->getHash();
2022-05-29 07:28:02 +00:00
}
2022-09-05 04:31:19 +00:00
checksums.files[index_name].file_size = index_file_hashing_stream->count();
checksums.files[index_name].file_hash = index_file_hashing_stream->getHash();
index_file_stream->preFinalize();
}
}
void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(bool sync)
{
2022-09-05 04:31:19 +00:00
if (index_file_hashing_stream)
{
index_file_stream->finalize();
if (sync)
2020-09-01 15:26:49 +00:00
index_file_stream->sync();
2022-05-29 07:28:02 +00:00
if (compress_primary_key)
2022-05-29 07:28:02 +00:00
{
2022-09-05 04:31:19 +00:00
index_source_hashing_stream = nullptr;
index_compressor_stream = nullptr;
2022-05-29 07:28:02 +00:00
}
2022-09-05 04:31:19 +00:00
index_file_hashing_stream = nullptr;
2019-11-05 11:53:22 +00:00
}
}
void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::DataPart::Checksums & checksums)
2019-11-05 11:53:22 +00:00
{
for (size_t i = 0; i < skip_indices.size(); ++i)
{
auto & stream = *skip_indices_streams[i];
if (!skip_indices_aggregators[i]->empty())
2022-09-05 04:31:19 +00:00
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing);
2019-11-05 11:53:22 +00:00
}
2022-01-11 19:02:48 +00:00
for (auto & stream : skip_indices_streams)
{
stream->preFinalize();
2022-02-01 02:49:40 +00:00
stream->addToChecksums(checksums);
}
}
void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync)
{
for (auto & stream : skip_indices_streams)
{
stream->finalize();
if (sync)
stream->sync();
2019-11-05 11:53:22 +00:00
}
2022-06-24 01:56:15 +00:00
for (auto & store: gin_index_stores)
store.second->finalize();
gin_index_stores.clear();
2019-11-07 11:11:38 +00:00
skip_indices_streams.clear();
2019-11-05 11:53:22 +00:00
skip_indices_aggregators.clear();
2020-12-15 10:34:28 +00:00
skip_index_accumulated_marks.clear();
2019-11-05 11:53:22 +00:00
}
2020-12-11 08:41:02 +00:00
Names MergeTreeDataPartWriterOnDisk::getSkipIndicesColumns() const
{
std::unordered_set<String> skip_indexes_column_names_set;
for (const auto & index : skip_indices)
std::copy(index->index.column_names.cbegin(), index->index.column_names.cend(),
std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end()));
return Names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end());
}
2019-10-21 15:33:59 +00:00
}