ClickHouse/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

724 lines
30 KiB
C++
Raw Normal View History

2019-10-21 00:28:29 +00:00
#include <Storages/MergeTree/MergeTreeDataPartWriterWide.h>
#include <Interpreters/Context.h>
2020-09-21 07:17:58 +00:00
#include <Compression/CompressionFactory.h>
2020-12-14 07:28:42 +00:00
#include <Compression/CompressedReadBufferFromFile.h>
2021-03-09 14:46:52 +00:00
#include <DataTypes/Serializations/ISerialization.h>
2021-04-16 14:19:22 +00:00
#include <Common/escapeForFileName.h>
2021-04-06 15:59:03 +00:00
#include <Columns/ColumnSparse.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
2019-10-21 00:28:29 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2019-10-21 00:28:29 +00:00
2019-10-21 15:33:59 +00:00
namespace
{
constexpr auto DATA_FILE_EXTENSION = ".bin";
}
2020-12-14 12:03:49 +00:00
namespace
{
2020-12-15 10:34:28 +00:00
/// Get granules for block using index_granularity
2020-12-14 12:03:49 +00:00
Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, size_t block_rows, size_t current_mark, size_t rows_written_in_last_mark)
{
if (current_mark >= index_granularity.getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Request to get granules from mark {} but index granularity size is {}", current_mark, index_granularity.getMarksCount());
Granules result;
size_t current_row = 0;
2022-09-05 04:31:19 +00:00
/// When our last mark is not finished yet and we have to write rows into it
2020-12-14 12:03:49 +00:00
if (rows_written_in_last_mark > 0)
{
size_t rows_left_in_last_mark = index_granularity.getMarkRows(current_mark) - rows_written_in_last_mark;
2020-12-17 08:17:39 +00:00
size_t rows_left_in_block = block_rows - current_row;
result.emplace_back(Granule{
.start_row = current_row,
2020-12-18 14:44:31 +00:00
.rows_to_write = std::min(rows_left_in_block, rows_left_in_last_mark),
2020-12-17 08:17:39 +00:00
.mark_number = current_mark,
.mark_on_start = false, /// Don't mark this granule because we have already marked it
2020-12-18 14:44:31 +00:00
.is_complete = (rows_left_in_block >= rows_left_in_last_mark),
2020-12-17 08:17:39 +00:00
});
2020-12-18 14:44:31 +00:00
current_row += result.back().rows_to_write;
2022-09-05 04:31:19 +00:00
++current_mark;
2020-12-14 12:03:49 +00:00
}
2020-12-15 10:34:28 +00:00
/// Calculating normal granules for block
2020-12-14 12:03:49 +00:00
while (current_row < block_rows)
{
2020-12-17 08:17:39 +00:00
size_t expected_rows_in_mark = index_granularity.getMarkRows(current_mark);
size_t rows_left_in_block = block_rows - current_row;
2020-12-15 10:34:28 +00:00
/// If we have less rows in block than expected in granularity
/// save incomplete granule
2020-12-17 08:17:39 +00:00
result.emplace_back(Granule{
.start_row = current_row,
2020-12-18 14:44:31 +00:00
.rows_to_write = std::min(rows_left_in_block, expected_rows_in_mark),
2020-12-17 08:17:39 +00:00
.mark_number = current_mark,
.mark_on_start = true,
2020-12-18 14:44:31 +00:00
.is_complete = (rows_left_in_block >= expected_rows_in_mark),
2020-12-17 08:17:39 +00:00
});
2020-12-18 14:44:31 +00:00
current_row += result.back().rows_to_write;
2022-09-05 04:31:19 +00:00
++current_mark;
2020-12-14 12:03:49 +00:00
}
return result;
}
}
2019-10-21 15:33:59 +00:00
MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide(
2022-10-22 22:51:59 +00:00
const MergeTreeMutableDataPartPtr & 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_)
2022-10-22 22:51:59 +00:00
: MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_,
indices_to_recalc_, marks_file_extension_,
2021-10-29 17:21:02 +00:00
default_codec_, settings_, index_granularity_)
2019-10-21 15:33:59 +00:00
{
2020-06-17 12:39:20 +00:00
const auto & columns = metadata_snapshot->getColumns();
2019-10-21 15:33:59 +00:00
for (const auto & it : columns_list)
addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec));
2019-10-21 15:33:59 +00:00
}
void MergeTreeDataPartWriterWide::addStreams(
const NameAndTypePair & column,
2021-01-10 00:28:59 +00:00
const ASTPtr & effective_codec_desc)
2019-10-21 15:33:59 +00:00
{
ISerialization::StreamCallback callback = [&](const auto & substream_path)
2019-10-21 15:33:59 +00:00
{
assert(!substream_path.empty());
2021-03-09 14:46:52 +00:00
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
2021-04-06 15:59:03 +00:00
2019-10-21 15:33:59 +00:00
/// Shared offsets for Nested type.
if (column_streams.contains(stream_name))
2019-10-21 15:33:59 +00:00
return;
const auto & subtype = substream_path.back().data.type;
2020-09-21 14:22:13 +00:00
CompressionCodecPtr compression_codec;
2020-09-29 22:23:42 +00:00
/// If we can use special codec then just get it
2021-03-09 14:46:52 +00:00
if (ISerialization::isSpecialCompressionAllowed(substream_path))
compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, subtype.get(), default_codec);
2021-03-09 14:46:52 +00:00
else /// otherwise return only generic codecs and don't use info about the` data_type
2020-09-21 14:47:10 +00:00
compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true);
2020-09-21 07:17:58 +00:00
2022-05-29 07:28:02 +00:00
ParserCodec codec_parser;
auto ast = parseQuery(codec_parser, "(" + Poco::toUpper(settings.marks_compression_codec) + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
CompressionCodecPtr marks_compression_codec = CompressionCodecFactory::instance().get(ast, nullptr);
2020-02-19 14:07:36 +00:00
column_streams[stream_name] = std::make_unique<Stream>(
2019-10-21 15:33:59 +00:00
stream_name,
data_part->getDataPartStoragePtr(),
2022-04-12 18:59:49 +00:00
stream_name, DATA_FILE_EXTENSION,
stream_name, marks_file_extension,
2020-09-21 11:24:10 +00:00
compression_codec,
2022-03-21 13:56:38 +00:00
settings.max_compress_block_size,
2022-05-29 07:28:02 +00:00
marks_compression_codec,
settings.marks_compress_block_size,
2022-03-21 13:56:38 +00:00
settings.query_write_settings);
2019-10-21 15:33:59 +00:00
};
ISerialization::SubstreamPath path;
data_part->getSerialization(column.name)->enumerateStreams(callback, column.type);
2019-10-21 15:33:59 +00:00
}
2021-03-09 14:46:52 +00:00
ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGetter(
const NameAndTypePair & column, WrittenOffsetColumns & offset_columns) const
2019-10-21 15:33:59 +00:00
{
2021-03-09 14:46:52 +00:00
return [&, this] (const ISerialization::SubstreamPath & substream_path) -> WriteBuffer *
2019-10-21 15:33:59 +00:00
{
2021-03-09 14:46:52 +00:00
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
2019-10-21 15:33:59 +00:00
2021-03-09 14:46:52 +00:00
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
2019-10-21 15:33:59 +00:00
/// Don't write offsets more than one time for Nested type.
if (is_offsets && offset_columns.contains(stream_name))
2019-10-21 15:33:59 +00:00
return nullptr;
2022-09-05 04:31:19 +00:00
return &column_streams.at(stream_name)->compressed_hashing;
2019-10-21 15:33:59 +00:00
};
}
2020-12-18 14:44:31 +00:00
2020-12-15 09:54:48 +00:00
void MergeTreeDataPartWriterWide::shiftCurrentMark(const Granules & granules_written)
{
auto last_granule = granules_written.back();
2020-12-15 10:34:28 +00:00
/// If we didn't finished last granule than we will continue to write it from new block
2020-12-18 14:44:31 +00:00
if (!last_granule.is_complete)
2020-12-15 09:54:48 +00:00
{
if (settings.can_use_adaptive_granularity && settings.blocks_are_granules_size)
2021-01-11 12:03:00 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incomplete granules are not allowed while blocks are granules size. "
"Mark number {} (rows {}), rows written in last mark {}, rows to write in last mark from block {} (from row {}), total marks currently {}",
last_granule.mark_number, index_granularity.getMarkRows(last_granule.mark_number), rows_written_in_last_mark,
last_granule.rows_to_write, last_granule.start_row, index_granularity.getMarksCount());
2020-12-15 10:34:28 +00:00
/// Shift forward except last granule
2020-12-15 09:54:48 +00:00
setCurrentMark(getCurrentMark() + granules_written.size() - 1);
bool still_in_the_same_granule = granules_written.size() == 1;
2020-12-15 10:34:28 +00:00
/// We wrote whole block in the same granule, but didn't finished it.
/// So add written rows to rows written in last_mark
2020-12-15 09:54:48 +00:00
if (still_in_the_same_granule)
2020-12-18 14:44:31 +00:00
rows_written_in_last_mark += last_granule.rows_to_write;
2020-12-15 09:54:48 +00:00
else
2020-12-18 14:44:31 +00:00
rows_written_in_last_mark = last_granule.rows_to_write;
2020-12-15 09:54:48 +00:00
}
else
{
setCurrentMark(getCurrentMark() + granules_written.size());
rows_written_in_last_mark = 0;
}
}
2020-12-10 08:57:52 +00:00
void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Permutation * permutation)
2019-10-21 00:28:29 +00:00
{
2019-11-07 11:11:38 +00:00
/// Fill index granularity for this block
/// if it's unknown (in case of insert data or horizontal merge,
2021-01-11 12:03:00 +00:00
/// but not in case of vertical part of vertical merge)
2019-11-07 11:11:38 +00:00
if (compute_granularity)
2020-04-26 21:19:25 +00:00
{
size_t index_granularity_for_block = computeIndexGranularity(block);
2020-12-18 13:49:45 +00:00
if (rows_written_in_last_mark > 0)
{
size_t rows_left_in_last_mark = index_granularity.getMarkRows(getCurrentMark()) - rows_written_in_last_mark;
2020-12-21 08:24:52 +00:00
/// Previous granularity was much bigger than our new block's
/// granularity let's adjust it, because we want add new
/// heavy-weight blocks into small old granule.
2020-12-18 13:49:45 +00:00
if (rows_left_in_last_mark > index_granularity_for_block)
2020-12-21 08:24:52 +00:00
{
/// We have already written more rows than granularity of our block.
/// adjust last mark rows and flush to disk.
if (rows_written_in_last_mark >= index_granularity_for_block)
adjustLastMarkIfNeedAndFlushToDisk(rows_written_in_last_mark);
2021-01-15 18:50:30 +00:00
else /// We still can write some rows from new block into previous granule. So the granule size will be block granularity size.
adjustLastMarkIfNeedAndFlushToDisk(index_granularity_for_block);
2020-12-21 08:24:52 +00:00
}
2020-12-18 13:49:45 +00:00
}
2020-04-26 21:19:25 +00:00
fillIndexGranularity(index_granularity_for_block, block.rows());
}
2019-11-27 19:57:07 +00:00
2021-03-12 16:33:41 +00:00
Block block_to_write = block;
auto granules_to_write = getGranulesToWrite(index_granularity, block_to_write.rows(), getCurrentMark(), rows_written_in_last_mark);
2020-12-11 13:20:19 +00:00
2019-12-09 21:21:17 +00:00
auto offset_columns = written_offset_columns ? *written_offset_columns : WrittenOffsetColumns{};
2020-12-10 08:57:52 +00:00
Block primary_key_block;
if (settings.rewrite_primary_key)
primary_key_block = getBlockAndPermute(block, metadata_snapshot->getPrimaryKeyColumns(), permutation);
2020-12-11 08:41:02 +00:00
Block skip_indexes_block = getBlockAndPermute(block, getSkipIndicesColumns(), permutation);
2019-10-21 00:28:29 +00:00
auto it = columns_list.begin();
for (size_t i = 0; i < columns_list.size(); ++i, ++it)
{
2021-04-06 15:59:03 +00:00
auto & column = block_to_write.getByName(it->name);
2022-07-27 14:05:16 +00:00
if (data_part->getSerialization(it->name)->getKind() != ISerialization::Kind::SPARSE)
2021-04-06 15:59:03 +00:00
column.column = recursiveRemoveSparse(column.column);
2019-10-21 00:28:29 +00:00
if (permutation)
{
if (primary_key_block.has(it->name))
{
const auto & primary_column = *primary_key_block.getByName(it->name).column;
writeColumn(*it, primary_column, offset_columns, granules_to_write);
2019-10-21 00:28:29 +00:00
}
else if (skip_indexes_block.has(it->name))
{
const auto & index_column = *skip_indexes_block.getByName(it->name).column;
writeColumn(*it, index_column, offset_columns, granules_to_write);
2019-10-21 00:28:29 +00:00
}
else
{
/// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM.
ColumnPtr permuted_column = column.column->permute(*permutation, 0);
writeColumn(*it, *permuted_column, offset_columns, granules_to_write);
2019-10-21 00:28:29 +00:00
}
}
else
{
writeColumn(*it, *column.column, offset_columns, granules_to_write);
2019-10-21 00:28:29 +00:00
}
}
2020-12-10 08:57:52 +00:00
if (settings.rewrite_primary_key)
2020-12-11 13:20:19 +00:00
calculateAndSerializePrimaryIndex(primary_key_block, granules_to_write);
calculateAndSerializeSkipIndices(skip_indexes_block, granules_to_write);
2020-12-10 08:57:52 +00:00
2020-12-15 09:54:48 +00:00
shiftCurrentMark(granules_to_write);
2019-10-21 00:28:29 +00:00
}
void MergeTreeDataPartWriterWide::writeSingleMark(
const NameAndTypePair & column,
2019-10-21 00:28:29 +00:00
WrittenOffsetColumns & offset_columns,
size_t number_of_rows)
2019-10-21 00:28:29 +00:00
{
StreamsWithMarks marks = getCurrentMarksForColumn(column, offset_columns);
2020-12-10 16:29:10 +00:00
for (const auto & mark : marks)
flushMarkToFile(mark, number_of_rows);
}
void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stream_with_mark, size_t rows_in_mark)
{
Stream & stream = *column_streams[stream_with_mark.stream_name];
2022-09-05 04:31:19 +00:00
WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing;
writeIntBinary(stream_with_mark.mark.offset_in_compressed_file, marks_out);
writeIntBinary(stream_with_mark.mark.offset_in_decompressed_block, marks_out);
2020-12-10 16:29:10 +00:00
if (settings.can_use_adaptive_granularity)
2022-09-05 04:31:19 +00:00
writeIntBinary(rows_in_mark, marks_out);
2020-12-10 16:29:10 +00:00
}
StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn(
const NameAndTypePair & column,
WrittenOffsetColumns & offset_columns)
2020-12-10 16:29:10 +00:00
{
StreamsWithMarks result;
2022-07-27 14:05:16 +00:00
data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path)
2020-12-10 16:29:10 +00:00
{
2021-03-09 14:46:52 +00:00
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
2019-10-21 00:28:29 +00:00
2021-03-09 14:46:52 +00:00
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
/// Don't write offsets more than one time for Nested type.
if (is_offsets && offset_columns.contains(stream_name))
2020-12-10 16:29:10 +00:00
return;
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
Stream & stream = *column_streams[stream_name];
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
/// There could already be enough data to compress into the new block.
2022-09-05 04:31:19 +00:00
if (stream.compressed_hashing.offset() >= settings.min_compress_block_size)
stream.compressed_hashing.next();
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
StreamNameAndMark stream_with_mark;
stream_with_mark.stream_name = stream_name;
stream_with_mark.mark.offset_in_compressed_file = stream.plain_hashing.count();
2022-09-05 04:31:19 +00:00
stream_with_mark.mark.offset_in_decompressed_block = stream.compressed_hashing.offset();
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
result.push_back(stream_with_mark);
});
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
return result;
2019-10-21 00:28:29 +00:00
}
2020-12-11 13:20:19 +00:00
void MergeTreeDataPartWriterWide::writeSingleGranule(
const NameAndTypePair & name_and_type,
2019-10-21 00:28:29 +00:00
const IColumn & column,
WrittenOffsetColumns & offset_columns,
2021-03-09 14:46:52 +00:00
ISerialization::SerializeBinaryBulkStatePtr & serialization_state,
ISerialization::SerializeBinaryBulkSettings & serialize_settings,
2020-12-15 10:34:28 +00:00
const Granule & granule)
2019-10-21 00:28:29 +00:00
{
2022-07-27 14:05:16 +00:00
const auto & serialization = data_part->getSerialization(name_and_type.name);
2021-03-09 14:46:52 +00:00
serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, serialize_settings, serialization_state);
2019-10-21 00:28:29 +00:00
/// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one.
2021-03-09 14:46:52 +00:00
serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path)
2019-10-21 00:28:29 +00:00
{
2021-03-09 14:46:52 +00:00
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
2019-10-21 00:28:29 +00:00
2021-03-09 14:46:52 +00:00
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path);
2019-10-21 00:28:29 +00:00
/// Don't write offsets more than one time for Nested type.
if (is_offsets && offset_columns.contains(stream_name))
2019-10-21 00:28:29 +00:00
return;
2022-09-05 04:31:19 +00:00
column_streams[stream_name]->compressed_hashing.nextIfAtEnd();
});
2019-10-21 00:28:29 +00:00
}
/// Column must not be empty. (column.size() !== 0)
void MergeTreeDataPartWriterWide::writeColumn(
const NameAndTypePair & name_and_type,
2019-10-21 00:28:29 +00:00
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-18 13:49:45 +00:00
if (granules.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty granules for column {}, current mark {}", backQuoteIfNeed(name_and_type.name), getCurrentMark());
2020-12-18 13:49:45 +00:00
const auto & [name, type] = name_and_type;
2019-11-07 11:11:38 +00:00
auto [it, inserted] = serialization_states.emplace(name, nullptr);
2022-07-27 14:05:16 +00:00
auto serialization = data_part->getSerialization(name_and_type.name);
2020-12-15 10:34:28 +00:00
2019-11-07 11:11:38 +00:00
if (inserted)
{
2021-03-09 14:46:52 +00:00
ISerialization::SerializeBinaryBulkSettings serialize_settings;
serialize_settings.getter = createStreamGetter(name_and_type, offset_columns);
serialization->serializeBinaryBulkStatePrefix(column, serialize_settings, it->second);
2019-11-07 11:11:38 +00:00
}
const auto & global_settings = storage.getContext()->getSettingsRef();
2021-03-09 14:46:52 +00:00
ISerialization::SerializeBinaryBulkSettings serialize_settings;
serialize_settings.getter = createStreamGetter(name_and_type, offset_columns);
2019-12-12 18:55:19 +00:00
serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size;
serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0;
2019-10-21 00:28:29 +00:00
2020-12-11 13:20:19 +00:00
for (const auto & granule : granules)
2019-10-21 00:28:29 +00:00
{
2020-12-18 14:44:31 +00:00
data_written = true;
2020-12-14 11:06:02 +00:00
2020-12-18 13:49:45 +00:00
if (granule.mark_on_start)
{
if (last_non_written_marks.contains(name))
2020-12-18 13:49:45 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "We have to add new mark for column, but already have non written mark. Current mark {}, total marks {}, offset {}", getCurrentMark(), index_granularity.getMarksCount(), rows_written_in_last_mark);
last_non_written_marks[name] = getCurrentMarksForColumn(name_and_type, offset_columns);
2020-12-18 13:49:45 +00:00
}
2019-11-07 11:11:38 +00:00
2020-12-14 11:06:02 +00:00
writeSingleGranule(
name_and_type,
2020-12-14 11:06:02 +00:00
column,
offset_columns,
it->second,
serialize_settings,
2020-12-15 10:34:28 +00:00
granule
2019-10-21 00:28:29 +00:00
);
2020-12-18 13:49:45 +00:00
2020-12-18 14:44:31 +00:00
if (granule.is_complete)
2020-12-18 13:49:45 +00:00
{
2020-12-20 08:01:39 +00:00
auto marks_it = last_non_written_marks.find(name);
if (marks_it == last_non_written_marks.end())
2020-12-18 13:49:45 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "No mark was saved for incomplete granule for column {}", backQuoteIfNeed(name));
2020-12-20 08:01:39 +00:00
for (const auto & mark : marks_it->second)
2020-12-18 13:49:45 +00:00
flushMarkToFile(mark, index_granularity.getMarkRows(granule.mark_number));
2020-12-20 08:01:39 +00:00
last_non_written_marks.erase(marks_it);
2020-12-18 13:49:45 +00:00
}
2019-10-21 00:28:29 +00:00
}
serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
2019-10-21 00:28:29 +00:00
{
2021-03-09 14:46:52 +00:00
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
2019-10-21 00:28:29 +00:00
if (is_offsets)
{
2021-03-09 14:46:52 +00:00
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path);
2019-10-21 00:28:29 +00:00
offset_columns.insert(stream_name);
}
});
2019-10-21 00:28:29 +00:00
}
2020-12-14 07:28:42 +00:00
2021-11-02 03:03:52 +00:00
void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePair & name_type)
2020-12-14 07:28:42 +00:00
{
2021-11-02 03:03:52 +00:00
const auto & [name, type] = name_type;
2022-07-27 14:05:16 +00:00
const auto & serialization = data_part->getSerialization(name_type.name);
2021-11-02 03:03:52 +00:00
if (!type->isValueRepresentedByNumber() || type->haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type->getName());
2020-12-14 07:28:42 +00:00
2021-04-16 14:19:22 +00:00
String escaped_name = escapeForFileName(name);
2022-04-12 18:59:49 +00:00
String mrk_path = escaped_name + marks_file_extension;
String bin_path = escaped_name + DATA_FILE_EXTENSION;
2022-02-04 16:05:02 +00:00
/// Some columns may be removed because of ttl. Skip them.
if (!data_part->getDataPartStorage().exists(mrk_path))
2022-02-04 16:05:02 +00:00
return;
auto mrk_file_in = data_part->getDataPartStorage().readFile(mrk_path, {}, std::nullopt, std::nullopt);
std::unique_ptr<ReadBuffer> mrk_in;
2022-09-05 05:26:58 +00:00
if (data_part->index_granularity_info.mark_type.compressed)
mrk_in = std::make_unique<CompressedReadBufferFromFile>(std::move(mrk_file_in));
else
mrk_in = std::move(mrk_file_in);
DB::CompressedReadBufferFromFile bin_in(data_part->getDataPartStorage().readFile(bin_path, {}, std::nullopt, std::nullopt));
2020-12-14 07:28:42 +00:00
bool must_be_last = false;
UInt64 offset_in_compressed_file = 0;
UInt64 offset_in_decompressed_block = 0;
UInt64 index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity;
2020-12-14 07:28:42 +00:00
size_t mark_num;
2020-12-14 11:06:02 +00:00
for (mark_num = 0; !mrk_in->eof(); ++mark_num)
2020-12-14 07:28:42 +00:00
{
2020-12-14 11:06:02 +00:00
if (mark_num > index_granularity.getMarksCount())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect number of marks in memory {}, on disk (at least) {}", index_granularity.getMarksCount(), mark_num + 1);
2020-12-14 07:28:42 +00:00
DB::readBinary(offset_in_compressed_file, *mrk_in);
DB::readBinary(offset_in_decompressed_block, *mrk_in);
2020-12-14 07:28:42 +00:00
if (settings.can_use_adaptive_granularity)
DB::readBinary(index_granularity_rows, *mrk_in);
2020-12-14 07:28:42 +00:00
else
index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity;
2020-12-14 07:28:42 +00:00
if (must_be_last)
{
if (index_granularity_rows != 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "We ran out of binary data but still have non empty mark #{} with rows number {}", mark_num, index_granularity_rows);
if (!mrk_in->eof())
2020-12-14 07:28:42 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark #{} must be last, but we still have some to read", mark_num);
2020-12-14 11:06:02 +00:00
break;
}
if (index_granularity_rows == 0)
{
2021-11-02 03:03:52 +00:00
auto column = type->createColumn();
2020-12-14 11:06:02 +00:00
2021-03-09 14:46:52 +00:00
serialization->deserializeBinaryBulk(*column, bin_in, 1000000000, 0.0);
2020-12-14 11:06:02 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Still have {} rows in bin stream, last mark #{} index granularity size {}, last rows {}", column->size(), mark_num, index_granularity.getMarksCount(), index_granularity_rows);
2020-12-14 07:28:42 +00:00
}
if (index_granularity_rows > data_part->index_granularity_info.fixed_index_granularity)
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Mark #{} has {} rows, but max fixed granularity is {}, index granularity size {}",
mark_num, index_granularity_rows, data_part->index_granularity_info.fixed_index_granularity, index_granularity.getMarksCount());
}
2020-12-14 07:28:42 +00:00
if (index_granularity_rows != index_granularity.getMarkRows(mark_num))
throw Exception(
2020-12-18 20:32:52 +00:00
ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{} (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}",
data_part->getDataPartStorage().getFullPath(), mark_num, offset_in_compressed_file, offset_in_decompressed_block, index_granularity.getMarkRows(mark_num), index_granularity_rows, index_granularity.getMarksCount());
2020-12-14 07:28:42 +00:00
2021-11-02 03:03:52 +00:00
auto column = type->createColumn();
2020-12-14 07:28:42 +00:00
2021-03-09 14:46:52 +00:00
serialization->deserializeBinaryBulk(*column, bin_in, index_granularity_rows, 0.0);
2020-12-14 07:28:42 +00:00
if (bin_in.eof())
{
must_be_last = true;
}
2020-12-18 13:49:45 +00:00
/// Now they must be equal
if (column->size() != index_granularity_rows)
2020-12-14 07:28:42 +00:00
{
2021-01-12 08:46:31 +00:00
if (must_be_last)
{
/// The only possible mark after bin.eof() is final mark. When we
/// cannot use adaptive granularity we cannot have last mark.
/// So finish validation.
if (!settings.can_use_adaptive_granularity)
break;
/// If we don't compute granularity then we are not responsible
/// for last mark (for example we mutating some column from part
/// with fixed granularity where last mark is not adjusted)
if (!compute_granularity)
continue;
}
2020-12-18 20:32:52 +00:00
2020-12-14 07:28:42 +00:00
throw Exception(
2021-01-11 12:03:00 +00:00
ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for mark #{} (compressed offset {}, decompressed offset {}), actually in bin file {}, in mrk file {}, total marks {}",
mark_num, offset_in_compressed_file, offset_in_decompressed_block, column->size(), index_granularity.getMarkRows(mark_num), index_granularity.getMarksCount());
2020-12-14 07:28:42 +00:00
}
}
if (!mrk_in->eof())
2020-12-14 11:06:02 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Still have something in marks stream, last mark #{} index granularity size {}, last rows {}", mark_num, index_granularity.getMarksCount(), index_granularity_rows);
if (!bin_in.eof())
2020-12-14 07:28:42 +00:00
{
2021-11-02 03:03:52 +00:00
auto column = type->createColumn();
2020-12-14 11:06:02 +00:00
2021-03-09 14:46:52 +00:00
serialization->deserializeBinaryBulk(*column, bin_in, 1000000000, 0.0);
2020-12-14 11:06:02 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Still have {} rows in bin stream, last mark #{} index granularity size {}, last rows {}", column->size(), mark_num, index_granularity.getMarksCount(), index_granularity_rows);
2020-12-14 07:28:42 +00:00
}
2019-10-21 00:28:29 +00:00
}
void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums)
2019-10-21 17:23:06 +00:00
{
const auto & global_settings = storage.getContext()->getSettingsRef();
2021-03-09 14:46:52 +00:00
ISerialization::SerializeBinaryBulkSettings serialize_settings;
2019-12-12 18:55:19 +00:00
serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size;
serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0;
2019-10-21 17:23:06 +00:00
WrittenOffsetColumns offset_columns;
2020-12-18 13:49:45 +00:00
if (rows_written_in_last_mark > 0)
2021-01-11 12:03:00 +00:00
{
if (settings.can_use_adaptive_granularity && settings.blocks_are_granules_size)
2021-01-11 12:03:00 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incomplete granule is not allowed while blocks are granules size even for last granule. "
"Mark number {} (rows {}), rows written for last mark {}, total marks {}",
getCurrentMark(), index_granularity.getMarkRows(getCurrentMark()), rows_written_in_last_mark, index_granularity.getMarksCount());
2020-12-21 08:24:52 +00:00
adjustLastMarkIfNeedAndFlushToDisk(rows_written_in_last_mark);
2021-01-11 12:03:00 +00:00
}
2019-10-21 17:23:06 +00:00
2019-11-18 15:18:50 +00:00
bool write_final_mark = (with_final_mark && data_written);
2019-10-21 17:23:06 +00:00
{
auto it = columns_list.begin();
for (size_t i = 0; i < columns_list.size(); ++i, ++it)
{
if (!serialization_states.empty())
{
serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns);
2022-07-27 14:05:16 +00:00
data_part->getSerialization(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]);
2019-10-21 17:23:06 +00:00
}
if (write_final_mark)
writeFinalMark(*it, offset_columns);
2019-10-21 17:23:06 +00:00
}
}
2020-03-09 01:59:08 +00:00
for (auto & stream : column_streams)
2019-10-21 17:23:06 +00:00
{
stream.second->preFinalize();
2022-02-01 02:49:40 +00:00
stream.second->addToChecksums(checksums);
}
}
void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync)
{
for (auto & stream : column_streams)
{
stream.second->finalize();
if (sync)
stream.second->sync();
2019-10-21 17:23:06 +00:00
}
column_streams.clear();
2019-10-22 17:42:59 +00:00
serialization_states.clear();
2020-12-14 07:28:42 +00:00
#ifndef NDEBUG
/// Heavy weight validation of written data. Checks that we are able to read
2021-01-11 12:03:00 +00:00
/// data according to marks. Otherwise throws LOGICAL_ERROR (equal to abort in debug mode)
2020-12-14 07:28:42 +00:00
for (const auto & column : columns_list)
{
if (column.type->isValueRepresentedByNumber()
&& !column.type->haveSubtypes()
2022-07-28 13:10:43 +00:00
&& data_part->getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT)
{
2021-11-02 03:03:52 +00:00
validateColumnOfFixedSize(column);
}
2020-12-14 07:28:42 +00:00
}
#endif
2019-10-21 17:23:06 +00:00
}
void MergeTreeDataPartWriterWide::fillChecksums(IMergeTreeDataPart::Checksums & checksums)
{
// If we don't have anything to write, skip finalization.
if (!columns_list.empty())
fillDataChecksums(checksums);
if (settings.rewrite_primary_key)
fillPrimaryIndexChecksums(checksums);
fillSkipIndicesChecksums(checksums);
}
void MergeTreeDataPartWriterWide::finish(bool sync)
2020-12-10 08:57:52 +00:00
{
// If we don't have anything to write, skip finalization.
if (!columns_list.empty())
finishDataSerialization(sync);
2020-12-10 08:57:52 +00:00
if (settings.rewrite_primary_key)
finishPrimaryIndexSerialization(sync);
2020-12-10 08:57:52 +00:00
finishSkipIndicesSerialization(sync);
2019-10-21 17:23:06 +00:00
}
void MergeTreeDataPartWriterWide::writeFinalMark(
const NameAndTypePair & column,
WrittenOffsetColumns & offset_columns)
2019-10-21 17:23:06 +00:00
{
writeSingleMark(column, offset_columns, 0);
2019-10-21 17:23:06 +00:00
/// Memoize information about offsets
2022-07-27 14:05:16 +00:00
data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path)
2019-10-21 17:23:06 +00:00
{
2021-03-09 14:46:52 +00:00
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
2019-10-21 17:23:06 +00:00
if (is_offsets)
{
2021-03-09 14:46:52 +00:00
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
2019-10-21 17:23:06 +00:00
offset_columns.insert(stream_name);
}
});
2019-10-21 17:23:06 +00:00
}
2020-12-09 18:10:09 +00:00
static void fillIndexGranularityImpl(
MergeTreeIndexGranularity & index_granularity,
2020-12-14 13:01:01 +00:00
size_t index_offset,
2020-12-09 18:10:09 +00:00
size_t index_granularity_for_block,
size_t rows_in_block)
{
2020-12-14 13:01:01 +00:00
for (size_t current_row = index_offset; current_row < rows_in_block; current_row += index_granularity_for_block)
2020-12-09 18:10:09 +00:00
index_granularity.appendMark(index_granularity_for_block);
}
void MergeTreeDataPartWriterWide::fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block)
{
2020-12-14 11:06:02 +00:00
if (getCurrentMark() < index_granularity.getMarksCount() && getCurrentMark() != index_granularity.getMarksCount() - 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to add marks, while current mark {}, but total marks {}", getCurrentMark(), index_granularity.getMarksCount());
2020-12-14 13:01:01 +00:00
size_t index_offset = 0;
if (rows_written_in_last_mark != 0)
index_offset = index_granularity.getLastMarkRows() - rows_written_in_last_mark;
2020-12-09 18:10:09 +00:00
fillIndexGranularityImpl(
index_granularity,
2020-12-14 13:01:01 +00:00
index_offset,
2020-12-09 18:10:09 +00:00
index_granularity_for_block,
rows_in_block);
}
2020-12-18 13:49:45 +00:00
2020-12-21 08:24:52 +00:00
void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_rows_in_last_mark)
2020-12-18 13:49:45 +00:00
{
2021-01-15 18:50:30 +00:00
/// We don't want to split already written granules to smaller
if (rows_written_in_last_mark > new_rows_in_last_mark)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Tryin to make mark #{} smaller ({} rows) then it already has {}",
getCurrentMark(), new_rows_in_last_mark, rows_written_in_last_mark);
/// We can adjust marks only if we computed granularity for blocks.
/// Otherwise we cannot change granularity because it will differ from
/// other columns
if (compute_granularity && settings.can_use_adaptive_granularity)
2020-12-18 20:32:52 +00:00
{
if (getCurrentMark() != index_granularity.getMarksCount() - 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Non last mark {} (with {} rows) having rows offset {}, total marks {}",
getCurrentMark(), index_granularity.getMarkRows(getCurrentMark()), rows_written_in_last_mark, index_granularity.getMarksCount());
2020-12-18 20:32:52 +00:00
index_granularity.popMark();
2020-12-21 08:24:52 +00:00
index_granularity.appendMark(new_rows_in_last_mark);
2020-12-18 20:32:52 +00:00
}
/// Last mark should be filled, otherwise it's a bug
if (last_non_written_marks.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "No saved marks for last mark {} having rows offset {}, total marks {}",
getCurrentMark(), rows_written_in_last_mark, index_granularity.getMarksCount());
2020-12-21 08:24:52 +00:00
if (rows_written_in_last_mark == new_rows_in_last_mark)
2020-12-18 20:32:52 +00:00
{
2020-12-21 08:24:52 +00:00
for (const auto & [name, marks] : last_non_written_marks)
{
for (const auto & mark : marks)
flushMarkToFile(mark, index_granularity.getMarkRows(getCurrentMark()));
}
2020-12-18 13:49:45 +00:00
2020-12-21 08:24:52 +00:00
last_non_written_marks.clear();
2020-12-18 13:49:45 +00:00
2020-12-21 08:24:52 +00:00
if (compute_granularity && settings.can_use_adaptive_granularity)
{
/// Also we add mark to each skip index because all of them
/// already accumulated all rows from current adjusting mark
for (size_t i = 0; i < skip_indices.size(); ++i)
++skip_index_accumulated_marks[i];
/// This mark completed, go further
setCurrentMark(getCurrentMark() + 1);
/// Without offset
rows_written_in_last_mark = 0;
}
2020-12-18 20:32:52 +00:00
}
2020-12-18 13:49:45 +00:00
}
2019-10-21 00:28:29 +00:00
}