ClickHouse/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h

122 lines
4.3 KiB
C++
Raw Normal View History

2020-10-10 18:37:02 +00:00
#pragma once
2020-04-14 01:26:34 +00:00
#include <Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h>
2019-10-21 00:28:29 +00:00
namespace DB
{
2020-12-14 12:03:49 +00:00
struct StreamNameAndMark
{
String stream_name;
MarkInCompressedFile mark;
};
using StreamsWithMarks = std::vector<StreamNameAndMark>;
using ColumnNameToMark = std::unordered_map<String, StreamsWithMarks>;
2020-02-03 12:08:40 +00:00
/// Writes data part in wide format.
2020-04-14 01:26:34 +00:00
class MergeTreeDataPartWriterWide : public MergeTreeDataPartWriterOnDisk
2019-10-21 00:28:29 +00:00
{
public:
2019-10-21 15:33:59 +00:00
MergeTreeDataPartWriterWide(
const MergeTreeData::DataPartPtr & data_part,
2019-10-21 15:33:59 +00:00
const NamesAndTypesList & columns_list,
2020-06-17 12:39:20 +00:00
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
2019-10-21 15:33:59 +00:00
const String & marks_file_extension,
const CompressionCodecPtr & default_codec,
const MergeTreeWriterSettings & settings,
2019-11-07 11:11:38 +00:00
const MergeTreeIndexGranularity & index_granularity);
2019-10-21 15:33:59 +00:00
2020-12-10 08:57:52 +00:00
void write(const Block & block, const IColumn::Permutation * permutation) override;
2019-10-21 00:28:29 +00:00
2020-12-10 08:57:52 +00:00
void finish(IMergeTreeDataPart::Checksums & checksums, bool sync) final;
2019-10-21 17:23:06 +00:00
2020-01-16 16:15:01 +00:00
private:
2020-12-15 10:34:28 +00:00
/// Finish serialization of data: write final mark if required and compute checksums
/// Also validate written data in debug mode
2020-12-10 08:57:52 +00:00
void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync);
2019-10-21 00:28:29 +00:00
/// Write data of one column.
/// Return how many marks were written and
/// how many rows were written for last mark
void writeColumn(
2019-10-21 00:28:29 +00:00
const String & name,
const IDataType & type,
const IColumn & column,
2020-12-11 13:20:19 +00:00
WrittenOffsetColumns & offset_columns,
const Granules & granules);
2019-10-21 00:28:29 +00:00
2020-12-15 10:34:28 +00:00
/// Write single granule of one column.
2020-12-11 13:20:19 +00:00
void writeSingleGranule(
2019-10-21 00:28:29 +00:00
const String & name,
const IDataType & type,
const IColumn & column,
WrittenOffsetColumns & offset_columns,
IDataType::SerializeBinaryBulkStatePtr & serialization_state,
IDataType::SerializeBinaryBulkSettings & serialize_settings,
2020-12-15 10:34:28 +00:00
const Granule & granule);
2020-12-10 16:29:10 +00:00
2020-12-15 10:34:28 +00:00
/// Take offsets from column and return as MarkInCompressed file with stream name
2020-12-10 16:29:10 +00:00
StreamsWithMarks getCurrentMarksForColumn(
const String & name,
const IDataType & type,
WrittenOffsetColumns & offset_columns,
DB::IDataType::SubstreamPath & path);
2020-12-15 10:34:28 +00:00
/// Write mark to disk using stream and rows count
void flushMarkToFile(
const StreamNameAndMark & stream_with_mark,
size_t rows_in_mark);
/// Write mark for column taking offsets from column stream
2019-10-21 00:28:29 +00:00
void writeSingleMark(
const String & name,
const IDataType & type,
WrittenOffsetColumns & offset_columns,
size_t number_of_rows,
DB::IDataType::SubstreamPath & path);
2019-10-21 17:23:06 +00:00
void writeFinalMark(
const std::string & column_name,
const DataTypePtr column_type,
WrittenOffsetColumns & offset_columns,
DB::IDataType::SubstreamPath & path);
2019-10-21 00:28:29 +00:00
2019-10-21 15:33:59 +00:00
void addStreams(
const String & name,
const IDataType & type,
2020-09-21 07:18:23 +00:00
const ASTPtr & effective_codec_desc,
2019-12-09 21:21:17 +00:00
size_t estimated_size);
2019-10-21 15:33:59 +00:00
2020-12-15 10:34:28 +00:00
/// Method for self check (used in debug-build only). Checks that written
/// data and corresponding marks are consistent. Otherwise throws logical
/// errors.
2020-12-14 07:28:42 +00:00
void validateColumnOfFixedSize(const String & name, const IDataType & type);
2020-12-09 18:10:09 +00:00
void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override;
2020-12-15 10:34:28 +00:00
/// Use information from just written granules to shift current mark
/// in our index_granularity array.
2020-12-15 09:54:48 +00:00
void shiftCurrentMark(const Granules & granules_written);
2020-12-18 13:49:45 +00:00
void adjustLastMarkAndFlushToDisk();
2020-12-09 18:10:09 +00:00
IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns) const;
using SerializationState = IDataType::SerializeBinaryBulkStatePtr;
using SerializationStates = std::unordered_map<String, SerializationState>;
2019-10-21 00:28:29 +00:00
SerializationStates serialization_states;
2020-02-19 14:07:36 +00:00
using ColumnStreams = std::map<String, StreamPtr>;
2019-10-21 00:28:29 +00:00
ColumnStreams column_streams;
2020-12-18 13:49:45 +00:00
using MarksForColumns = std::unordered_map<String, StreamsWithMarks>;
MarksForColumns last_non_written_marks;
2020-12-14 12:03:49 +00:00
2020-12-15 10:34:28 +00:00
/// How many rows we have already written in the current mark.
/// More than zero when incoming blocks are smaller then their granularity.
2020-12-14 12:03:49 +00:00
size_t rows_written_in_last_mark = 0;
2019-10-21 00:28:29 +00:00
};
2019-10-21 15:33:59 +00:00
}