ClickHouse/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h

167 lines
6.2 KiB
C++
Raw Normal View History

2019-10-21 15:33:59 +00:00
#pragma once
2020-04-14 01:26:34 +00:00
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
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>
#include <Disks/IDisk.h>
2019-10-16 18:27:53 +00:00
namespace DB
{
2020-12-15 10:34:28 +00:00
/// Single unit for writing data to disk. Contains information about
/// amount of rows to write and marks.
2020-12-14 12:03:49 +00:00
struct Granule
{
2020-12-15 10:34:28 +00:00
/// Start row in block for granule
2020-12-15 08:04:37 +00:00
size_t start_row;
2020-12-15 10:34:28 +00:00
/// Amount of rows which granule have to contain according to index
/// granularity.
/// NOTE: Sometimes it's not equal to actually written rows, for example
/// for the last granule if it's smaller than computed granularity.
2020-12-14 12:51:14 +00:00
size_t granularity_rows;
2020-12-15 10:34:28 +00:00
/// Amount of rows from block which have to be written to disk from start_row
2020-12-15 08:04:37 +00:00
size_t block_rows;
2020-12-15 10:34:28 +00:00
/// Global mark number in the list of all marks (index_granularity) for this part
2020-12-14 12:03:49 +00:00
size_t mark_number;
2020-12-15 10:34:28 +00:00
/// Should writer write mark for the first of this granule to disk.
/// NOTE: Sometimes we don't write mark for the start row, because
/// this granule can be continuation of the previous one.
2020-12-14 12:03:49 +00:00
bool mark_on_start;
2020-12-15 08:04:37 +00:00
2020-12-15 10:34:28 +00:00
/// Is this granule contain amout of rows equal to the value in index granularity
2020-12-15 08:04:37 +00:00
bool isCompleted() const
{
return granularity_rows == block_rows;
}
2020-12-14 12:03:49 +00:00
};
2020-12-15 10:34:28 +00:00
/// Multiple granules to write for concrete block.
2020-12-14 12:03:49 +00:00
using Granules = std::vector<Granule>;
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.
2020-04-14 01:26:34 +00:00
class MergeTreeDataPartWriterOnDisk : public IMergeTreeDataPartWriter
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_,
DiskPtr disk_,
2019-10-16 18:27:53 +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_,
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
std::unique_ptr<WriteBufferFromFileBase> marks_file;
2019-10-16 18:27:53 +00:00
HashingWriteBuffer marks;
void finalize();
void sync() const;
2019-10-16 18:27:53 +00:00
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
2020-04-14 01:26:34 +00:00
MergeTreeDataPartWriterOnDisk(
const MergeTreeData::DataPartPtr & data_part_,
2019-10-19 16:49:36 +00:00
const NamesAndTypesList & columns_list,
2020-06-26 11:30:23 +00:00
const StorageMetadataPtr & metadata_snapshot_,
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,
const MergeTreeIndexGranularity & index_granularity);
2019-10-19 16:49:36 +00:00
void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_)
2019-12-09 21:21:17 +00:00
{
written_offset_columns = written_offset_columns_;
}
2020-12-15 10:34:28 +00:00
2019-10-16 18:27:53 +00:00
protected:
2020-12-09 18:10:09 +00:00
/// Count index_granularity for block and store in `index_granularity`
size_t computeIndexGranularity(const Block & block) const;
2020-12-14 12:51:14 +00:00
2020-12-15 10:34:28 +00:00
/// Write primary index according to granules_to_write
2020-12-11 13:20:19 +00:00
void calculateAndSerializePrimaryIndex(const Block & primary_index_block, const Granules & granules_to_write);
2020-12-15 10:34:28 +00:00
/// Write skip indices according to granules_to_write. Skip indices also have their own marks
/// and one skip index granule can contain multiple "normal" marks. So skip indices serialization
/// require additional state: skip_indices_aggregators and skip_index_accumulated_marks
2020-12-11 13:20:19 +00:00
void calculateAndSerializeSkipIndices(const Block & skip_indexes_block, const Granules & granules_to_write);
2020-12-10 08:57:52 +00:00
2020-12-15 10:34:28 +00:00
/// Finishes primary index serialization: write final primary index row (if required) and compute checksums
2020-12-10 08:57:52 +00:00
void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync);
2020-12-15 10:34:28 +00:00
/// Finishes skip indices serialization: write all accumulated data to disk and compute checksums
2020-12-10 08:57:52 +00:00
void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync);
2020-12-14 12:51:14 +00:00
2020-12-15 10:34:28 +00:00
/// Get global number of the current which we are writing (or going to start to write)
2020-12-14 12:03:49 +00:00
size_t getCurrentMark() const { return current_mark; }
2020-12-15 10:34:28 +00:00
2020-12-14 12:03:49 +00:00
void setCurrentMark(size_t mark) { current_mark = mark; }
2020-12-10 08:57:52 +00:00
2020-12-15 10:34:28 +00:00
/// Get unique non ordered skip indices column.
2020-12-11 08:41:02 +00:00
Names getSkipIndicesColumns() const;
2020-12-10 09:22:43 +00:00
const MergeTreeIndices skip_indices;
2019-10-19 16:49:36 +00:00
2020-12-09 18:10:09 +00:00
const String part_path;
2019-10-19 16:49:36 +00:00
const String marks_file_extension;
2020-12-09 18:10:09 +00:00
const CompressionCodecPtr default_codec;
2019-10-19 16:49:36 +00:00
2020-12-09 18:10:09 +00:00
const bool compute_granularity;
2019-11-05 11:53:22 +00:00
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;
2020-12-15 10:34:28 +00:00
std::vector<size_t> skip_index_accumulated_marks;
2019-11-05 11:53:22 +00:00
std::unique_ptr<WriteBufferFromFileBase> index_file_stream;
2019-11-05 11:53:22 +00:00
std::unique_ptr<HashingWriteBuffer> index_stream;
DataTypes index_types;
2020-11-02 12:19:37 +00:00
/// Index columns from the last block
2019-11-05 11:53:22 +00:00
/// It's written to index file in the `writeSuffixAndFinalizePart` method
2020-11-02 12:19:37 +00:00
Columns last_block_index_columns;
2019-11-05 11:53:22 +00:00
2019-11-07 11:11:38 +00:00
bool data_written = false;
2019-12-09 21:21:17 +00:00
/// To correctly write Nested elements column-by-column.
WrittenOffsetColumns * written_offset_columns = nullptr;
2020-12-14 12:03:49 +00:00
/// Data is already written up to this mark.
size_t current_mark = 0;
private:
2020-12-09 18:10:09 +00:00
void initSkipIndices();
void initPrimaryIndex();
virtual void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) = 0;
2019-10-16 18:27:53 +00:00
};
2019-10-19 16:49:36 +00:00
}