ClickHouse/src/Storages/MergeTree/MergeTreeReaderCompact.cpp

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

466 lines
16 KiB
C++
Raw Normal View History

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>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/NestedUtils.h>
2019-10-11 15:37:16 +00:00
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;
extern const int MEMORY_LIMIT_EXCEEDED;
2019-10-16 18:27:53 +00:00
}
MergeTreeReaderCompact::MergeTreeReaderCompact(
2022-09-05 16:55:00 +00:00
MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_,
NamesAndTypesList columns_,
const StorageMetadataPtr & metadata_snapshot_,
UncompressedCache * uncompressed_cache_,
MarkCache * mark_cache_,
MarkRanges mark_ranges_,
MergeTreeReaderSettings settings_,
ThreadPool * load_marks_threadpool_,
ValueSizeMap avg_value_size_hints_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
clockid_t clock_type_)
: IMergeTreeReader(
2022-09-05 16:55:00 +00:00
data_part_info_for_read_,
columns_,
metadata_snapshot_,
uncompressed_cache_,
mark_cache_,
mark_ranges_,
settings_,
avg_value_size_hints_)
, marks_loader(
2022-09-05 16:55:00 +00:00
data_part_info_for_read_->getDataPartStorage(),
mark_cache,
2022-09-05 16:55:00 +00:00
data_part_info_for_read_->getIndexGranularityInfo().getMarksFilePath(MergeTreeDataPartCompact::DATA_FILE_NAME),
data_part_info_for_read_->getMarksCount(),
data_part_info_for_read_->getIndexGranularityInfo(),
settings.save_marks_in_cache,
2022-07-18 12:09:57 +00:00
settings.read_settings,
load_marks_threadpool_,
2022-09-05 16:55:00 +00:00
data_part_info_for_read_->getColumns().size())
2023-02-14 14:14:11 +00:00
, profile_callback(profile_callback_)
, clock_type(clock_type_)
{
}
void MergeTreeReaderCompact::initialize()
2019-10-11 15:37:16 +00:00
{
try
2019-10-16 18:27:53 +00:00
{
fillColumnPositions();
2019-12-12 18:55:19 +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.
2022-09-05 16:55:00 +00:00
auto buffer_size = getReadBufferSize(*data_part_info_for_read, marks_loader, column_positions, all_mark_ranges);
2021-08-24 22:07:06 +00:00
if (buffer_size)
settings.read_settings = settings.read_settings.adjustBufferSize(buffer_size);
if (!settings.read_settings.local_fs_buffer_size || !settings.read_settings.remote_fs_buffer_size)
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read to empty buffer.");
2022-04-12 18:59:49 +00:00
const String path = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION;
auto data_part_storage = data_part_info_for_read->getDataPartStorage();
if (uncompressed_cache)
{
auto buffer = std::make_unique<CachedCompressedReadBuffer>(
std::string(fs::path(data_part_storage->getFullPath()) / path),
[this, path, data_part_storage]()
{
return data_part_storage->readFile(
2022-05-05 09:23:23 +00:00
path,
settings.read_settings,
2022-04-12 18:59:49 +00:00
std::nullopt, std::nullopt);
},
uncompressed_cache,
/* allow_different_codecs = */ true);
2023-02-14 14:14:11 +00:00
if (profile_callback)
buffer->setProfileCallback(profile_callback, clock_type);
2021-01-25 20:29:04 +00:00
if (!settings.checksum_on_read)
buffer->disableChecksumming();
cached_buffer = std::move(buffer);
data_buffer = cached_buffer.get();
compressed_data_buffer = cached_buffer.get();
}
else
{
auto buffer =
std::make_unique<CompressedReadBufferFromFile>(
data_part_storage->readFile(
2022-04-12 18:59:49 +00:00
path,
settings.read_settings,
std::nullopt, std::nullopt),
/* allow_different_codecs = */ true);
2023-02-14 14:14:11 +00:00
if (profile_callback)
buffer->setProfileCallback(profile_callback, clock_type);
2021-01-25 20:29:04 +00:00
if (!settings.checksum_on_read)
buffer->disableChecksumming();
non_cached_buffer = std::move(buffer);
data_buffer = non_cached_buffer.get();
compressed_data_buffer = non_cached_buffer.get();
}
2019-10-16 18:27:53 +00:00
}
catch (...)
2019-10-16 18:27:53 +00:00
{
2022-09-05 16:55:00 +00:00
data_part_info_for_read->reportBroken();
throw;
}
2019-10-11 15:37:16 +00:00
}
void MergeTreeReaderCompact::fillColumnPositions()
{
size_t columns_num = columns_to_read.size();
column_positions.resize(columns_num);
read_only_offsets.resize(columns_num);
for (size_t i = 0; i < columns_num; ++i)
{
const auto & column_to_read = columns_to_read[i];
auto position = data_part_info_for_read->getColumnPosition(column_to_read.getNameInStorage());
bool is_array = isArray(column_to_read.type);
if (column_to_read.isSubcolumn())
{
auto storage_column_from_part = getColumnInPart(
{column_to_read.getNameInStorage(), column_to_read.getTypeInStorage()});
auto subcolumn_name = column_to_read.getSubcolumnName();
if (!storage_column_from_part.type->hasSubcolumn(subcolumn_name))
position.reset();
}
if (!position && is_array)
{
/// If array of Nested column is missing in part,
/// we have to read its offsets if they exist.
auto position_level = findColumnForOffsets(column_to_read);
if (position_level.has_value())
{
column_positions[i].emplace(position_level->first);
read_only_offsets[i].emplace(position_level->second);
partially_read_columns.insert(column_to_read.name);
}
}
else
column_positions[i] = std::move(position);
}
}
size_t MergeTreeReaderCompact::readRows(
2021-10-19 07:51:18 +00:00
size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns)
2019-10-11 15:37:16 +00:00
{
2023-02-14 14:14:11 +00:00
if (!initialized)
{
initialize();
initialized = true;
}
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;
2022-07-27 14:05:16 +00:00
size_t num_columns = columns_to_read.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);
2022-07-27 14:05:16 +00:00
for (size_t i = 0; i < num_columns; ++i)
2020-01-15 16:39:29 +00:00
{
2022-07-27 14:05:16 +00:00
if (column_positions[i] && res_columns[i] == nullptr)
res_columns[i] = columns_to_read[i].type->createColumn(*serializations[i]);
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
{
2022-09-05 16:55:00 +00:00
size_t rows_to_read = data_part_info_for_read->getIndexGranularity().getMarkRows(from_mark);
2019-10-16 18:27:53 +00:00
2022-07-27 14:05:16 +00:00
for (size_t pos = 0; pos < num_columns; ++pos)
2019-10-16 18:27:53 +00:00
{
2020-11-05 13:24:31 +00:00
if (!res_columns[pos])
2019-12-12 18:55:19 +00:00
continue;
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
2022-07-27 14:05:16 +00:00
readData(columns_to_read[pos], column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]);
size_t read_rows_in_column = column->size() - column_size_before_reading;
if (read_rows_in_column != rows_to_read)
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA,
"Cannot read all data in MergeTreeReaderCompact. Rows read: {}. Rows expected: {}.",
read_rows_in_column, rows_to_read);
2019-11-27 11:35:27 +00:00
}
catch (Exception & e)
{
if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
2022-09-05 16:55:00 +00:00
data_part_info_for_read->reportBroken();
2019-11-27 11:35:27 +00:00
/// Better diagnostics.
2022-07-27 14:05:16 +00:00
e.addMessage("(while reading column " + columns_to_read[pos].name + ")");
2019-11-27 11:35:27 +00:00
throw;
}
catch (...)
{
2022-09-05 16:55:00 +00:00
data_part_info_for_read->reportBroken();
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,
size_t from_mark, size_t current_task_last_mark, size_t column_position, size_t rows_to_read,
std::optional<size_t> only_offsets_level)
2019-10-16 18:27:53 +00:00
{
const auto & [name, type] = name_and_type;
2021-12-17 09:00:22 +00:00
adjustUpperBound(current_task_last_mark); /// Must go before seek.
if (!isContinuousReading(from_mark, column_position))
seekToMark(from_mark, column_position);
2021-03-09 14:46:52 +00:00
auto buffer_getter = [&](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer *
{
/// Offset stream from another column could be read, in case of current
/// column does not exists (see findColumnForOffsets() in
/// MergeTreeReaderCompact::fillColumnPositions())
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
if (only_offsets_level.has_value())
{
if (!is_offsets)
return nullptr;
/// Offset stream can be read only from columns of current level or
/// below (since it is OK to read all parent streams from the
/// alternative).
///
/// Consider the following columns in nested "root":
/// - root.array Array(UInt8) - exists
/// - root.nested_array Array(Array(UInt8)) - does not exists (only_offsets_level=1)
///
/// For root.nested_array it will try to read multiple streams:
/// - offsets (substream_path = {ArraySizes})
/// OK
/// - root.nested_array elements (substream_path = {ArrayElements, ArraySizes})
/// NOT OK - cannot use root.array offsets stream for this
///
/// Here only_offsets_level is the level of the alternative stream,
/// and substream_path.size() is the level of the current stream.
if (only_offsets_level.value() < ISerialization::getArrayLevel(substream_path))
return nullptr;
}
return data_buffer;
};
2021-03-09 14:46:52 +00:00
ISerialization::DeserializeBinaryBulkStatePtr state;
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
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
if (name_and_type.isSubcolumn())
{
const auto & type_in_storage = name_and_type.getTypeInStorage();
const auto & name_in_storage = name_and_type.getNameInStorage();
2022-07-27 14:05:16 +00:00
auto serialization = getSerializationInPart({name_in_storage, type_in_storage});
2021-05-19 01:48:46 +00:00
ColumnPtr temp_column = type_in_storage->createColumn(*serialization);
2021-03-09 14:46:52 +00:00
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
2021-03-13 18:05:18 +00:00
serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr);
auto subcolumn = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), temp_column);
2021-09-10 17:22:28 +00:00
/// TODO: Avoid extra copying.
if (column->empty())
column = subcolumn;
else
column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size());
}
else
{
2022-07-27 14:05:16 +00:00
auto serialization = getSerializationInPart(name_and_type);
2021-03-09 14:46:52 +00:00
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
2021-03-13 18:05:18 +00:00
serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr);
}
/// The buffer is left in inconsistent state after reading single offsets
if (only_offsets_level.has_value())
last_read_granule.reset();
else
last_read_granule.emplace(from_mark, column_position);
2019-10-16 18:27:53 +00:00
}
2023-02-14 14:14:11 +00:00
void MergeTreeReaderCompact::prefetchBeginOfRange(int64_t priority)
{
if (!initialized)
{
initialize();
initialized = true;
}
adjustUpperBound(all_mark_ranges.back().end);
seekToMark(all_mark_ranges.front().begin, 0);
data_buffer->prefetch(priority);
}
2019-10-16 18:27:53 +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
{
compressed_data_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;
}
}
2021-12-17 09:00:22 +00:00
void MergeTreeReaderCompact::adjustUpperBound(size_t last_mark)
{
2022-01-30 19:36:19 +00:00
size_t right_offset = 0;
2022-09-05 16:55:00 +00:00
if (last_mark < data_part_info_for_read->getMarksCount()) /// Otherwise read until the end of file
2022-01-30 19:36:19 +00:00
right_offset = marks_loader.getMark(last_mark).offset_in_compressed_file;
if (right_offset == 0)
2021-12-17 09:00:22 +00:00
{
/// If already reading till the end of file.
if (last_right_offset && *last_right_offset == 0)
return;
last_right_offset = 0; // Zero value means the end of file.
data_buffer->setReadUntilEnd();
2021-12-17 09:00:22 +00:00
}
else
{
if (last_right_offset && right_offset <= last_right_offset.value())
return;
last_right_offset = right_offset;
data_buffer->setReadUntilPosition(right_offset);
2021-12-17 09:00:22 +00:00
}
}
bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_position)
2019-10-16 18:27:53 +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)
2022-09-05 16:55:00 +00:00
|| (mark == last_mark + 1 && column_position == 0 && last_column == data_part_info_for_read->getColumns().size() - 1);
2019-10-16 18:27:53 +00:00
}
namespace
{
/// A simple class that helps to iterate over 2-dim marks of compact parts.
class MarksCounter
{
public:
MarksCounter(size_t rows_num_, size_t columns_num_)
: rows_num(rows_num_), columns_num(columns_num_) {}
struct Iterator
{
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++()
{
if (column + 1 == counter->columns_num)
{
++row;
column = 0;
}
else
{
++column;
}
return *this;
}
bool operator==(const Iterator & other) const { return row == other.row && column == other.column; }
bool operator!=(const Iterator & other) const { return !(*this == other); }
};
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(
2022-09-05 16:55:00 +00:00
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
MergeTreeMarksLoader & marks_loader,
const ColumnPositions & column_positions,
const MarkRanges & mark_ranges)
{
size_t buffer_size = 0;
size_t columns_num = column_positions.size();
2022-09-05 16:55:00 +00:00
size_t file_size = data_part_info_for_reader.getFileSizeOrZero(MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION);
2022-09-05 16:55:00 +00:00
MarksCounter counter(data_part_info_for_reader.getMarksCount(), data_part_info_for_reader.getColumns().size());
for (const auto & mark_range : mark_ranges)
{
for (size_t mark = mark_range.begin; mark < mark_range.end; ++mark)
{
for (size_t i = 0; i < columns_num; ++i)
{
if (!column_positions[i])
continue;
auto it = counter.get(mark, *column_positions[i]);
size_t cur_offset = marks_loader.getMark(it.row, it.column).offset_in_compressed_file;
while (it != counter.end() && cur_offset == marks_loader.getMark(it.row, it.column).offset_in_compressed_file)
++it;
size_t next_offset = (it == counter.end() ? file_size : marks_loader.getMark(it.row, it.column).offset_in_compressed_file);
buffer_size = std::max(buffer_size, next_offset - cur_offset);
}
}
}
return buffer_size;
}
2019-10-11 15:37:16 +00:00
}