ClickHouse/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp

64 lines
2.4 KiB
C++
Raw Normal View History

#include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
namespace DB
{
MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
2019-11-07 11:11:38 +00:00
const MergeTreeDataPartPtr & data_part, const Block & header_, bool sync_,
CompressionCodecPtr default_codec, bool skip_offsets_,
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
2019-12-09 21:21:17 +00:00
WrittenOffsetColumns * offset_columns_,
2019-11-08 14:36:10 +00:00
const MergeTreeIndexGranularity & index_granularity,
2019-12-09 21:21:17 +00:00
const MergeTreeIndexGranularityInfo * index_granularity_info,
const String & filename_suffix)
2019-11-07 11:11:38 +00:00
: IMergedBlockOutputStream(data_part),
2019-12-09 21:21:17 +00:00
header(header_), sync(sync_)
{
MergeTreeWriterSettings writer_settings(
2019-11-08 14:36:10 +00:00
data_part->storage.global_context.getSettings(),
index_granularity_info ? index_granularity_info->is_adaptive : data_part->storage.canUseAdaptiveGranularity());
2019-12-09 21:21:17 +00:00
writer_settings.filename_suffix = filename_suffix;
writer_settings.skip_offsets = skip_offsets_;
2019-11-08 14:36:10 +00:00
2020-01-17 12:24:27 +00:00
writer = data_part->getWriter(header.getNamesAndTypesList(), indices_to_recalc,
default_codec,std::move(writer_settings), index_granularity);
writer->setWrittenOffsetColumns(offset_columns_);
2019-11-08 14:36:10 +00:00
writer->initSkipIndices();
}
void MergedColumnOnlyOutputStream::write(const Block & block)
{
2020-01-16 16:15:01 +00:00
std::unordered_set<String> skip_indexes_column_names_set;
for (const auto & index : writer->getSkipIndices())
std::copy(index->columns.cbegin(), index->columns.cend(),
std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end()));
Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end());
2019-12-09 21:21:17 +00:00
Block skip_indexes_block = getBlockAndPermute(block, skip_indexes_column_names, nullptr);
size_t rows = block.rows();
2019-06-24 15:07:01 +00:00
if (!rows)
return;
2019-12-09 21:21:17 +00:00
writer->write(block);
writer->calculateAndSerializeSkipIndices(skip_indexes_block, rows);
writer->next();
}
void MergedColumnOnlyOutputStream::writeSuffix()
{
throw Exception("Method writeSuffix is not supported by MergedColumnOnlyOutputStream", ErrorCodes::NOT_IMPLEMENTED);
}
MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums()
{
/// Finish columns serialization.
MergeTreeData::DataPart::Checksums checksums;
2019-11-18 15:18:50 +00:00
writer->finishDataSerialization(checksums, sync);
2019-12-09 21:21:17 +00:00
writer->finishSkipIndicesSerialization(checksums);
return checksums;
}
}