ClickHouse/dbms/src/Storages/MergeTree/IMergeTreeDataPartWriter.h

166 lines
5.6 KiB
C++
Raw Normal View History

2019-10-21 15:33:59 +00:00
#pragma once
2019-10-16 18:27:53 +00:00
#include <IO/WriteBufferFromFile.h>
2019-10-19 16:49:36 +00:00
#include <IO/WriteBufferFromFileBase.h>
2019-10-16 18:27:53 +00:00
#include <Compression/CompressedWriteBuffer.h>
#include <IO/HashingWriteBuffer.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <DataStreams/IBlockOutputStream.h>
2019-11-05 11:53:22 +00:00
#include <Storages/MergeTree/IMergeTreeDataPart.h>
2019-10-16 18:27:53 +00:00
namespace DB
{
2020-02-03 12:08:40 +00:00
/// Writes data part to disk in different formats.
/// Calculates and serializes primary and skip indices if needed.
2019-12-27 21:17:53 +00:00
class IMergeTreeDataPartWriter : private boost::noncopyable
2019-10-16 18:27:53 +00:00
{
public:
using WrittenOffsetColumns = std::set<std::string>;
2020-02-19 14:07:36 +00:00
/// Helper class, which holds chain of buffers to write data file with marks.
/// It is used to write: one column, skip index or all columns (in compact format).
struct Stream
2019-10-16 18:27:53 +00:00
{
2020-02-19 14:07:36 +00:00
Stream(
2019-10-16 18:27:53 +00:00
const String & escaped_column_name_,
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_,
size_t max_compress_block_size_,
size_t estimated_size_,
size_t aio_threshold_);
String escaped_column_name;
std::string data_file_extension;
std::string marks_file_extension;
/// compressed -> compressed_buf -> plain_hashing -> plain_file
std::unique_ptr<WriteBufferFromFileBase> plain_file;
2019-10-20 23:36:27 +00:00
HashingWriteBuffer plain_hashing;
2019-10-16 18:27:53 +00:00
CompressedWriteBuffer compressed_buf;
HashingWriteBuffer compressed;
/// marks -> marks_file
WriteBufferFromFile marks_file;
HashingWriteBuffer marks;
void finalize();
void sync();
2019-10-21 00:28:29 +00:00
void addToChecksums(IMergeTreeDataPart::Checksums & checksums);
2019-10-16 18:27:53 +00:00
};
2020-02-19 14:07:36 +00:00
using StreamPtr = std::unique_ptr<Stream>;
2019-10-19 16:49:36 +00:00
IMergeTreeDataPartWriter(
const String & part_path,
const MergeTreeData & storage,
const NamesAndTypesList & columns_list,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
2019-10-19 16:49:36 +00:00
const String & marks_file_extension,
const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings,
2019-11-26 09:48:22 +00:00
const MergeTreeIndexGranularity & index_granularity,
bool need_finish_last_granule);
2019-10-19 16:49:36 +00:00
virtual ~IMergeTreeDataPartWriter();
2019-11-07 11:11:38 +00:00
virtual void write(
2019-12-09 21:21:17 +00:00
const Block & block, const IColumn::Permutation * permutation = nullptr,
2019-10-16 18:27:53 +00:00
/* Blocks with already sorted index columns */
2019-11-07 11:11:38 +00:00
const Block & primary_key_block = {}, const Block & skip_indexes_block = {}) = 0;
2019-10-21 17:23:06 +00:00
void calculateAndSerializePrimaryIndex(const Block & primary_index_block, size_t rows);
void calculateAndSerializeSkipIndices(const Block & skip_indexes_block, size_t rows);
2019-10-21 15:33:59 +00:00
/// Shift mark and offset to prepare read next mark.
/// You must call it after calling write method and optionally
/// calling calculations of primary and skip indices.
void next();
2019-10-16 18:27:53 +00:00
2019-11-05 11:53:22 +00:00
/// Count index_granularity for block and store in `index_granularity`
void fillIndexGranularity(const Block & block);
2019-11-07 11:11:38 +00:00
const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; }
2019-11-18 15:18:50 +00:00
Columns releaseIndexColumns()
2019-11-07 11:11:38 +00:00
{
2019-11-18 15:18:50 +00:00
return Columns(std::make_move_iterator(index_columns.begin()), std::make_move_iterator(index_columns.end()));
2019-11-07 11:11:38 +00:00
}
void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_)
2019-12-09 21:21:17 +00:00
{
written_offset_columns = written_offset_columns_;
}
using SkipIndices = std::vector<MergeTreeIndexPtr>;
const SkipIndices & getSkipIndices() { return skip_indices; }
2019-11-05 11:53:22 +00:00
void initSkipIndices();
void initPrimaryIndex();
virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync = false) = 0;
2019-11-18 15:18:50 +00:00
void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums);
2019-11-05 11:53:22 +00:00
void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums);
2019-10-16 18:27:53 +00:00
protected:
2019-10-19 16:49:36 +00:00
using SerializationState = IDataType::SerializeBinaryBulkStatePtr;
2019-11-07 11:11:38 +00:00
using SerializationStates = std::unordered_map<String, SerializationState>;
2019-10-19 16:49:36 +00:00
String part_path;
2019-10-20 23:36:27 +00:00
const MergeTreeData & storage;
2019-10-19 16:49:36 +00:00
NamesAndTypesList columns_list;
const String marks_file_extension;
2019-10-16 18:27:53 +00:00
2019-11-05 11:53:22 +00:00
MergeTreeIndexGranularity index_granularity;
2019-10-19 16:49:36 +00:00
CompressionCodecPtr default_codec;
2019-11-07 11:11:38 +00:00
std::vector<MergeTreeIndexPtr> skip_indices;
MergeTreeWriterSettings settings;
2019-11-07 11:11:38 +00:00
2019-11-05 11:53:22 +00:00
bool compute_granularity;
bool with_final_mark;
2019-11-26 09:48:22 +00:00
bool need_finish_last_granule;
2019-11-05 11:53:22 +00:00
size_t current_mark = 0;
/// The offset to the first row of the block for which you want to write the index.
2019-11-05 11:53:22 +00:00
size_t index_offset = 0;
size_t next_mark = 0;
size_t next_index_offset = 0;
2020-02-19 14:07:36 +00:00
/// Number of marsk in data from which skip indices have to start
2019-11-05 11:53:22 +00:00
/// aggregation. I.e. it's data mark number, not skip indices mark.
size_t skip_index_data_mark = 0;
2020-02-19 14:07:36 +00:00
std::vector<StreamPtr> skip_indices_streams;
2019-11-05 11:53:22 +00:00
MergeTreeIndexAggregators skip_indices_aggregators;
std::vector<size_t> skip_index_filling;
std::unique_ptr<WriteBufferFromFile> index_file_stream;
std::unique_ptr<HashingWriteBuffer> index_stream;
MutableColumns index_columns;
DataTypes index_types;
/// Index columns values from the last row from the last block
/// It's written to index file in the `writeSuffixAndFinalizePart` method
Row last_index_row;
2019-11-07 11:11:38 +00:00
bool data_written = false;
2019-11-08 14:36:10 +00:00
bool primary_index_initialized = false;
bool skip_indices_initialized = false;
2019-12-09 21:21:17 +00:00
/// To correctly write Nested elements column-by-column.
WrittenOffsetColumns * written_offset_columns = nullptr;
2019-10-16 18:27:53 +00:00
};
2019-10-19 16:49:36 +00:00
}