2013-11-26 11:55:11 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/NamesAndTypes.h>
|
2021-08-15 10:32:56 +00:00
|
|
|
#include <Common/HashTable/HashMap.h>
|
2019-02-05 14:50:25 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeReaderStream.h>
|
2019-10-10 16:30:30 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
2013-11-26 11:55:11 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-11-20 12:43:20 +00:00
|
|
|
class IDataType;
|
2017-01-24 17:25:47 +00:00
|
|
|
|
|
|
|
/// Reads the data between pairs of marks in the same part. When reading consecutive ranges, avoids unnecessary seeks.
|
|
|
|
/// When ranges are almost consecutive, seeks are fast because they are performed inside the buffer.
|
|
|
|
/// Avoids loading the marks file if it is not needed (e.g. when reading the whole part).
|
2019-10-10 16:30:30 +00:00
|
|
|
class IMergeTreeReader : private boost::noncopyable
|
2013-11-26 11:55:11 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
using ValueSizeMap = std::map<std::string, double>;
|
2021-03-09 14:46:52 +00:00
|
|
|
using DeserializeBinaryBulkStateMap = std::map<std::string, ISerialization::DeserializeBinaryBulkStatePtr>;
|
2015-12-13 04:52:13 +00:00
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
IMergeTreeReader(
|
|
|
|
const MergeTreeData::DataPartPtr & data_part_,
|
2019-10-10 16:30:30 +00:00
|
|
|
const NamesAndTypesList & columns_,
|
2020-06-17 16:39:58 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
2019-08-03 11:02:40 +00:00
|
|
|
UncompressedCache * uncompressed_cache_,
|
|
|
|
MarkCache * mark_cache_,
|
2019-10-10 16:30:30 +00:00
|
|
|
const MarkRanges & all_mark_ranges_,
|
2019-12-18 15:54:45 +00:00
|
|
|
const MergeTreeReaderSettings & settings_,
|
2019-10-10 16:30:30 +00:00
|
|
|
const ValueSizeMap & avg_value_size_hints_ = ValueSizeMap{});
|
2014-07-23 15:24:45 +00:00
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
/// Return the number of rows has been read or zero if there is no columns to read.
|
2021-10-20 22:57:43 +00:00
|
|
|
/// If continue_reading is true, continue reading from last state, otherwise seek to from_mark.
|
|
|
|
/// current_task_last mark is needed for asynchronous reading (mainly from remote fs).
|
2021-10-15 08:36:26 +00:00
|
|
|
virtual size_t readRows(size_t from_mark, size_t current_task_last_mark,
|
|
|
|
bool continue_reading, size_t max_rows_to_read, Columns & res_columns) = 0;
|
2019-10-10 16:30:30 +00:00
|
|
|
|
2019-12-02 17:10:22 +00:00
|
|
|
virtual bool canReadIncompleteGranules() const = 0;
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
virtual ~IMergeTreeReader();
|
2013-11-26 11:55:11 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const ValueSizeMap & getAvgValueSizeHints() const;
|
2015-09-16 17:49:08 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Add columns from ordered_names that are not present in the block.
|
|
|
|
/// Missing columns are added in the order specified by ordered_names.
|
2019-09-23 19:22:02 +00:00
|
|
|
/// num_rows is needed in case if all res_columns are nullptr.
|
2022-02-09 00:18:53 +00:00
|
|
|
void fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) const;
|
2018-02-22 12:43:57 +00:00
|
|
|
/// Evaluate defaulted columns if necessary.
|
2022-02-09 00:18:53 +00:00
|
|
|
void evaluateMissingDefaults(Block additional_columns, Columns & res_columns) const;
|
2014-12-04 15:50:48 +00:00
|
|
|
|
2020-02-13 20:09:48 +00:00
|
|
|
/// If part metadata is not equal to storage metadata, than
|
|
|
|
/// try to perform conversions of columns.
|
2022-02-09 00:18:53 +00:00
|
|
|
void performRequiredConversions(Columns & res_columns) const;
|
2020-01-15 13:00:08 +00:00
|
|
|
|
2018-02-13 19:34:15 +00:00
|
|
|
const NamesAndTypesList & getColumns() const { return columns; }
|
2019-10-31 11:32:24 +00:00
|
|
|
size_t numColumnsInResult() const { return columns.size(); }
|
2018-02-13 19:34:15 +00:00
|
|
|
|
2019-03-25 16:55:48 +00:00
|
|
|
size_t getFirstMarkToRead() const
|
|
|
|
{
|
2020-02-10 12:36:01 +00:00
|
|
|
return all_mark_ranges.front().begin;
|
2019-03-25 16:55:48 +00:00
|
|
|
}
|
2013-11-26 11:55:11 +00:00
|
|
|
|
2019-11-07 11:11:38 +00:00
|
|
|
MergeTreeData::DataPartPtr data_part;
|
|
|
|
|
2019-10-10 16:30:30 +00:00
|
|
|
protected:
|
2022-07-21 14:47:19 +00:00
|
|
|
/// Returns actual column name in part, which can differ from table metadata.
|
|
|
|
String getColumnNameInPart(const NameAndTypePair & required_column) const;
|
|
|
|
/// Returns actual column name and type in part, which can differ from table metadata.
|
|
|
|
NameAndTypePair getColumnInPart(const NameAndTypePair & required_column) const;
|
2020-04-08 16:20:52 +00:00
|
|
|
|
2020-06-03 22:00:02 +00:00
|
|
|
void checkNumberOfColumns(size_t num_columns_to_read) const;
|
2020-04-14 19:47:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size.
|
|
|
|
ValueSizeMap avg_value_size_hints;
|
2018-05-21 16:21:15 +00:00
|
|
|
/// Stores states for IDataType::deserializeBinaryBulk
|
|
|
|
DeserializeBinaryBulkStateMap deserialize_binary_bulk_state_map;
|
2017-01-24 20:44:12 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Columns that are read.
|
2017-12-25 21:57:29 +00:00
|
|
|
NamesAndTypesList columns;
|
2015-04-09 00:37:08 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
UncompressedCache * uncompressed_cache;
|
|
|
|
MarkCache * mark_cache;
|
2019-10-10 16:30:30 +00:00
|
|
|
|
2019-12-18 15:54:45 +00:00
|
|
|
MergeTreeReaderSettings settings;
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2018-10-17 03:13:00 +00:00
|
|
|
const MergeTreeData & storage;
|
2020-06-17 16:39:58 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2017-04-01 07:20:54 +00:00
|
|
|
MarkRanges all_mark_ranges;
|
2015-04-02 03:08:43 +00:00
|
|
|
|
2020-06-01 17:52:09 +00:00
|
|
|
using ColumnPosition = std::optional<size_t>;
|
|
|
|
ColumnPosition findColumnForOffsets(const String & column_name) const;
|
|
|
|
|
2020-04-08 16:20:52 +00:00
|
|
|
private:
|
2020-04-03 11:09:27 +00:00
|
|
|
/// Alter conversions, which must be applied on fly if required
|
2020-03-25 18:44:08 +00:00
|
|
|
MergeTreeData::AlterConversions alter_conversions;
|
2013-11-26 11:55:11 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|