2020-04-14 01:26:34 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
|
2023-01-20 11:40:19 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIndexInverted.h>
|
2022-01-10 19:39:10 +00:00
|
|
|
#include <Common/MemoryTrackerBlockerInThread.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
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
template<bool only_plain_file>
|
|
|
|
void MergeTreeDataPartWriterOnDisk::Stream<only_plain_file>::preFinalize()
|
2019-10-19 16:49:36 +00:00
|
|
|
{
|
2023-06-19 21:33:37 +00:00
|
|
|
/// Here the main goal is to do preFinalize calls for plain_file and marks_file
|
|
|
|
/// Before that all hashing and compression buffers have to be finalized
|
|
|
|
/// Otherwise some data might stuck in the buffers above plain_file and marks_file
|
|
|
|
/// Also the order is important
|
|
|
|
|
2023-06-01 17:00:47 +00:00
|
|
|
compressed_hashing.finalize();
|
|
|
|
compressor.finalize();
|
|
|
|
plain_hashing.finalize();
|
2022-05-29 07:28:02 +00:00
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
if constexpr (!only_plain_file)
|
2022-09-05 04:31:19 +00:00
|
|
|
{
|
2023-08-09 20:57:49 +00:00
|
|
|
if (compress_marks)
|
|
|
|
{
|
|
|
|
marks_compressed_hashing.finalize();
|
|
|
|
marks_compressor.finalize();
|
|
|
|
}
|
2022-05-29 07:28:02 +00:00
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
marks_hashing.finalize();
|
|
|
|
}
|
2020-10-06 09:38:00 +00:00
|
|
|
|
2022-02-01 10:36:51 +00:00
|
|
|
plain_file->preFinalize();
|
2023-08-09 20:57:49 +00:00
|
|
|
if constexpr (!only_plain_file)
|
|
|
|
marks_file->preFinalize();
|
2022-02-01 10:36:51 +00:00
|
|
|
|
|
|
|
is_prefinalized = true;
|
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
template<bool only_plain_file>
|
|
|
|
void MergeTreeDataPartWriterOnDisk::Stream<only_plain_file>::finalize()
|
2022-02-01 10:36:51 +00:00
|
|
|
{
|
|
|
|
if (!is_prefinalized)
|
|
|
|
preFinalize();
|
|
|
|
|
2020-10-06 09:38:00 +00:00
|
|
|
plain_file->finalize();
|
2023-08-09 20:57:49 +00:00
|
|
|
if constexpr (!only_plain_file)
|
|
|
|
marks_file->finalize();
|
2019-10-19 16:49:36 +00:00
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
template<bool only_plain_file>
|
|
|
|
void MergeTreeDataPartWriterOnDisk::Stream<only_plain_file>::sync() const
|
2019-10-19 16:49:36 +00:00
|
|
|
{
|
|
|
|
plain_file->sync();
|
2023-08-09 20:57:49 +00:00
|
|
|
if constexpr (!only_plain_file)
|
|
|
|
marks_file->sync();
|
2019-10-19 16:49:36 +00:00
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
template<>
|
|
|
|
MergeTreeDataPartWriterOnDisk::Stream<false>::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)),
|
2021-01-13 13:22:59 +00:00
|
|
|
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
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
template<>
|
|
|
|
MergeTreeDataPartWriterOnDisk::Stream<true>::Stream(
|
|
|
|
const String & escaped_column_name_,
|
|
|
|
const MutableDataPartStoragePtr & data_part_storage,
|
|
|
|
const String & data_path_,
|
|
|
|
const std::string & data_file_extension_,
|
|
|
|
const CompressionCodecPtr & compression_codec_,
|
|
|
|
size_t max_compress_block_size_,
|
|
|
|
const WriteSettings & query_write_settings) :
|
|
|
|
escaped_column_name(escaped_column_name_),
|
|
|
|
data_file_extension{data_file_extension_},
|
|
|
|
plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)),
|
|
|
|
plain_hashing(*plain_file),
|
|
|
|
compressor(plain_hashing, compression_codec_, max_compress_block_size_),
|
|
|
|
compressed_hashing(compressor),
|
|
|
|
compress_marks(false)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
template<bool only_plain_file>
|
|
|
|
void MergeTreeDataPartWriterOnDisk::Stream<only_plain_file>::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
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
if constexpr (!only_plain_file)
|
2022-05-29 07:28:02 +00:00
|
|
|
{
|
2023-08-09 20:57:49 +00:00
|
|
|
if (compress_marks)
|
|
|
|
{
|
|
|
|
checksums.files[name + marks_file_extension].is_compressed = true;
|
|
|
|
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-09-05 04:31:19 +00:00
|
|
|
|
2023-08-09 20:57:49 +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_,
|
2023-08-09 20:57:49 +00:00
|
|
|
const Statistics & stats_to_recalc_,
|
2019-11-05 11:53:22 +00:00
|
|
|
const String & marks_file_extension_,
|
|
|
|
const CompressionCodecPtr & default_codec_,
|
2019-12-18 15:54:45 +00:00
|
|
|
const MergeTreeWriterSettings & settings_,
|
2020-04-30 15:27:39 +00:00
|
|
|
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_)
|
2023-08-09 20:57:49 +00:00
|
|
|
, stats(stats_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())
|
2022-08-28 02:19:14 +00:00
|
|
|
, compress_primary_key(settings.compress_primary_key)
|
2019-11-05 11:53:22 +00:00
|
|
|
{
|
|
|
|
if (settings.blocks_are_granules_size && !index_granularity.empty())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
|
|
|
"Can't take information about index granularity from blocks, when non empty index_granularity array specified");
|
2019-12-18 16:41:11 +00:00
|
|
|
|
2022-10-23 03:29:26 +00:00
|
|
|
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();
|
2023-08-09 20:57:49 +00:00
|
|
|
initStatistics();
|
2019-11-05 11:53:22 +00:00
|
|
|
}
|
|
|
|
|
2020-08-08 01:01:47 +00:00
|
|
|
// Implementation is split into static functions for ability
|
2020-04-30 15:27:39 +00:00
|
|
|
/// 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,
|
2020-04-30 15:27:39 +00:00
|
|
|
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;
|
2023-02-10 14:13:29 +00:00
|
|
|
|
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)
|
2023-02-10 14:13:29 +00:00
|
|
|
{
|
2019-11-05 11:53:22 +00:00
|
|
|
index_granularity_for_block = rows_in_block;
|
2023-02-10 14:13:29 +00:00
|
|
|
}
|
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
|
|
|
|
2023-02-10 14:13:29 +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;
|
|
|
|
|
2020-04-30 15:27:39 +00:00
|
|
|
return index_granularity_for_block;
|
|
|
|
}
|
2019-12-18 16:41:11 +00:00
|
|
|
|
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();
|
2020-04-30 15:27:39 +00:00
|
|
|
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
|
|
|
{
|
2022-08-28 02:19:14 +00:00
|
|
|
String index_name = "primary" + getIndexExtension(compress_primary_key);
|
2022-10-23 03:29:26 +00:00
|
|
|
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
|
|
|
|
2022-08-28 02:19:14 +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
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
void MergeTreeDataPartWriterOnDisk::initStatistics()
|
|
|
|
{
|
|
|
|
for (const auto & stat_ptr : stats)
|
|
|
|
{
|
|
|
|
String stats_name = stat_ptr->getFileName();
|
|
|
|
stats_streams.emplace_back(std::make_unique<MergeTreeDataPartWriterOnDisk::Stream<true>>(
|
|
|
|
stats_name,
|
|
|
|
data_part->getDataPartStoragePtr(),
|
|
|
|
stats_name, STAT_FILE_SUFFIX,
|
|
|
|
default_codec, settings.max_compress_block_size,
|
|
|
|
settings.query_write_settings));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
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);
|
|
|
|
|
2023-02-26 20:01:35 +00:00
|
|
|
for (const auto & skip_index : skip_indices)
|
2019-11-05 11:53:22 +00:00
|
|
|
{
|
2023-02-26 20:01:35 +00:00
|
|
|
String stream_name = skip_index->getFileName();
|
2019-11-05 11:53:22 +00:00
|
|
|
skip_indices_streams.emplace_back(
|
2023-08-09 20:57:49 +00:00
|
|
|
std::make_unique<MergeTreeDataPartWriterOnDisk::Stream<false>>(
|
2019-11-05 11:53:22 +00:00
|
|
|
stream_name,
|
2022-10-23 03:29:26 +00:00
|
|
|
data_part->getDataPartStoragePtr(),
|
2023-02-26 20:01:35 +00:00
|
|
|
stream_name, skip_index->getSerializedFileExtension(),
|
2022-04-12 18:59:49 +00:00
|
|
|
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-02-27 09:30:50 +00:00
|
|
|
if (typeid_cast<const MergeTreeIndexInverted *>(&*skip_index) != nullptr)
|
2022-06-24 01:56:15 +00:00
|
|
|
{
|
2022-12-29 16:00:17 +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;
|
|
|
|
}
|
2023-08-29 19:53:40 +00:00
|
|
|
skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store, settings));
|
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
|
|
|
}
|
|
|
|
|
|
|
|
{
|
2021-01-11 21:50:37 +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)
|
|
|
|
*/
|
2022-01-10 19:39:10 +00:00
|
|
|
MemoryTrackerBlockerInThread temporarily_disable_memory_tracker;
|
2021-01-11 21:50:37 +00:00
|
|
|
|
|
|
|
/// 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
|
|
|
{
|
2021-01-11 21:50:37 +00:00
|
|
|
if (metadata_snapshot->hasPrimaryKey() && granule.mark_on_start)
|
2019-11-05 11:53:22 +00:00
|
|
|
{
|
2021-01-11 21:50:37 +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(
|
2022-12-02 12:57:11 +00:00
|
|
|
*primary_column.column, granule.start_row, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {});
|
2021-01-11 21:50:37 +00:00
|
|
|
}
|
2019-11-05 11:53:22 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-01-11 21:50:37 +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
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
void MergeTreeDataPartWriterOnDisk::calculateAndSerializeStatistics(const Block & block)
|
|
|
|
{
|
|
|
|
for (const auto & stat_ptr : stats)
|
|
|
|
{
|
2023-10-03 22:58:26 +00:00
|
|
|
stat_ptr->update(block.getByName(stat_ptr->columnName()).column);
|
2023-08-09 20:57:49 +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)
|
|
|
|
{
|
2020-06-03 13:27:54 +00:00
|
|
|
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-02-27 09:30:50 +00:00
|
|
|
if (typeid_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);
|
2023-01-24 10:09:42 +00:00
|
|
|
if (it == gin_index_stores.end())
|
2023-01-23 21:13:58 +00:00
|
|
|
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
|
|
|
{
|
2023-08-29 19:53:40 +00:00
|
|
|
skip_indices_aggregators[i] = index_helper->createIndexAggregatorForPart(store, settings);
|
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();
|
|
|
|
|
2023-06-22 03:15:12 +00:00
|
|
|
writeBinaryLittleEndian(stream.plain_hashing.count(), marks_out);
|
|
|
|
writeBinaryLittleEndian(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)
|
2023-06-22 03:15:12 +00:00
|
|
|
writeBinaryLittleEndian(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
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-02-01 10:36:51 +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(
|
2022-12-02 12:57:11 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
2022-08-28 02:19:14 +00:00
|
|
|
if (compress_primary_key)
|
2023-06-16 14:34:50 +00:00
|
|
|
{
|
2023-06-01 17:00:47 +00:00
|
|
|
index_source_hashing_stream->finalize();
|
2023-06-16 14:34:50 +00:00
|
|
|
index_compressor_stream->finalize();
|
|
|
|
}
|
2022-05-29 07:28:02 +00:00
|
|
|
|
2023-06-01 17:00:47 +00:00
|
|
|
index_file_hashing_stream->finalize();
|
2022-05-29 07:28:02 +00:00
|
|
|
|
2022-08-28 02:19:14 +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();
|
2022-02-01 10:36:51 +00:00
|
|
|
index_file_stream->preFinalize();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(bool sync)
|
|
|
|
{
|
2022-09-05 04:31:19 +00:00
|
|
|
if (index_file_hashing_stream)
|
2022-02-01 10:36:51 +00:00
|
|
|
{
|
2020-10-06 09:38:00 +00:00
|
|
|
index_file_stream->finalize();
|
2020-08-24 09:57:27 +00:00
|
|
|
if (sync)
|
2020-09-01 15:26:49 +00:00
|
|
|
index_file_stream->sync();
|
2022-05-29 07:28:02 +00:00
|
|
|
|
2022-08-28 02:19:14 +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
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-02-01 10:36:51 +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);
|
2023-02-26 20:01:35 +00:00
|
|
|
|
|
|
|
/// Register additional files written only by the inverted index. Required because otherwise DROP TABLE complains about unknown
|
|
|
|
/// files. Note that the provided actual checksums are bogus. The problem is that at this point the file writes happened already and
|
|
|
|
/// we'd need to re-open + hash the files (fixing this is TODO). For now, CHECK TABLE skips these four files.
|
2023-02-27 09:30:50 +00:00
|
|
|
if (typeid_cast<const MergeTreeIndexInverted *>(&*skip_indices[i]) != nullptr)
|
2023-02-26 20:01:35 +00:00
|
|
|
{
|
|
|
|
String filename_without_extension = skip_indices[i]->getFileName();
|
|
|
|
checksums.files[filename_without_extension + ".gin_dict"] = MergeTreeDataPartChecksums::Checksum();
|
|
|
|
checksums.files[filename_without_extension + ".gin_post"] = MergeTreeDataPartChecksums::Checksum();
|
|
|
|
checksums.files[filename_without_extension + ".gin_seg"] = MergeTreeDataPartChecksums::Checksum();
|
|
|
|
checksums.files[filename_without_extension + ".gin_sid"] = MergeTreeDataPartChecksums::Checksum();
|
|
|
|
}
|
2019-11-05 11:53:22 +00:00
|
|
|
}
|
|
|
|
|
2022-01-11 19:02:48 +00:00
|
|
|
for (auto & stream : skip_indices_streams)
|
|
|
|
{
|
2022-02-01 10:36:51 +00:00
|
|
|
stream->preFinalize();
|
2022-02-01 02:49:40 +00:00
|
|
|
stream->addToChecksums(checksums);
|
2022-02-01 10:36:51 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
void MergeTreeDataPartWriterOnDisk::finishStatisticsSerialization(bool sync)
|
|
|
|
{
|
|
|
|
for (auto & stream : stats_streams)
|
|
|
|
{
|
|
|
|
stream->finalize();
|
|
|
|
if (sync)
|
|
|
|
stream->sync();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void MergeTreeDataPartWriterOnDisk::fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums)
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < stats.size(); i++)
|
|
|
|
{
|
|
|
|
auto & stream = *stats_streams[i];
|
|
|
|
stats[i]->serialize(stream.compressed_hashing);
|
|
|
|
stream.preFinalize();
|
|
|
|
stream.addToChecksums(checksums);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-02-01 10:36:51 +00:00
|
|
|
void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync)
|
|
|
|
{
|
|
|
|
for (auto & stream : skip_indices_streams)
|
|
|
|
{
|
|
|
|
stream->finalize();
|
2020-08-24 09:57:27 +00:00
|
|
|
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());
|
|
|
|
}
|
|
|
|
|
2023-08-09 20:57:49 +00:00
|
|
|
template struct MergeTreeDataPartWriterOnDisk::Stream<false>;
|
|
|
|
template struct MergeTreeDataPartWriterOnDisk::Stream<true>;
|
|
|
|
|
2019-10-21 15:33:59 +00:00
|
|
|
}
|