2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
|
|
|
#include <IO/createWriteBufferFromFileBase.h>
|
|
|
|
#include <Common/escapeForFileName.h>
|
2017-12-25 18:58:39 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2018-01-15 19:07:47 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-04-08 01:32:05 +00:00
|
|
|
#include <Common/MemoryTracker.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
#include <Poco/File.h>
|
|
|
|
|
2016-07-21 16:22:24 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
constexpr auto DATA_FILE_EXTENSION = ".bin";
|
|
|
|
constexpr auto MARKS_FILE_EXTENSION = ".mrk";
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Implementation of IMergedBlockOutputStream.
|
|
|
|
|
|
|
|
IMergedBlockOutputStream::IMergedBlockOutputStream(
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeData & storage_,
|
|
|
|
size_t min_compress_block_size_,
|
|
|
|
size_t max_compress_block_size_,
|
2017-10-13 01:02:16 +00:00
|
|
|
CompressionSettings compression_settings_,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t aio_threshold_)
|
|
|
|
: storage(storage_),
|
|
|
|
min_compress_block_size(min_compress_block_size_),
|
|
|
|
max_compress_block_size(max_compress_block_size_),
|
|
|
|
aio_threshold(aio_threshold_),
|
2017-10-13 01:02:16 +00:00
|
|
|
compression_settings(compression_settings_)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2016-12-10 04:51:36 +00:00
|
|
|
|
2017-11-20 02:15:15 +00:00
|
|
|
void IMergedBlockOutputStream::addStreams(
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & path,
|
|
|
|
const String & name,
|
|
|
|
const IDataType & type,
|
|
|
|
size_t estimated_size,
|
|
|
|
bool skip_offsets)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
if (skip_offsets && !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes)
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-11-20 02:15:15 +00:00
|
|
|
/// Shared offsets for Nested type.
|
2017-08-07 07:31:16 +00:00
|
|
|
if (column_streams.count(stream_name))
|
|
|
|
return;
|
|
|
|
|
|
|
|
column_streams[stream_name] = std::make_unique<ColumnStream>(
|
|
|
|
stream_name,
|
|
|
|
path + stream_name, DATA_FILE_EXTENSION,
|
|
|
|
path + stream_name, MARKS_FILE_EXTENSION,
|
2017-04-01 07:20:54 +00:00
|
|
|
max_compress_block_size,
|
2017-10-13 01:02:16 +00:00
|
|
|
compression_settings,
|
2017-04-01 07:20:54 +00:00
|
|
|
estimated_size,
|
|
|
|
aio_threshold);
|
2017-08-07 07:31:16 +00:00
|
|
|
};
|
2016-07-21 16:22:24 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
type.enumerateStreams(callback, {});
|
2016-10-20 10:13:07 +00:00
|
|
|
}
|
|
|
|
|
2016-12-10 04:51:36 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
void IMergedBlockOutputStream::writeData(
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & name,
|
|
|
|
const IDataType & type,
|
|
|
|
const IColumn & column,
|
|
|
|
OffsetColumns & offset_columns,
|
|
|
|
bool skip_offsets)
|
2016-10-20 10:13:07 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = column.size();
|
2017-08-07 07:31:16 +00:00
|
|
|
size_t prev_mark = 0;
|
|
|
|
while (prev_mark < size)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
size_t limit = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
/// If there is `index_offset`, then the first mark goes not immediately, but after this number of rows.
|
|
|
|
if (prev_mark == 0 && index_offset != 0)
|
|
|
|
limit = index_offset;
|
|
|
|
else
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
limit = storage.index_granularity;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
/// Write marks.
|
|
|
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-11-20 02:15:15 +00:00
|
|
|
bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes;
|
|
|
|
if (is_offsets && skip_offsets)
|
2017-08-07 07:31:16 +00:00
|
|
|
return;
|
|
|
|
|
|
|
|
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
2017-11-20 02:15:15 +00:00
|
|
|
|
|
|
|
/// Don't write offsets more than one time for Nested type.
|
|
|
|
if (is_offsets && offset_columns.count(stream_name))
|
|
|
|
return;
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
ColumnStream & stream = *column_streams[stream_name];
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// There could already be enough data to compress into the new block.
|
|
|
|
if (stream.compressed.offset() >= min_compress_block_size)
|
|
|
|
stream.compressed.next();
|
|
|
|
|
|
|
|
writeIntBinary(stream.plain_hashing.count(), stream.marks);
|
|
|
|
writeIntBinary(stream.compressed.offset(), stream.marks);
|
2017-08-07 07:31:16 +00:00
|
|
|
}, {});
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer *
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-11-20 02:15:15 +00:00
|
|
|
bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes;
|
|
|
|
if (is_offsets && skip_offsets)
|
2017-08-07 07:31:16 +00:00
|
|
|
return nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
2017-11-20 02:15:15 +00:00
|
|
|
|
|
|
|
/// Don't write offsets more than one time for Nested type.
|
|
|
|
if (is_offsets && offset_columns.count(stream_name))
|
|
|
|
return nullptr;
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
return &column_streams[stream_name]->compressed;
|
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, prev_mark, limit, true, {});
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +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.
|
|
|
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-11-20 02:15:15 +00:00
|
|
|
bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes;
|
|
|
|
if (is_offsets && skip_offsets)
|
2017-08-07 07:31:16 +00:00
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
2017-11-20 02:15:15 +00:00
|
|
|
|
|
|
|
/// Don't write offsets more than one time for Nested type.
|
|
|
|
if (is_offsets && offset_columns.count(stream_name))
|
|
|
|
return;
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
column_streams[stream_name]->compressed.nextIfAtEnd();
|
|
|
|
}, {});
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
prev_mark += limit;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-11-20 02:15:15 +00:00
|
|
|
|
|
|
|
/// Memoize offsets for Nested types, that are already written. They will not be written again for next columns of Nested structure.
|
|
|
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path)
|
|
|
|
{
|
|
|
|
bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes;
|
|
|
|
if (is_offsets)
|
|
|
|
{
|
|
|
|
String stream_name = IDataType::getFileNameForStream(name, substream_path);
|
|
|
|
offset_columns.insert(stream_name);
|
|
|
|
}
|
|
|
|
}, {});
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2016-12-10 04:51:36 +00:00
|
|
|
|
2016-07-21 16:22:24 +00:00
|
|
|
/// Implementation of IMergedBlockOutputStream::ColumnStream.
|
|
|
|
|
|
|
|
IMergedBlockOutputStream::ColumnStream::ColumnStream(
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & escaped_column_name_,
|
|
|
|
const String & data_path,
|
|
|
|
const std::string & data_file_extension_,
|
|
|
|
const std::string & marks_path,
|
|
|
|
const std::string & marks_file_extension_,
|
|
|
|
size_t max_compress_block_size,
|
2017-10-13 01:02:16 +00:00
|
|
|
CompressionSettings compression_settings,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t estimated_size,
|
|
|
|
size_t aio_threshold) :
|
|
|
|
escaped_column_name(escaped_column_name_),
|
|
|
|
data_file_extension{data_file_extension_},
|
|
|
|
marks_file_extension{marks_file_extension_},
|
|
|
|
plain_file(createWriteBufferFromFileBase(data_path + data_file_extension, estimated_size, aio_threshold, max_compress_block_size)),
|
2017-10-13 01:02:16 +00:00
|
|
|
plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_settings), compressed(compressed_buf),
|
2017-04-01 07:20:54 +00:00
|
|
|
marks_file(marks_path + marks_file_extension, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
void IMergedBlockOutputStream::ColumnStream::finalize()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
compressed.next();
|
|
|
|
plain_file->next();
|
|
|
|
marks.next();
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void IMergedBlockOutputStream::ColumnStream::sync()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
plain_file->sync();
|
|
|
|
marks_file.sync();
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2016-12-10 06:10:29 +00:00
|
|
|
void IMergedBlockOutputStream::ColumnStream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
String name = escaped_column_name;
|
2016-07-21 16:22:24 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
checksums.files[name + data_file_extension].is_compressed = true;
|
|
|
|
checksums.files[name + data_file_extension].uncompressed_size = compressed.count();
|
|
|
|
checksums.files[name + data_file_extension].uncompressed_hash = compressed.getHash();
|
|
|
|
checksums.files[name + data_file_extension].file_size = plain_hashing.count();
|
|
|
|
checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash();
|
2016-07-21 16:22:24 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
checksums.files[name + marks_file_extension].file_size = marks.count();
|
|
|
|
checksums.files[name + marks_file_extension].file_hash = marks.getHash();
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2016-12-10 04:51:36 +00:00
|
|
|
|
2016-07-21 16:22:24 +00:00
|
|
|
/// Implementation of MergedBlockOutputStream.
|
|
|
|
|
|
|
|
MergedBlockOutputStream::MergedBlockOutputStream(
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeData & storage_,
|
|
|
|
String part_path_,
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList & columns_list_,
|
2017-10-13 01:02:16 +00:00
|
|
|
CompressionSettings compression_settings)
|
2017-04-01 07:20:54 +00:00
|
|
|
: IMergedBlockOutputStream(
|
|
|
|
storage_, storage_.context.getSettings().min_compress_block_size,
|
2017-10-13 01:02:16 +00:00
|
|
|
storage_.context.getSettings().max_compress_block_size, compression_settings,
|
2017-04-01 07:20:54 +00:00
|
|
|
storage_.context.getSettings().min_bytes_to_use_direct_io),
|
|
|
|
columns_list(columns_list_), part_path(part_path_)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
init();
|
|
|
|
for (const auto & it : columns_list)
|
2017-11-20 02:15:15 +00:00
|
|
|
addStreams(part_path, it.name, *it.type, 0, false);
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
MergedBlockOutputStream::MergedBlockOutputStream(
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeData & storage_,
|
|
|
|
String part_path_,
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList & columns_list_,
|
2017-10-13 01:02:16 +00:00
|
|
|
CompressionSettings compression_settings,
|
2017-04-01 07:20:54 +00:00
|
|
|
const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size_,
|
|
|
|
size_t aio_threshold_)
|
|
|
|
: IMergedBlockOutputStream(
|
|
|
|
storage_, storage_.context.getSettings().min_compress_block_size,
|
2017-10-13 01:02:16 +00:00
|
|
|
storage_.context.getSettings().max_compress_block_size, compression_settings,
|
2017-04-01 07:20:54 +00:00
|
|
|
aio_threshold_),
|
|
|
|
columns_list(columns_list_), part_path(part_path_)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
init();
|
|
|
|
for (const auto & it : columns_list)
|
|
|
|
{
|
|
|
|
size_t estimated_size = 0;
|
|
|
|
if (aio_threshold > 0)
|
|
|
|
{
|
|
|
|
auto it2 = merged_column_to_size_.find(it.name);
|
|
|
|
if (it2 != merged_column_to_size_.end())
|
|
|
|
estimated_size = it2->second;
|
|
|
|
}
|
2017-11-20 02:15:15 +00:00
|
|
|
addStreams(part_path, it.name, *it.type, estimated_size, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
std::string MergedBlockOutputStream::getPartPath() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return part_path;
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2018-05-07 02:01:11 +00:00
|
|
|
/// If data is pre-sorted.
|
2016-07-21 16:22:24 +00:00
|
|
|
void MergedBlockOutputStream::write(const Block & block)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
writeImpl(block, nullptr);
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2017-03-12 19:18:07 +00:00
|
|
|
/** If the data is not sorted, but we pre-calculated the permutation, after which they will be sorted.
|
2017-04-01 07:20:54 +00:00
|
|
|
* This method is used to save RAM, since you do not need to keep two blocks at once - the source and the sorted.
|
|
|
|
*/
|
2016-07-21 16:22:24 +00:00
|
|
|
void MergedBlockOutputStream::writeWithPermutation(const Block & block, const IColumn::Permutation * permutation)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
writeImpl(block, permutation);
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void MergedBlockOutputStream::writeSuffix()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Method writeSuffix is not supported by MergedBlockOutputStream", ErrorCodes::NOT_IMPLEMENTED);
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2017-08-30 19:03:19 +00:00
|
|
|
void MergedBlockOutputStream::writeSuffixAndFinalizePart(
|
|
|
|
MergeTreeData::MutableDataPartPtr & new_part,
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList * total_column_list,
|
2017-08-30 19:03:19 +00:00
|
|
|
MergeTreeData::DataPart::Checksums * additional_column_checksums)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
2017-08-30 19:03:19 +00:00
|
|
|
if (!total_column_list)
|
|
|
|
total_column_list = &columns_list;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Finish write and get checksums.
|
|
|
|
MergeTreeData::DataPart::Checksums checksums;
|
|
|
|
|
|
|
|
if (additional_column_checksums)
|
|
|
|
checksums = std::move(*additional_column_checksums);
|
|
|
|
|
2018-02-19 17:31:30 +00:00
|
|
|
if (index_stream)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
index_stream->next();
|
|
|
|
checksums.files["primary.idx"].file_size = index_stream->count();
|
|
|
|
checksums.files["primary.idx"].file_hash = index_stream->getHash();
|
|
|
|
index_stream = nullptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
for (ColumnStreams::iterator it = column_streams.begin(); it != column_streams.end(); ++it)
|
|
|
|
{
|
|
|
|
it->second->finalize();
|
|
|
|
it->second->addToChecksums(checksums);
|
|
|
|
}
|
|
|
|
|
|
|
|
column_streams.clear();
|
|
|
|
|
2017-10-19 18:20:41 +00:00
|
|
|
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
|
|
|
{
|
|
|
|
new_part->partition.store(storage, part_path, checksums);
|
2018-05-23 19:34:37 +00:00
|
|
|
if (new_part->minmax_idx.initialized)
|
|
|
|
new_part->minmax_idx.store(storage, part_path, checksums);
|
2018-08-06 16:42:43 +00:00
|
|
|
else if (rows_count)
|
|
|
|
throw Exception("MinMax index is was not initialized for new non-empty part " + new_part->name
|
|
|
|
+ ". It is a bug.", ErrorCodes::LOGICAL_ERROR);
|
2017-10-24 14:11:53 +00:00
|
|
|
|
|
|
|
WriteBufferFromFile count_out(part_path + "count.txt", 4096);
|
|
|
|
HashingWriteBuffer count_out_hashing(count_out);
|
|
|
|
writeIntText(rows_count, count_out_hashing);
|
|
|
|
count_out_hashing.next();
|
|
|
|
checksums.files["count.txt"].file_size = count_out_hashing.count();
|
|
|
|
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
{
|
|
|
|
/// Write a file with a description of columns.
|
|
|
|
WriteBufferFromFile out(part_path + "columns.txt", 4096);
|
2017-08-30 19:03:19 +00:00
|
|
|
total_column_list->writeText(out);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
{
|
|
|
|
/// Write file with checksums.
|
|
|
|
WriteBufferFromFile out(part_path + "checksums.txt", 4096);
|
|
|
|
checksums.write(out);
|
|
|
|
}
|
|
|
|
|
2017-10-24 14:11:53 +00:00
|
|
|
new_part->rows_count = rows_count;
|
|
|
|
new_part->marks_count = marks_count;
|
2017-08-30 19:03:19 +00:00
|
|
|
new_part->modification_time = time(nullptr);
|
|
|
|
new_part->columns = *total_column_list;
|
2017-12-15 20:48:46 +00:00
|
|
|
new_part->index.assign(std::make_move_iterator(index_columns.begin()), std::make_move_iterator(index_columns.end()));
|
2017-08-30 19:03:19 +00:00
|
|
|
new_part->checksums = checksums;
|
2018-03-26 14:18:04 +00:00
|
|
|
new_part->bytes_on_disk = MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_part->getFullPath());
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void MergedBlockOutputStream::init()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::File(part_path).createDirectories();
|
|
|
|
|
2018-02-19 17:31:30 +00:00
|
|
|
if (storage.hasPrimaryKey())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
index_file_stream = std::make_unique<WriteBufferFromFile>(
|
|
|
|
part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
|
|
|
|
index_stream = std::make_unique<HashingWriteBuffer>(*index_file_stream);
|
|
|
|
}
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2016-12-10 04:51:36 +00:00
|
|
|
|
2016-07-21 16:22:24 +00:00
|
|
|
void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Permutation * permutation)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
block.checkNumberOfRows();
|
|
|
|
size_t rows = block.rows();
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
/// The set of written offset columns so that you do not write shared offsets of nested structures columns several times
|
2017-04-01 07:20:54 +00:00
|
|
|
OffsetColumns offset_columns;
|
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
auto sort_columns = storage.getPrimarySortColumns();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Here we will add the columns related to the Primary Key, then write the index.
|
2018-06-30 21:35:01 +00:00
|
|
|
std::vector<ColumnWithTypeAndName> primary_columns(sort_columns.size());
|
2017-04-01 07:20:54 +00:00
|
|
|
std::map<String, size_t> primary_columns_name_to_position;
|
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-06-30 21:35:01 +00:00
|
|
|
const auto & name = sort_columns[i];
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!primary_columns_name_to_position.emplace(name, i).second)
|
|
|
|
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
|
2018-06-30 21:35:01 +00:00
|
|
|
primary_columns[i] = block.getByName(name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Reorder primary key columns in advance and add them to `primary_columns`.
|
|
|
|
if (permutation)
|
|
|
|
primary_columns[i].column = primary_columns[i].column->permute(*permutation, 0);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (index_columns.empty())
|
|
|
|
{
|
2018-06-30 21:35:01 +00:00
|
|
|
index_columns.resize(sort_columns.size());
|
|
|
|
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
|
2017-09-01 18:21:01 +00:00
|
|
|
index_columns[i] = primary_columns[i].column->cloneEmpty();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Now write the data.
|
|
|
|
for (const auto & it : columns_list)
|
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & column = block.getByName(it.name);
|
|
|
|
|
|
|
|
if (permutation)
|
|
|
|
{
|
|
|
|
auto primary_column_it = primary_columns_name_to_position.find(it.name);
|
|
|
|
if (primary_columns_name_to_position.end() != primary_column_it)
|
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
writeData(column.name, *column.type, *primary_columns[primary_column_it->second].column, offset_columns, false);
|
2017-04-01 07:20:54 +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 permutted_column = column.column->permute(*permutation, 0);
|
2017-08-07 07:31:16 +00:00
|
|
|
writeData(column.name, *column.type, *permutted_column, offset_columns, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
writeData(column.name, *column.type, *column.column, offset_columns, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-24 14:11:53 +00:00
|
|
|
rows_count += rows;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/** While filling index (index_columns), disable memory tracker.
|
|
|
|
* Because memory is allocated here (maybe in context of INSERT query),
|
|
|
|
* but then freed in completely different place (while merging parts), where query memory_tracker is not available.
|
|
|
|
* And otherwise it will look like excessively growing memory consumption in context of query.
|
|
|
|
* (observed in long INSERT SELECTs)
|
|
|
|
*/
|
|
|
|
TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker;
|
|
|
|
|
|
|
|
/// Write index. The index contains Primary Key value for each `index_granularity` row.
|
|
|
|
for (size_t i = index_offset; i < rows; i += storage.index_granularity)
|
|
|
|
{
|
2018-02-19 17:31:30 +00:00
|
|
|
if (storage.hasPrimaryKey())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
for (size_t j = 0, size = primary_columns.size(); j < size; ++j)
|
|
|
|
{
|
|
|
|
const IColumn & primary_column = *primary_columns[j].column.get();
|
2017-09-01 18:21:01 +00:00
|
|
|
index_columns[j]->insertFrom(primary_column, i);
|
|
|
|
primary_columns[j].type->serializeBinary(primary_column, i, *index_stream);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
++marks_count;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
|
|
|
|
index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity;
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
2016-12-10 04:51:36 +00:00
|
|
|
|
2016-07-21 16:22:24 +00:00
|
|
|
/// Implementation of MergedColumnOnlyOutputStream.
|
|
|
|
|
2016-12-10 04:51:36 +00:00
|
|
|
MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
|
2018-02-19 00:45:32 +00:00
|
|
|
MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_)
|
2017-04-01 07:20:54 +00:00
|
|
|
: IMergedBlockOutputStream(
|
|
|
|
storage_, storage_.context.getSettings().min_compress_block_size,
|
2017-10-13 01:02:16 +00:00
|
|
|
storage_.context.getSettings().max_compress_block_size, compression_settings,
|
2017-04-01 07:20:54 +00:00
|
|
|
storage_.context.getSettings().min_bytes_to_use_direct_io),
|
2018-02-19 00:45:32 +00:00
|
|
|
header(header_), part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_)
|
2016-07-21 16:22:24 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
void MergedColumnOnlyOutputStream::write(const Block & block)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!initialized)
|
|
|
|
{
|
|
|
|
column_streams.clear();
|
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
{
|
2017-11-20 02:15:15 +00:00
|
|
|
addStreams(part_path, block.safeGetByPosition(i).name,
|
2017-08-07 07:31:16 +00:00
|
|
|
*block.safeGetByPosition(i).type, 0, skip_offsets);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
initialized = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t rows = block.rows();
|
|
|
|
|
|
|
|
OffsetColumns offset_columns;
|
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & column = block.safeGetByPosition(i);
|
2017-08-07 07:31:16 +00:00
|
|
|
writeData(column.name, *column.type, *column.column, offset_columns, skip_offsets);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
|
|
|
|
index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity;
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void MergedColumnOnlyOutputStream::writeSuffix()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Method writeSuffix is not supported by MergedColumnOnlyOutputStream", ErrorCodes::NOT_IMPLEMENTED);
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeData::DataPart::Checksums checksums;
|
2016-07-21 16:22:24 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (auto & column_stream : column_streams)
|
|
|
|
{
|
|
|
|
column_stream.second->finalize();
|
|
|
|
if (sync)
|
|
|
|
column_stream.second->sync();
|
2016-08-29 16:57:59 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
column_stream.second->addToChecksums(checksums);
|
|
|
|
}
|
2016-07-21 16:22:24 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
column_streams.clear();
|
|
|
|
initialized = false;
|
2016-07-21 16:22:24 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return checksums;
|
2016-07-21 16:22:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|