2019-06-18 12:54:27 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
|
2019-08-19 10:37:04 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIndexGranularityInfo.h>
|
2019-06-18 12:54:27 +00:00
|
|
|
#include <IO/WriteBufferFromFile.h>
|
|
|
|
#include <Compression/CompressedWriteBuffer.h>
|
|
|
|
#include <IO/HashingWriteBuffer.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
2019-10-19 16:49:36 +00:00
|
|
|
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
|
2019-06-18 12:54:27 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class IMergedBlockOutputStream : public IBlockOutputStream
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
IMergedBlockOutputStream(
|
2019-10-19 16:49:36 +00:00
|
|
|
const MergeTreeDataPartPtr & data_part,
|
|
|
|
CompressionCodecPtr codec_,
|
|
|
|
const WriterSettings & writer_settings_,
|
2019-06-18 12:54:27 +00:00
|
|
|
bool blocks_are_granules_size_,
|
2019-07-28 11:10:35 +00:00
|
|
|
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
2019-08-19 10:37:04 +00:00
|
|
|
const MergeTreeIndexGranularity & index_granularity_,
|
2019-10-19 16:49:36 +00:00
|
|
|
bool can_use_adaptive_granularity_);
|
2019-06-18 12:54:27 +00:00
|
|
|
|
|
|
|
using WrittenOffsetColumns = std::set<std::string>;
|
|
|
|
|
|
|
|
protected:
|
|
|
|
using SerializationState = IDataType::SerializeBinaryBulkStatePtr;
|
|
|
|
using SerializationStates = std::vector<SerializationState>;
|
|
|
|
|
|
|
|
IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns, bool skip_offsets);
|
|
|
|
|
|
|
|
/// Count index_granularity for block and store in `index_granularity`
|
|
|
|
void fillIndexGranularity(const Block & block);
|
|
|
|
|
|
|
|
/// Write final mark to the end of column
|
|
|
|
void writeFinalMark(
|
|
|
|
const std::string & column_name,
|
|
|
|
const DataTypePtr column_type,
|
|
|
|
WrittenOffsetColumns & offset_columns,
|
|
|
|
bool skip_offsets,
|
|
|
|
DB::IDataType::SubstreamPath & path);
|
|
|
|
|
2019-07-28 11:10:35 +00:00
|
|
|
void initSkipIndices();
|
|
|
|
void calculateAndSerializeSkipIndices(const ColumnsWithTypeAndName & skip_indexes_columns, size_t rows);
|
|
|
|
void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums);
|
2019-06-18 12:54:27 +00:00
|
|
|
protected:
|
|
|
|
MergeTreeData & storage;
|
|
|
|
|
2019-07-28 11:10:35 +00:00
|
|
|
SerializationStates serialization_states;
|
|
|
|
String part_path;
|
|
|
|
|
2019-06-18 12:54:27 +00:00
|
|
|
/// The offset to the first row of the block for which you want to write the index.
|
|
|
|
size_t index_offset = 0;
|
|
|
|
|
2019-10-19 16:49:36 +00:00
|
|
|
WriterSettings writer_settings;
|
2019-06-18 12:54:27 +00:00
|
|
|
|
|
|
|
size_t current_mark = 0;
|
2019-08-30 14:29:08 +00:00
|
|
|
|
2019-08-30 14:30:28 +00:00
|
|
|
/// Number of mark in data from which skip indices have to start
|
2019-08-30 14:29:08 +00:00
|
|
|
/// aggregation. I.e. it's data mark number, not skip indices mark.
|
|
|
|
size_t skip_index_data_mark = 0;
|
2019-06-18 12:54:27 +00:00
|
|
|
|
2019-08-19 10:37:04 +00:00
|
|
|
const bool can_use_adaptive_granularity;
|
2019-06-18 12:54:27 +00:00
|
|
|
const std::string marks_file_extension;
|
|
|
|
const bool blocks_are_granules_size;
|
|
|
|
|
|
|
|
MergeTreeIndexGranularity index_granularity;
|
|
|
|
|
|
|
|
const bool compute_granularity;
|
|
|
|
CompressionCodecPtr codec;
|
|
|
|
|
2019-07-28 11:10:35 +00:00
|
|
|
std::vector<MergeTreeIndexPtr> skip_indices;
|
2019-10-19 16:49:36 +00:00
|
|
|
std::vector<std::unique_ptr<IMergeTreeDataPartWriter::ColumnStream>> skip_indices_streams;
|
2019-07-28 11:10:35 +00:00
|
|
|
MergeTreeIndexAggregators skip_indices_aggregators;
|
|
|
|
std::vector<size_t> skip_index_filling;
|
|
|
|
|
2019-10-19 16:49:36 +00:00
|
|
|
std::unique_ptr<IMergeTreeDataPartWriter> writer;
|
|
|
|
|
2019-06-18 12:54:27 +00:00
|
|
|
const bool with_final_mark;
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|