2019-10-11 15:37:16 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
|
2019-12-16 14:51:19 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
2020-01-09 15:46:32 +00:00
|
|
|
#include <DataTypes/DataTypeArray.h>
|
|
|
|
#include <DataTypes/NestedUtils.h>
|
2019-10-11 15:37:16 +00:00
|
|
|
#include <Poco/File.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-10-16 18:27:53 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int CANNOT_READ_ALL_DATA;
|
2019-10-16 18:27:53 +00:00
|
|
|
extern const int ARGUMENT_OUT_OF_BOUND;
|
2020-08-03 16:10:09 +00:00
|
|
|
extern const int MEMORY_LIMIT_EXCEEDED;
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
2020-02-10 20:27:06 +00:00
|
|
|
|
|
|
|
MergeTreeReaderCompact::MergeTreeReaderCompact(
|
2020-03-23 02:12:31 +00:00
|
|
|
DataPartCompactPtr data_part_,
|
|
|
|
NamesAndTypesList columns_,
|
2020-06-17 16:39:58 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
2020-02-10 20:27:06 +00:00
|
|
|
UncompressedCache * uncompressed_cache_,
|
|
|
|
MarkCache * mark_cache_,
|
2020-03-23 02:12:31 +00:00
|
|
|
MarkRanges mark_ranges_,
|
|
|
|
MergeTreeReaderSettings settings_,
|
|
|
|
ValueSizeMap avg_value_size_hints_,
|
2020-02-10 20:27:06 +00:00
|
|
|
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
|
|
|
clockid_t clock_type_)
|
2020-06-17 16:39:58 +00:00
|
|
|
: IMergeTreeReader(
|
|
|
|
std::move(data_part_),
|
|
|
|
std::move(columns_),
|
|
|
|
metadata_snapshot_,
|
|
|
|
uncompressed_cache_,
|
|
|
|
mark_cache_,
|
|
|
|
std::move(mark_ranges_),
|
|
|
|
std::move(settings_),
|
|
|
|
std::move(avg_value_size_hints_))
|
2020-02-27 16:47:40 +00:00
|
|
|
, marks_loader(
|
2020-06-17 16:39:58 +00:00
|
|
|
data_part->volume->getDisk(),
|
|
|
|
mark_cache,
|
|
|
|
data_part->index_granularity_info.getMarksFilePath(data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME),
|
|
|
|
data_part->getMarksCount(),
|
|
|
|
data_part->index_granularity_info,
|
|
|
|
settings.save_marks_in_cache,
|
|
|
|
data_part->getColumns().size())
|
2019-10-11 15:37:16 +00:00
|
|
|
{
|
2020-08-03 16:10:09 +00:00
|
|
|
try
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2020-08-03 16:10:09 +00:00
|
|
|
size_t columns_num = columns.size();
|
2020-07-10 23:33:36 +00:00
|
|
|
|
2020-08-03 16:10:09 +00:00
|
|
|
column_positions.resize(columns_num);
|
|
|
|
read_only_offsets.resize(columns_num);
|
|
|
|
auto name_and_type = columns.begin();
|
|
|
|
for (size_t i = 0; i < columns_num; ++i, ++name_and_type)
|
2020-07-10 15:57:10 +00:00
|
|
|
{
|
2020-10-06 12:46:17 +00:00
|
|
|
auto column_from_part = getColumnFromPart(*name_and_type);
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2021-01-12 23:20:32 +00:00
|
|
|
auto position = data_part->getColumnPosition(column_from_part.name);
|
2020-10-06 12:46:17 +00:00
|
|
|
if (!position && typeid_cast<const DataTypeArray *>(column_from_part.type.get()))
|
2020-08-03 16:10:09 +00:00
|
|
|
{
|
|
|
|
/// If array of Nested column is missing in part,
|
2020-08-04 09:42:51 +00:00
|
|
|
/// we have to read its offsets if they exist.
|
2020-10-06 12:46:17 +00:00
|
|
|
position = findColumnForOffsets(column_from_part.name);
|
2020-08-03 16:10:09 +00:00
|
|
|
read_only_offsets[i] = (position != std::nullopt);
|
|
|
|
}
|
|
|
|
|
|
|
|
column_positions[i] = std::move(position);
|
2020-07-14 14:03:09 +00:00
|
|
|
}
|
2019-12-12 18:55:19 +00:00
|
|
|
|
2020-08-03 16:10:09 +00:00
|
|
|
/// Do not use max_read_buffer_size, but try to lower buffer size with maximal size of granule to avoid reading much data.
|
|
|
|
auto buffer_size = getReadBufferSize(data_part, marks_loader, column_positions, all_mark_ranges);
|
|
|
|
if (!buffer_size || settings.max_read_buffer_size < buffer_size)
|
|
|
|
buffer_size = settings.max_read_buffer_size;
|
2020-01-09 17:06:34 +00:00
|
|
|
|
2020-08-03 16:10:09 +00:00
|
|
|
const String full_data_path = data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION;
|
2020-09-04 12:48:55 +00:00
|
|
|
if (uncompressed_cache)
|
2020-01-09 17:06:34 +00:00
|
|
|
{
|
2020-09-04 12:48:55 +00:00
|
|
|
auto buffer = std::make_unique<CachedCompressedReadBuffer>(
|
|
|
|
fullPath(data_part->volume->getDisk(), full_data_path),
|
|
|
|
[this, full_data_path, buffer_size]()
|
|
|
|
{
|
|
|
|
return data_part->volume->getDisk()->readFile(
|
|
|
|
full_data_path,
|
|
|
|
buffer_size,
|
|
|
|
0,
|
|
|
|
settings.min_bytes_to_use_direct_io,
|
2021-03-28 01:10:30 +00:00
|
|
|
settings.min_bytes_to_use_mmap_io,
|
|
|
|
settings.mmap_cache.get());
|
2020-09-04 12:48:55 +00:00
|
|
|
},
|
|
|
|
uncompressed_cache,
|
|
|
|
/* allow_different_codecs = */ true);
|
|
|
|
|
|
|
|
if (profile_callback_)
|
|
|
|
buffer->setProfileCallback(profile_callback_, clock_type_);
|
|
|
|
|
2021-01-25 20:29:04 +00:00
|
|
|
if (!settings.checksum_on_read)
|
|
|
|
buffer->disableChecksumming();
|
|
|
|
|
2020-09-04 12:48:55 +00:00
|
|
|
cached_buffer = std::move(buffer);
|
|
|
|
data_buffer = cached_buffer.get();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto buffer =
|
|
|
|
std::make_unique<CompressedReadBufferFromFile>(
|
|
|
|
data_part->volume->getDisk()->readFile(
|
2021-03-28 01:10:30 +00:00
|
|
|
full_data_path,
|
|
|
|
buffer_size,
|
|
|
|
0,
|
|
|
|
settings.min_bytes_to_use_direct_io,
|
|
|
|
settings.min_bytes_to_use_mmap_io,
|
|
|
|
settings.mmap_cache.get()),
|
2020-09-04 12:48:55 +00:00
|
|
|
/* allow_different_codecs = */ true);
|
|
|
|
|
|
|
|
if (profile_callback_)
|
|
|
|
buffer->setProfileCallback(profile_callback_, clock_type_);
|
2020-01-09 17:06:34 +00:00
|
|
|
|
2021-01-25 20:29:04 +00:00
|
|
|
if (!settings.checksum_on_read)
|
|
|
|
buffer->disableChecksumming();
|
|
|
|
|
2020-09-04 12:48:55 +00:00
|
|
|
non_cached_buffer = std::move(buffer);
|
|
|
|
data_buffer = non_cached_buffer.get();
|
2020-08-03 16:10:09 +00:00
|
|
|
}
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
2020-08-03 16:10:09 +00:00
|
|
|
catch (...)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2020-08-03 16:10:09 +00:00
|
|
|
storage.reportBrokenPart(data_part->name);
|
|
|
|
throw;
|
2020-01-09 17:06:34 +00:00
|
|
|
}
|
2019-10-11 15:37:16 +00:00
|
|
|
}
|
|
|
|
|
2019-12-19 13:10:57 +00:00
|
|
|
size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns)
|
2019-10-11 15:37:16 +00:00
|
|
|
{
|
2019-11-28 20:14:41 +00:00
|
|
|
if (continue_reading)
|
|
|
|
from_mark = next_mark;
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2019-11-28 20:14:41 +00:00
|
|
|
size_t read_rows = 0;
|
2019-12-12 18:55:19 +00:00
|
|
|
size_t num_columns = columns.size();
|
2020-04-14 19:47:19 +00:00
|
|
|
checkNumberOfColumns(num_columns);
|
2019-12-12 18:55:19 +00:00
|
|
|
|
2020-01-15 16:39:29 +00:00
|
|
|
MutableColumns mutable_columns(num_columns);
|
|
|
|
auto column_it = columns.begin();
|
|
|
|
for (size_t i = 0; i < num_columns; ++i, ++column_it)
|
|
|
|
{
|
|
|
|
if (!column_positions[i])
|
|
|
|
continue;
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
if (res_columns[i] == nullptr)
|
2020-04-08 16:20:52 +00:00
|
|
|
res_columns[i] = getColumnFromPart(*column_it).type->createColumn();
|
2020-01-15 16:39:29 +00:00
|
|
|
}
|
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
while (read_rows < max_rows_to_read)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2019-11-27 11:35:27 +00:00
|
|
|
size_t rows_to_read = data_part->index_granularity.getMarkRows(from_mark);
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2019-12-19 13:10:57 +00:00
|
|
|
auto name_and_type = columns.begin();
|
|
|
|
for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2020-10-26 18:55:52 +00:00
|
|
|
auto column_from_part = getColumnFromPart(*name_and_type);
|
2020-11-05 13:24:31 +00:00
|
|
|
if (!res_columns[pos])
|
2019-12-12 18:55:19 +00:00
|
|
|
continue;
|
2019-12-19 14:05:26 +00:00
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
try
|
|
|
|
{
|
2020-11-10 17:32:00 +00:00
|
|
|
auto & column = res_columns[pos];
|
2019-11-28 20:14:41 +00:00
|
|
|
size_t column_size_before_reading = column->size();
|
2019-12-12 18:55:19 +00:00
|
|
|
|
2021-01-12 23:20:32 +00:00
|
|
|
readData(column_from_part, column, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]);
|
2020-01-09 15:46:32 +00:00
|
|
|
|
|
|
|
size_t read_rows_in_column = column->size() - column_size_before_reading;
|
2019-11-28 20:14:41 +00:00
|
|
|
if (read_rows_in_column < rows_to_read)
|
2019-12-18 16:41:11 +00:00
|
|
|
throw Exception("Cannot read all data in MergeTreeReaderCompact. Rows read: " + toString(read_rows_in_column) +
|
2019-12-12 18:55:19 +00:00
|
|
|
". Rows expected: " + toString(rows_to_read) + ".", ErrorCodes::CANNOT_READ_ALL_DATA);
|
2019-11-27 11:35:27 +00:00
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
2020-08-03 16:10:09 +00:00
|
|
|
if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
|
|
|
storage.reportBrokenPart(data_part->name);
|
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
/// Better diagnostics.
|
2020-10-06 12:46:17 +00:00
|
|
|
e.addMessage("(while reading column " + column_from_part.name + ")");
|
2019-11-27 11:35:27 +00:00
|
|
|
throw;
|
|
|
|
}
|
2020-08-03 16:10:09 +00:00
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
storage.reportBrokenPart(data_part->name);
|
|
|
|
throw;
|
|
|
|
}
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
2019-11-27 11:35:27 +00:00
|
|
|
++from_mark;
|
|
|
|
read_rows += rows_to_read;
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
2019-11-28 20:14:41 +00:00
|
|
|
next_mark = from_mark;
|
2019-11-27 11:35:27 +00:00
|
|
|
|
2019-10-16 18:27:53 +00:00
|
|
|
return read_rows;
|
2019-10-11 15:37:16 +00:00
|
|
|
}
|
|
|
|
|
2019-10-16 18:27:53 +00:00
|
|
|
void MergeTreeReaderCompact::readData(
|
2020-11-10 17:32:00 +00:00
|
|
|
const NameAndTypePair & name_and_type, ColumnPtr & column,
|
2021-01-12 23:20:32 +00:00
|
|
|
size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2020-10-06 12:46:17 +00:00
|
|
|
const auto & [name, type] = name_and_type;
|
|
|
|
|
2019-12-25 17:34:23 +00:00
|
|
|
if (!isContinuousReading(from_mark, column_position))
|
2020-09-04 12:48:55 +00:00
|
|
|
seekToMark(from_mark, column_position);
|
2019-12-18 16:41:11 +00:00
|
|
|
|
2021-03-09 14:46:52 +00:00
|
|
|
auto buffer_getter = [&](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer *
|
2020-01-09 15:46:32 +00:00
|
|
|
{
|
2021-03-09 14:46:52 +00:00
|
|
|
if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != ISerialization::Substream::ArraySizes))
|
2020-01-09 15:46:32 +00:00
|
|
|
return nullptr;
|
|
|
|
|
2020-09-04 12:48:55 +00:00
|
|
|
return data_buffer;
|
2020-01-09 15:46:32 +00:00
|
|
|
};
|
|
|
|
|
2021-03-09 14:46:52 +00:00
|
|
|
ISerialization::DeserializeBinaryBulkStatePtr state;
|
|
|
|
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
|
2020-01-09 15:46:32 +00:00
|
|
|
deserialize_settings.getter = buffer_getter;
|
2020-01-17 12:24:27 +00:00
|
|
|
deserialize_settings.avg_value_size_hint = avg_value_size_hints[name];
|
2019-10-16 18:27:53 +00:00
|
|
|
|
2020-10-06 12:46:17 +00:00
|
|
|
if (name_and_type.isSubcolumn())
|
|
|
|
{
|
2020-12-22 15:03:48 +00:00
|
|
|
auto type_in_storage = name_and_type.getTypeInStorage();
|
|
|
|
ColumnPtr temp_column = type_in_storage->createColumn();
|
2020-10-06 12:46:17 +00:00
|
|
|
|
2021-03-09 14:46:52 +00:00
|
|
|
auto serialization = type_in_storage->getDefaultSerialization();
|
|
|
|
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
|
2021-03-13 18:05:18 +00:00
|
|
|
serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr);
|
2020-12-22 15:03:48 +00:00
|
|
|
column = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column);
|
2020-10-06 12:46:17 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-03-09 14:46:52 +00:00
|
|
|
auto serialization = type->getDefaultSerialization();
|
|
|
|
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
|
2021-03-13 18:05:18 +00:00
|
|
|
serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr);
|
2020-10-06 12:46:17 +00:00
|
|
|
}
|
2019-12-25 17:34:23 +00:00
|
|
|
|
2020-01-09 17:27:44 +00:00
|
|
|
/// The buffer is left in inconsistent state after reading single offsets
|
|
|
|
if (only_offsets)
|
|
|
|
last_read_granule.reset();
|
|
|
|
else
|
|
|
|
last_read_granule.emplace(from_mark, column_position);
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-09-04 12:48:55 +00:00
|
|
|
void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2019-11-20 13:33:41 +00:00
|
|
|
MarkInCompressedFile mark = marks_loader.getMark(row_index, column_index);
|
2019-10-16 18:27:53 +00:00
|
|
|
try
|
|
|
|
{
|
2020-09-04 12:48:55 +00:00
|
|
|
if (cached_buffer)
|
|
|
|
cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block);
|
|
|
|
if (non_cached_buffer)
|
|
|
|
non_cached_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block);
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
|
|
|
/// Better diagnostics.
|
|
|
|
if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND)
|
2019-11-20 13:33:41 +00:00
|
|
|
e.addMessage("(while seeking to mark (" + toString(row_index) + ", " + toString(column_index) + ")");
|
2019-10-16 18:27:53 +00:00
|
|
|
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-12-25 17:34:23 +00:00
|
|
|
bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_position)
|
2019-10-16 18:27:53 +00:00
|
|
|
{
|
2019-12-25 17:34:23 +00:00
|
|
|
if (!last_read_granule)
|
|
|
|
return false;
|
|
|
|
const auto & [last_mark, last_column] = *last_read_granule;
|
|
|
|
return (mark == last_mark && column_position == last_column + 1)
|
2020-01-16 16:15:01 +00:00
|
|
|
|| (mark == last_mark + 1 && column_position == 0 && last_column == data_part->getColumns().size() - 1);
|
2019-10-16 18:27:53 +00:00
|
|
|
}
|
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
namespace
|
|
|
|
{
|
2020-07-14 14:03:09 +00:00
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
/// A simple class that helps to iterate over 2-dim marks of compact parts.
|
|
|
|
class MarksCounter
|
2020-07-14 14:03:09 +00:00
|
|
|
{
|
2020-07-23 15:37:21 +00:00
|
|
|
public:
|
|
|
|
MarksCounter(size_t rows_num_, size_t columns_num_)
|
|
|
|
: rows_num(rows_num_), columns_num(columns_num_) {}
|
2020-07-14 14:03:09 +00:00
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
struct Iterator
|
2020-07-14 14:03:09 +00:00
|
|
|
{
|
2020-07-23 15:37:21 +00:00
|
|
|
size_t row;
|
|
|
|
size_t column;
|
|
|
|
MarksCounter * counter;
|
|
|
|
|
|
|
|
Iterator(size_t row_, size_t column_, MarksCounter * counter_)
|
|
|
|
: row(row_), column(column_), counter(counter_) {}
|
|
|
|
|
|
|
|
Iterator operator++()
|
2020-07-14 14:03:09 +00:00
|
|
|
{
|
2020-07-23 15:37:21 +00:00
|
|
|
if (column + 1 == counter->columns_num)
|
|
|
|
{
|
|
|
|
++row;
|
|
|
|
column = 0;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
++column;
|
|
|
|
}
|
|
|
|
|
|
|
|
return *this;
|
2020-07-14 14:03:09 +00:00
|
|
|
}
|
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
bool operator==(const Iterator & other) const { return row == other.row && column == other.column; }
|
|
|
|
bool operator!=(const Iterator & other) const { return !(*this == other); }
|
2020-07-14 14:03:09 +00:00
|
|
|
};
|
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
Iterator get(size_t row, size_t column) { return Iterator(row, column, this); }
|
|
|
|
Iterator end() { return get(rows_num, 0); }
|
|
|
|
|
|
|
|
private:
|
|
|
|
size_t rows_num;
|
|
|
|
size_t columns_num;
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t MergeTreeReaderCompact::getReadBufferSize(
|
2020-07-23 16:39:56 +00:00
|
|
|
const DataPartPtr & part,
|
2020-07-23 15:37:21 +00:00
|
|
|
MergeTreeMarksLoader & marks_loader,
|
|
|
|
const ColumnPositions & column_positions,
|
|
|
|
const MarkRanges & mark_ranges)
|
|
|
|
{
|
|
|
|
size_t buffer_size = 0;
|
|
|
|
size_t columns_num = column_positions.size();
|
2020-07-23 16:39:56 +00:00
|
|
|
size_t file_size = part->getFileSizeOrZero(MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION);
|
2020-07-23 15:37:21 +00:00
|
|
|
|
2020-07-23 16:39:56 +00:00
|
|
|
MarksCounter counter(part->getMarksCount(), part->getColumns().size());
|
2020-07-23 15:37:21 +00:00
|
|
|
|
|
|
|
for (const auto & mark_range : mark_ranges)
|
2020-07-14 14:03:09 +00:00
|
|
|
{
|
2020-07-23 15:37:21 +00:00
|
|
|
for (size_t mark = mark_range.begin; mark < mark_range.end; ++mark)
|
2020-07-14 14:03:09 +00:00
|
|
|
{
|
|
|
|
for (size_t i = 0; i < columns_num; ++i)
|
|
|
|
{
|
|
|
|
if (!column_positions[i])
|
|
|
|
continue;
|
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
auto it = counter.get(mark, *column_positions[i]);
|
|
|
|
size_t cur_offset = marks_loader.getMark(it.row, it.column).offset_in_compressed_file;
|
2020-07-14 14:03:09 +00:00
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
while (it != counter.end() && cur_offset == marks_loader.getMark(it.row, it.column).offset_in_compressed_file)
|
|
|
|
++it;
|
2020-07-14 14:03:09 +00:00
|
|
|
|
2020-07-23 15:37:21 +00:00
|
|
|
size_t next_offset = (it == counter.end() ? file_size : marks_loader.getMark(it.row, it.column).offset_in_compressed_file);
|
2020-07-14 14:03:09 +00:00
|
|
|
buffer_size = std::max(buffer_size, next_offset - cur_offset);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return buffer_size;
|
|
|
|
}
|
|
|
|
|
2019-10-11 15:37:16 +00:00
|
|
|
}
|