2015-06-24 11:03:53 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/NamesAndTypes.h>
|
|
|
|
#include <Storages/MergeTree/RangesInDataPart.h>
|
2017-03-24 13:52:50 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
2020-01-23 10:08:17 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2018-04-11 14:31:54 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2015-06-24 11:03:53 +00:00
|
|
|
#include <mutex>
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
|
|
|
|
|
2022-05-09 19:13:02 +00:00
|
|
|
/** Provides read tasks for MergeTreeThreadSelectProcessor`s in fine-grained batches, allowing for more
|
2017-04-01 07:20:54 +00:00
|
|
|
* uniform distribution of work amongst multiple threads. All parts and their ranges are divided into `threads`
|
|
|
|
* workloads with at most `sum_marks / threads` marks. Then, threads are performing reads from these workloads
|
|
|
|
* in "sequential" manner, requesting work in small batches. As soon as some thread has exhausted
|
|
|
|
* it's workload, it either is signaled that no more work is available (`do_not_steal_tasks == false`) or
|
|
|
|
* continues taking small batches from other threads' workloads (`do_not_steal_tasks == true`).
|
2015-09-09 17:39:28 +00:00
|
|
|
*/
|
2016-11-20 12:43:20 +00:00
|
|
|
class MergeTreeReadPool : private boost::noncopyable
|
2015-06-24 11:03:53 +00:00
|
|
|
{
|
2015-12-13 04:52:13 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Pull could dynamically lower (backoff) number of threads, if read operation are too slow.
|
|
|
|
* Settings for that backoff.
|
|
|
|
*/
|
|
|
|
struct BackoffSettings
|
|
|
|
{
|
|
|
|
/// Pay attention only to reads, that took at least this amount of time. If set to 0 - means backoff is disabled.
|
|
|
|
size_t min_read_latency_ms = 1000;
|
|
|
|
/// Count events, when read throughput is less than specified bytes per second.
|
|
|
|
size_t max_throughput = 1048576;
|
|
|
|
/// Do not pay attention to event, if not enough time passed since previous event.
|
|
|
|
size_t min_interval_between_events_ms = 1000;
|
|
|
|
/// Number of events to do backoff - to lower number of threads in pool.
|
|
|
|
size_t min_events = 2;
|
2020-10-27 08:55:57 +00:00
|
|
|
/// Try keeping the minimal number of threads in pool.
|
|
|
|
size_t min_concurrency = 1;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Constants above is just an example.
|
2022-03-13 12:23:51 +00:00
|
|
|
explicit BackoffSettings(const Settings & settings)
|
2017-04-01 07:20:54 +00:00
|
|
|
: min_read_latency_ms(settings.read_backoff_min_latency_ms.totalMilliseconds()),
|
|
|
|
max_throughput(settings.read_backoff_max_throughput),
|
|
|
|
min_interval_between_events_ms(settings.read_backoff_min_interval_between_events_ms.totalMilliseconds()),
|
2020-10-27 08:55:57 +00:00
|
|
|
min_events(settings.read_backoff_min_events),
|
|
|
|
min_concurrency(settings.read_backoff_min_concurrency)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
BackoffSettings() : min_read_latency_ms(0) {}
|
|
|
|
};
|
|
|
|
|
|
|
|
BackoffSettings backoff_settings;
|
2015-12-13 04:52:13 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
/** State to track numbers of slow reads.
|
|
|
|
*/
|
|
|
|
struct BackoffState
|
|
|
|
{
|
|
|
|
size_t current_threads;
|
|
|
|
Stopwatch time_since_prev_event {CLOCK_MONOTONIC_COARSE};
|
|
|
|
size_t num_events = 0;
|
2015-12-13 04:52:13 +00:00
|
|
|
|
2022-03-13 12:23:51 +00:00
|
|
|
explicit BackoffState(size_t threads) : current_threads(threads) {}
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
2015-12-13 04:52:13 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
BackoffState backoff_state;
|
2015-12-13 04:52:13 +00:00
|
|
|
|
2015-06-24 11:03:53 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
MergeTreeReadPool(
|
2022-03-13 12:23:51 +00:00
|
|
|
size_t threads_, size_t sum_marks_, size_t min_marks_for_concurrent_read_,
|
2021-07-09 03:15:41 +00:00
|
|
|
RangesInDataParts && parts_, const MergeTreeData & data_, const StorageSnapshotPtr & storage_snapshot_,
|
2021-01-27 01:48:41 +00:00
|
|
|
const PrewhereInfoPtr & prewhere_info_,
|
2021-10-20 21:56:17 +00:00
|
|
|
const Names & column_names_,
|
2019-08-03 11:02:40 +00:00
|
|
|
const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_,
|
2022-03-13 12:23:51 +00:00
|
|
|
bool do_not_steal_tasks_ = false);
|
2015-06-24 11:03:53 +00:00
|
|
|
|
2022-03-13 12:23:51 +00:00
|
|
|
MergeTreeReadTaskPtr getTask(size_t min_marks_to_read, size_t thread, const Names & ordered_names);
|
2015-06-24 11:03:53 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** Each worker could call this method and pass information about read performance.
|
|
|
|
* If read performance is too low, pool could decide to lower number of threads: do not assign more tasks to several threads.
|
|
|
|
* This allows to overcome excessive load to disk subsystem, when reads are not from page cache.
|
|
|
|
*/
|
2022-03-13 12:23:51 +00:00
|
|
|
void profileFeedback(ReadBufferFromFileBase::ProfileInfo info);
|
2015-12-13 04:52:13 +00:00
|
|
|
|
2018-01-09 01:51:08 +00:00
|
|
|
Block getHeader() const;
|
|
|
|
|
2015-12-05 07:01:18 +00:00
|
|
|
private:
|
2021-10-20 21:56:17 +00:00
|
|
|
std::vector<size_t> fillPerPartInfo(const RangesInDataParts & parts);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
void fillPerThreadInfo(
|
2022-03-13 12:23:51 +00:00
|
|
|
size_t threads, size_t sum_marks, std::vector<size_t> per_part_sum_marks,
|
|
|
|
const RangesInDataParts & parts, size_t min_marks_for_concurrent_read);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-17 03:13:00 +00:00
|
|
|
const MergeTreeData & data;
|
2021-07-09 03:15:41 +00:00
|
|
|
StorageSnapshotPtr storage_snapshot;
|
2021-03-30 10:25:26 +00:00
|
|
|
const Names column_names;
|
2017-04-01 07:20:54 +00:00
|
|
|
bool do_not_steal_tasks;
|
2017-04-06 17:21:45 +00:00
|
|
|
bool predict_block_size_bytes;
|
2022-06-02 17:58:25 +00:00
|
|
|
|
|
|
|
struct PerPartParams
|
|
|
|
{
|
|
|
|
MergeTreeReadTaskColumns task_columns;
|
|
|
|
NameSet column_name_set;
|
|
|
|
MergeTreeBlockSizePredictorPtr size_predictor;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::vector<PerPartParams> per_part_params;
|
|
|
|
|
2018-04-11 14:31:54 +00:00
|
|
|
PrewhereInfoPtr prewhere_info;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
struct Part
|
|
|
|
{
|
|
|
|
MergeTreeData::DataPartPtr data_part;
|
2017-07-21 06:35:58 +00:00
|
|
|
size_t part_index_in_query;
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
std::vector<Part> parts_with_idx;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
struct ThreadTask
|
|
|
|
{
|
|
|
|
struct PartIndexAndRange
|
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
size_t part_idx;
|
2017-04-01 07:20:54 +00:00
|
|
|
MarkRanges ranges;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::vector<PartIndexAndRange> parts_and_ranges;
|
2017-07-21 06:35:58 +00:00
|
|
|
std::vector<size_t> sum_marks_in_parts;
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
std::vector<ThreadTask> threads_tasks;
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
std::set<size_t> remaining_thread_tasks;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-17 14:56:15 +00:00
|
|
|
RangesInDataParts parts_ranges;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
mutable std::mutex mutex;
|
|
|
|
|
2020-05-30 21:57:37 +00:00
|
|
|
Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool");
|
2021-10-15 12:07:39 +00:00
|
|
|
|
2021-10-20 22:57:43 +00:00
|
|
|
std::vector<bool> is_part_on_remote_disk;
|
2015-06-24 11:03:53 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
using MergeTreeReadPoolPtr = std::shared_ptr<MergeTreeReadPool>;
|
|
|
|
|
|
|
|
}
|