2019-10-16 18:27:53 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-10-22 10:50:17 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
constexpr auto DATA_FILE_NAME = "data";
|
|
|
|
constexpr auto DATA_FILE_EXTENSION = ".bin";
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact(
|
|
|
|
const String & part_path_,
|
|
|
|
const MergeTreeData & storage_,
|
|
|
|
const NamesAndTypesList & columns_list_,
|
2019-12-18 16:41:11 +00:00
|
|
|
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
|
2019-10-22 10:50:17 +00:00
|
|
|
const String & marks_file_extension_,
|
|
|
|
const CompressionCodecPtr & default_codec_,
|
2019-12-18 15:54:45 +00:00
|
|
|
const MergeTreeWriterSettings & settings_,
|
2019-11-07 11:11:38 +00:00
|
|
|
const MergeTreeIndexGranularity & index_granularity_)
|
2019-10-22 10:50:17 +00:00
|
|
|
: IMergeTreeDataPartWriter(part_path_,
|
|
|
|
storage_, columns_list_,
|
2019-11-07 11:11:38 +00:00
|
|
|
indices_to_recalc_, marks_file_extension_,
|
2019-11-26 09:48:22 +00:00
|
|
|
default_codec_, settings_, index_granularity_, true)
|
2019-12-02 15:21:07 +00:00
|
|
|
, squashing(storage.getSettings()->index_granularity, storage.getSettings()->index_granularity_bytes) /// FIXME
|
2019-10-22 10:50:17 +00:00
|
|
|
{
|
2019-12-09 21:21:17 +00:00
|
|
|
String data_file_name = DATA_FILE_NAME + settings.filename_suffix;
|
2019-10-22 10:50:17 +00:00
|
|
|
stream = std::make_unique<ColumnStream>(
|
2019-12-09 21:21:17 +00:00
|
|
|
data_file_name,
|
|
|
|
part_path + data_file_name, DATA_FILE_EXTENSION,
|
|
|
|
part_path + data_file_name, marks_file_extension,
|
2019-10-22 10:50:17 +00:00
|
|
|
default_codec,
|
|
|
|
settings.max_compress_block_size,
|
2019-11-05 11:53:22 +00:00
|
|
|
settings.estimated_size,
|
2019-10-22 10:50:17 +00:00
|
|
|
settings.aio_threshold);
|
|
|
|
}
|
|
|
|
|
2019-11-07 11:11:38 +00:00
|
|
|
void MergeTreeDataPartWriterCompact::write(
|
2019-10-21 17:23:06 +00:00
|
|
|
const Block & block, const IColumn::Permutation * permutation,
|
2019-11-07 11:11:38 +00:00
|
|
|
const Block & primary_key_block, const Block & skip_indexes_block)
|
2019-11-27 11:35:27 +00:00
|
|
|
{
|
2019-11-27 19:57:07 +00:00
|
|
|
/// Fill index granularity for this block
|
|
|
|
/// if it's unknown (in case of insert data or horizontal merge,
|
|
|
|
/// but not in case of vertical merge)
|
|
|
|
/// FIXME maybe it's wrong at this stage.
|
|
|
|
if (compute_granularity)
|
|
|
|
fillIndexGranularity(block);
|
|
|
|
|
|
|
|
Block result_block;
|
2019-11-27 11:35:27 +00:00
|
|
|
|
|
|
|
if (permutation)
|
|
|
|
{
|
2019-11-27 19:57:07 +00:00
|
|
|
for (const auto & it : columns_list)
|
2019-11-27 11:35:27 +00:00
|
|
|
{
|
2019-11-27 19:57:07 +00:00
|
|
|
if (primary_key_block.has(it.name))
|
|
|
|
result_block.insert(primary_key_block.getByName(it.name));
|
|
|
|
else if (skip_indexes_block.has(it.name))
|
|
|
|
result_block.insert(skip_indexes_block.getByName(it.name));
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto column = block.getByName(it.name);
|
|
|
|
column.column = column.column->permute(*permutation, 0);
|
|
|
|
result_block.insert(column);
|
|
|
|
}
|
2019-11-27 11:35:27 +00:00
|
|
|
}
|
|
|
|
}
|
2019-11-27 19:57:07 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
result_block = block;
|
|
|
|
}
|
2019-11-27 11:35:27 +00:00
|
|
|
|
2019-11-28 20:14:41 +00:00
|
|
|
if (!header)
|
|
|
|
header = result_block.cloneEmpty();
|
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
auto result = squashing.add(result_block.mutateColumns());
|
|
|
|
if (!result.ready)
|
|
|
|
return;
|
2019-12-18 16:41:11 +00:00
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
result_block = header.cloneWithColumns(std::move(result.columns));
|
|
|
|
|
|
|
|
writeBlock(result_block);
|
|
|
|
}
|
|
|
|
|
|
|
|
void MergeTreeDataPartWriterCompact::writeBlock(const Block & block)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2019-12-18 16:41:11 +00:00
|
|
|
size_t total_rows = block.rows();
|
2019-11-07 11:11:38 +00:00
|
|
|
size_t from_mark = current_mark;
|
2019-10-16 18:27:53 +00:00
|
|
|
size_t current_row = 0;
|
|
|
|
|
|
|
|
while (current_row < total_rows)
|
|
|
|
{
|
2019-11-27 11:35:27 +00:00
|
|
|
size_t rows_to_write = index_granularity.getMarkRows(from_mark);
|
|
|
|
|
2019-11-07 11:11:38 +00:00
|
|
|
if (rows_to_write)
|
|
|
|
data_written = true;
|
|
|
|
|
2019-10-22 17:42:59 +00:00
|
|
|
/// There could already be enough data to compress into the new block.
|
2019-10-31 14:44:17 +00:00
|
|
|
if (stream->compressed.offset() >= settings.min_compress_block_size)
|
2019-10-22 17:42:59 +00:00
|
|
|
stream->compressed.next();
|
2019-12-18 16:41:11 +00:00
|
|
|
|
2019-10-31 14:44:17 +00:00
|
|
|
size_t next_row = 0;
|
2019-11-27 11:35:27 +00:00
|
|
|
writeIntBinary(rows_to_write, stream->marks);
|
|
|
|
for (const auto & it : columns_list)
|
|
|
|
next_row = writeColumnSingleGranule(block.getByName(it.name), current_row, rows_to_write);
|
2019-11-05 11:53:22 +00:00
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
++from_mark;
|
2019-10-31 14:44:17 +00:00
|
|
|
current_row = next_row;
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
2019-11-07 11:11:38 +00:00
|
|
|
next_mark = from_mark;
|
|
|
|
next_index_offset = total_rows - current_row;
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
|
2019-10-21 00:28:29 +00:00
|
|
|
size_t MergeTreeDataPartWriterCompact::writeColumnSingleGranule(const ColumnWithTypeAndName & column, size_t from_row, size_t number_of_rows)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2019-12-03 14:33:56 +00:00
|
|
|
size_t old_uncompressed_size = stream->compressed.count();
|
|
|
|
|
|
|
|
writeIntBinary(stream->plain_hashing.count(), stream->marks);
|
|
|
|
writeIntBinary(stream->compressed.offset(), stream->marks);
|
|
|
|
|
2019-10-21 00:28:29 +00:00
|
|
|
IDataType::SerializeBinaryBulkStatePtr state;
|
|
|
|
IDataType::SerializeBinaryBulkSettings serialize_settings;
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2019-10-21 15:33:59 +00:00
|
|
|
serialize_settings.getter = [this](IDataType::SubstreamPath) -> WriteBuffer * { return &stream->compressed; };
|
2019-12-02 15:21:07 +00:00
|
|
|
serialize_settings.position_independent_encoding = true;
|
2019-10-21 00:28:29 +00:00
|
|
|
serialize_settings.low_cardinality_max_dictionary_size = 0;
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2019-10-21 00:28:29 +00:00
|
|
|
column.type->serializeBinaryBulkStatePrefix(serialize_settings, state);
|
|
|
|
column.type->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, state);
|
|
|
|
column.type->serializeBinaryBulkStateSuffix(serialize_settings, state);
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2019-12-12 18:55:19 +00:00
|
|
|
/// We can't calculate compressed size by single column in compact format.
|
2019-12-03 14:33:56 +00:00
|
|
|
size_t uncompressed_size = stream->compressed.count();
|
2019-12-18 16:41:11 +00:00
|
|
|
columns_sizes[column.name].add(ColumnSize{0, 0, uncompressed_size - old_uncompressed_size});
|
2019-12-03 14:33:56 +00:00
|
|
|
|
2019-10-21 00:28:29 +00:00
|
|
|
return from_row + number_of_rows;
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
2019-11-18 15:18:50 +00:00
|
|
|
void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync)
|
2019-12-18 16:41:11 +00:00
|
|
|
{
|
2019-11-27 11:35:27 +00:00
|
|
|
auto result = squashing.add({});
|
|
|
|
if (result.ready && !result.columns.empty())
|
|
|
|
writeBlock(header.cloneWithColumns(std::move(result.columns)));
|
|
|
|
|
2019-11-07 11:11:38 +00:00
|
|
|
if (with_final_mark && data_written)
|
2019-10-21 17:23:06 +00:00
|
|
|
{
|
2019-10-31 14:44:17 +00:00
|
|
|
writeIntBinary(0ULL, stream->marks);
|
2019-10-21 17:23:06 +00:00
|
|
|
for (size_t i = 0; i < columns_list.size(); ++i)
|
|
|
|
{
|
|
|
|
writeIntBinary(stream->plain_hashing.count(), stream->marks);
|
|
|
|
writeIntBinary(stream->compressed.offset(), stream->marks);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-12-03 14:33:56 +00:00
|
|
|
size_t marks_size = stream->marks.count();
|
|
|
|
for (auto it = columns_sizes.begin(); it != columns_sizes.end(); ++it)
|
|
|
|
it->second.marks = marks_size;
|
|
|
|
|
2019-10-21 17:23:06 +00:00
|
|
|
stream->finalize();
|
2019-10-22 17:42:59 +00:00
|
|
|
if (sync)
|
|
|
|
stream->sync();
|
2019-10-21 17:23:06 +00:00
|
|
|
stream->addToChecksums(checksums);
|
|
|
|
stream.reset();
|
|
|
|
}
|
|
|
|
|
2019-10-21 15:33:59 +00:00
|
|
|
}
|