ClickHouse/src/Storages/MergeTree/MergeTreeReaderWide.cpp

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

311 lines
11 KiB
C++
Raw Normal View History

2019-10-10 16:30:30 +00:00
#include <Storages/MergeTree/MergeTreeReaderWide.h>
2020-02-25 09:49:45 +00:00
#include <Columns/ColumnArray.h>
2021-03-12 16:33:41 +00:00
#include <Columns/ColumnSparse.h>
2020-02-25 09:49:45 +00:00
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/NestedUtils.h>
2022-07-26 17:31:56 +00:00
#include <DataTypes/DataTypeNested.h>
2020-02-25 08:53:14 +00:00
#include <Interpreters/inplaceBlockConversions.h>
2022-08-30 17:47:34 +00:00
#include <Interpreters/Context.h>
2020-02-25 09:49:45 +00:00
#include <Storages/MergeTree/IMergeTreeReader.h>
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
2020-02-25 09:49:45 +00:00
#include <Common/escapeForFileName.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
2016-07-19 10:57:57 +00:00
namespace DB
{
namespace
{
constexpr auto DATA_FILE_EXTENSION = ".bin";
2016-07-19 10:57:57 +00:00
}
namespace ErrorCodes
{
2016-11-20 12:43:20 +00:00
extern const int MEMORY_LIMIT_EXCEEDED;
}
MergeTreeReaderWide::MergeTreeReaderWide(
2022-09-05 16:55:00 +00:00
MergeTreeDataPartInfoForReaderPtr data_part_info_,
NamesAndTypesList columns_,
const StorageMetadataPtr & metadata_snapshot_,
UncompressedCache * uncompressed_cache_,
MarkCache * mark_cache_,
MarkRanges mark_ranges_,
MergeTreeReaderSettings settings_,
IMergeTreeDataPart::ValueSizeMap avg_value_size_hints_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
clockid_t clock_type_)
2020-02-25 09:49:45 +00:00
: IMergeTreeReader(
2022-09-05 16:55:00 +00:00
data_part_info_,
columns_,
metadata_snapshot_,
uncompressed_cache_,
mark_cache_,
mark_ranges_,
settings_,
avg_value_size_hints_)
2016-07-19 10:57:57 +00:00
{
try
{
2022-07-27 14:05:16 +00:00
for (size_t i = 0; i < columns_to_read.size(); ++i)
addStreams(columns_to_read[i], serializations[i], profile_callback_, clock_type_);
2016-07-19 10:57:57 +00:00
}
catch (...)
{
2022-09-05 16:55:00 +00:00
data_part_info_for_read->reportBroken();
2016-07-19 10:57:57 +00:00
throw;
}
}
size_t MergeTreeReaderWide::readRows(
size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns)
2016-07-19 10:57:57 +00:00
{
size_t read_rows = 0;
2016-07-19 10:57:57 +00:00
try
{
size_t num_columns = res_columns.size();
2020-04-14 19:47:19 +00:00
checkNumberOfColumns(num_columns);
2019-09-23 19:22:02 +00:00
if (num_columns == 0)
return max_rows_to_read;
2021-03-09 14:46:52 +00:00
std::unordered_map<String, ISerialization::SubstreamsCache> caches;
2021-10-05 09:11:25 +00:00
std::unordered_set<std::string> prefetched_streams;
2022-09-05 16:55:00 +00:00
if (data_part_info_for_read->getDataPartStorage()->isStoredOnRemoteDisk() ? settings.read_settings.remote_fs_prefetch : settings.read_settings.local_fs_prefetch)
2021-07-26 00:34:36 +00:00
{
2021-08-24 23:38:08 +00:00
/// Request reading of data in advance,
/// so if reading can be asynchronous, it will also be performed in parallel for all columns.
2022-07-27 14:05:16 +00:00
for (size_t pos = 0; pos < num_columns; ++pos)
2021-07-26 00:34:36 +00:00
{
2021-08-24 23:38:08 +00:00
try
{
2022-07-27 14:05:16 +00:00
auto & cache = caches[columns_to_read[pos].getNameInStorage()];
prefetch(columns_to_read[pos], serializations[pos], from_mark, continue_reading, current_task_last_mark, cache, prefetched_streams);
2021-08-24 23:38:08 +00:00
}
catch (Exception & e)
{
/// Better diagnostics.
2022-07-27 14:05:16 +00:00
e.addMessage("(while reading column " + columns_to_read[pos].name + ")");
2021-08-24 23:38:08 +00:00
throw;
}
2021-07-26 00:34:36 +00:00
}
}
2022-07-27 14:05:16 +00:00
for (size_t pos = 0; pos < num_columns; ++pos)
2016-07-19 10:57:57 +00:00
{
2022-07-27 14:05:16 +00:00
const auto & column_to_read = columns_to_read[pos];
2019-09-23 19:22:02 +00:00
2017-01-24 17:25:47 +00:00
/// The column is already present in the block so we will append the values to the end.
2019-09-23 19:22:02 +00:00
bool append = res_columns[pos] != nullptr;
if (!append)
2022-07-27 14:05:16 +00:00
res_columns[pos] = column_to_read.type->createColumn(*serializations[pos]);
2020-11-10 17:32:00 +00:00
auto & column = res_columns[pos];
2016-07-19 10:57:57 +00:00
try
{
size_t column_size_before_reading = column->size();
2022-07-27 14:05:16 +00:00
auto & cache = caches[column_to_read.getNameInStorage()];
2021-10-05 09:11:25 +00:00
readData(
2022-07-27 14:05:16 +00:00
column_to_read, serializations[pos], column,
from_mark, continue_reading, current_task_last_mark,
2021-10-05 09:11:25 +00:00
max_rows_to_read, cache, /* was_prefetched =*/ !prefetched_streams.empty());
/// For elements of Nested, column_size_before_reading may be greater than column size
/// if offsets are not empty and were already read, but elements are empty.
2019-09-23 19:22:02 +00:00
if (!column->empty())
read_rows = std::max(read_rows, column->size() - column_size_before_reading);
2016-07-19 10:57:57 +00:00
}
catch (Exception & e)
{
2017-01-24 17:25:47 +00:00
/// Better diagnostics.
2022-07-27 14:05:16 +00:00
e.addMessage("(while reading column " + column_to_read.name + ")");
2016-07-19 10:57:57 +00:00
throw;
}
2019-09-23 19:22:02 +00:00
if (column->empty())
res_columns[pos] = nullptr;
2016-07-19 10:57:57 +00:00
}
2019-09-23 19:22:02 +00:00
/// NOTE: positions for all streams must be kept in sync.
/// In particular, even if for some streams there are no rows to be read,
/// you must ensure that no seeks are skipped and at this point they all point to to_mark.
2016-07-19 10:57:57 +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();
2017-01-24 17:25:47 +00:00
/// Better diagnostics.
2022-09-05 16:55:00 +00:00
e.addMessage("(while reading from part " + data_part_info_for_read->getDataPartStorage()->getFullPath() + " "
2019-09-23 19:22:02 +00:00
"from mark " + toString(from_mark) + " "
"with max_rows_to_read = " + toString(max_rows_to_read) + ")");
2016-07-19 10:57:57 +00:00
throw;
}
catch (...)
{
2022-09-05 16:55:00 +00:00
data_part_info_for_read->reportBroken();
2016-07-19 10:57:57 +00:00
throw;
}
return read_rows;
2016-07-19 10:57:57 +00:00
}
2022-07-27 14:05:16 +00:00
void MergeTreeReaderWide::addStreams(
const NameAndTypePair & name_and_type,
const SerializationPtr & serialization,
const ReadBufferFromFileBase::ProfileCallback & profile_callback,
clockid_t clock_type)
2016-07-19 10:57:57 +00:00
{
bool has_any_stream = false;
bool has_all_streams = true;
2021-03-09 14:46:52 +00:00
ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path)
2016-07-19 10:57:57 +00:00
{
2021-03-09 14:46:52 +00:00
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path);
if (streams.contains(stream_name))
{
has_any_stream = true;
return;
}
2022-09-05 16:55:00 +00:00
bool data_file_exists = data_part_info_for_read->getChecksums().files.contains(stream_name + DATA_FILE_EXTENSION);
/** If data file is missing then we will not try to open it.
* It is necessary since it allows to add new column to structure of the table without creating new files for old parts.
*/
if (!data_file_exists)
{
has_all_streams = false;
return;
}
has_any_stream = true;
bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys;
auto context = data_part_info_for_read->getContext();
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
streams.emplace(stream_name, std::make_unique<MergeTreeReaderStream>(
2022-09-05 16:55:00 +00:00
data_part_info_for_read->getDataPartStorage(), stream_name, DATA_FILE_EXTENSION,
data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache,
uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION),
&data_part_info_for_read->getIndexGranularityInfo(),
profile_callback, clock_type, is_lc_dict, load_marks_threadpool));
};
2022-07-27 14:05:16 +00:00
serialization->enumerateStreams(callback);
if (has_any_stream && !has_all_streams)
partially_read_columns.insert(name_and_type.name);
2016-07-19 10:57:57 +00:00
}
2021-07-26 00:34:36 +00:00
static ReadBuffer * getStream(
2021-10-05 09:11:25 +00:00
bool seek_to_start,
2021-07-26 00:34:36 +00:00
const ISerialization::SubstreamPath & substream_path,
MergeTreeReaderWide::FileStreams & streams,
const NameAndTypePair & name_and_type,
2021-10-05 09:11:25 +00:00
size_t from_mark, bool seek_to_mark,
size_t current_task_last_mark,
2021-03-09 14:46:52 +00:00
ISerialization::SubstreamsCache & cache)
2016-07-19 10:57:57 +00:00
{
2021-07-26 00:34:36 +00:00
/// If substream have already been read.
if (cache.contains(ISerialization::getSubcolumnNameForStream(substream_path)))
2021-07-26 00:34:36 +00:00
return nullptr;
2021-07-26 00:34:36 +00:00
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path);
2021-07-26 00:34:36 +00:00
auto it = streams.find(stream_name);
if (it == streams.end())
return nullptr;
2021-07-26 00:34:36 +00:00
MergeTreeReaderStream & stream = *it->second;
2022-02-16 13:35:23 +00:00
stream.adjustRightMark(current_task_last_mark);
2021-10-05 09:11:25 +00:00
if (seek_to_start)
2021-07-26 00:34:36 +00:00
stream.seekToStart();
2021-10-05 09:11:25 +00:00
else if (seek_to_mark)
2021-07-26 00:34:36 +00:00
stream.seekToMark(from_mark);
2022-04-29 17:39:06 +00:00
return stream.getDataBuffer();
2021-07-26 00:34:36 +00:00
}
2021-12-22 12:26:16 +00:00
void MergeTreeReaderWide::deserializePrefix(
const SerializationPtr & serialization,
const NameAndTypePair & name_and_type,
size_t current_task_last_mark,
ISerialization::SubstreamsCache & cache)
{
const auto & name = name_and_type.name;
if (!deserialize_binary_bulk_state_map.contains(name))
2021-12-22 12:26:16 +00:00
{
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path)
{
return getStream(/* seek_to_start = */true, substream_path, streams, name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, cache);
};
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name]);
}
}
2021-07-26 00:34:36 +00:00
void MergeTreeReaderWide::prefetch(
const NameAndTypePair & name_and_type,
2022-07-27 14:05:16 +00:00
const SerializationPtr & serialization,
2021-07-26 00:34:36 +00:00
size_t from_mark,
bool continue_reading,
size_t current_task_last_mark,
2021-10-05 09:11:25 +00:00
ISerialization::SubstreamsCache & cache,
std::unordered_set<std::string> & prefetched_streams)
2021-07-26 00:34:36 +00:00
{
2021-12-22 12:26:16 +00:00
deserializePrefix(serialization, name_and_type, current_task_last_mark, cache);
2021-07-26 00:34:36 +00:00
serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{
2021-10-05 09:11:25 +00:00
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path);
if (!prefetched_streams.contains(stream_name))
2021-10-05 09:11:25 +00:00
{
bool seek_to_mark = !continue_reading;
if (ReadBuffer * buf = getStream(false, substream_path, streams, name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache))
2021-10-05 09:11:25 +00:00
buf->prefetch();
prefetched_streams.insert(stream_name);
}
2021-07-26 00:34:36 +00:00
});
}
2021-07-26 00:34:36 +00:00
void MergeTreeReaderWide::readData(
2022-07-27 14:05:16 +00:00
const NameAndTypePair & name_and_type, const SerializationPtr & serialization, ColumnPtr & column,
size_t from_mark, bool continue_reading, size_t current_task_last_mark,
size_t max_rows_to_read, ISerialization::SubstreamsCache & cache, bool was_prefetched)
2021-07-26 00:34:36 +00:00
{
double & avg_value_size_hint = avg_value_size_hints[name_and_type.name];
2021-03-09 14:46:52 +00:00
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
2019-12-12 18:55:19 +00:00
deserialize_settings.avg_value_size_hint = avg_value_size_hint;
2021-12-22 12:26:16 +00:00
deserializePrefix(serialization, name_and_type, current_task_last_mark, cache);
2021-07-26 00:34:36 +00:00
deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path)
{
2021-10-05 09:11:25 +00:00
bool seek_to_mark = !was_prefetched && !continue_reading;
return getStream(
/* seek_to_start = */false, substream_path, streams, name_and_type, from_mark,
seek_to_mark, current_task_last_mark, cache);
2021-07-26 00:34:36 +00:00
};
2019-12-12 18:55:19 +00:00
deserialize_settings.continuous_reading = continue_reading;
2022-07-27 14:05:16 +00:00
auto & deserialize_state = deserialize_binary_bulk_state_map[name_and_type.name];
2021-03-09 14:46:52 +00:00
2021-07-26 00:34:36 +00:00
serialization->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, deserialize_settings, deserialize_state, &cache);
2020-11-10 17:32:00 +00:00
IDataType::updateAvgValueSizeHint(*column, avg_value_size_hint);
2016-07-19 10:57:57 +00:00
}
}