2015-07-30 23:41:02 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Aggregator.h>
|
2019-01-23 14:48:50 +00:00
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/ConcurrentBoundedQueue.h>
|
2018-06-19 20:30:35 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2019-01-11 19:12:36 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2015-12-13 15:07:01 +00:00
|
|
|
#include <condition_variable>
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/** Pre-aggregates block streams, holding in RAM only one or more (up to merging_threads) blocks from each source.
|
|
|
|
* This saves RAM in case of using two-level aggregation, where in each source there will be up to 256 blocks with parts of the result.
|
2015-07-30 23:41:02 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* Aggregate functions in blocks should not be finalized so that their states can be combined.
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* Used to solve two tasks:
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* 1. External aggregation with data flush to disk.
|
|
|
|
* Partially aggregated data (previously divided into 256 buckets) is flushed to some number of files on the disk.
|
|
|
|
* We need to read them and merge them by buckets - keeping only a few buckets from each file in RAM simultaneously.
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* 2. Merge aggregation results for distributed query processing.
|
|
|
|
* Partially aggregated data arrives from different servers, which can be splitted down or not, into 256 buckets,
|
|
|
|
* and these buckets are passed to us by the network from each server in sequence, one by one.
|
|
|
|
* You should also read and merge by the buckets.
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* The essence of the work:
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* There are a number of sources. They give out blocks with partially aggregated data.
|
|
|
|
* Each source can return one of the following block sequences:
|
|
|
|
* 1. "unsplitted" block with bucket_num = -1;
|
|
|
|
* 2. "splitted" (two_level) blocks with bucket_num from 0 to 255;
|
|
|
|
* In both cases, there may also be a block of "overflows" with bucket_num = -1 and is_overflows = true;
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* We start from the convention that splitted blocks are always passed in the order of bucket_num.
|
|
|
|
* That is, if a < b, then the bucket_num = a block goes before bucket_num = b.
|
|
|
|
* This is needed for a memory-efficient merge
|
|
|
|
* - so that you do not need to read the blocks up front, but go all the way up by bucket_num.
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* In this case, not all bucket_num from the range of 0..255 can be present.
|
|
|
|
* The overflow block can be presented in any order relative to other blocks (but it can be only one).
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* It is necessary to combine these sequences of blocks and return the result as a sequence with the same properties.
|
|
|
|
* That is, at the output, if there are "splitted" blocks in the sequence, then they should go in the order of bucket_num.
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* The merge can be performed using several (merging_threads) threads.
|
|
|
|
* For this, receiving of a set of blocks for the next bucket_num should be done sequentially,
|
|
|
|
* and then, when we have several received sets, they can be merged in parallel.
|
2015-12-13 15:07:01 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* When you receive next blocks from different sources,
|
|
|
|
* data from sources can also be read in several threads (reading_threads)
|
|
|
|
* for optimal performance in the presence of a fast network or disks (from where these blocks are read).
|
2015-07-30 23:41:02 +00:00
|
|
|
*/
|
2019-01-23 14:48:50 +00:00
|
|
|
class MergingAggregatedMemoryEfficientBlockInputStream final : public IBlockInputStream
|
2015-07-30 23:41:02 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
MergingAggregatedMemoryEfficientBlockInputStream(
|
|
|
|
BlockInputStreams inputs_, const Aggregator::Params & params, bool final_,
|
|
|
|
size_t reading_threads_, size_t merging_threads_);
|
2015-07-30 23:41:02 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
~MergingAggregatedMemoryEfficientBlockInputStream() override;
|
2015-12-05 03:04:13 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return "MergingAggregatedMemoryEfficient"; }
|
2015-07-30 23:41:02 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Sends the request (initiates calculations) earlier than `read`.
|
2017-04-01 07:20:54 +00:00
|
|
|
void readPrefix() override;
|
2015-12-05 04:20:37 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Called either after everything is read, or after cancel.
|
2017-04-01 07:20:54 +00:00
|
|
|
void readSuffix() override;
|
2015-12-13 15:07:01 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/** Different from the default implementation by trying to stop all sources,
|
|
|
|
* skipping failed by execution.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-03-05 21:09:39 +00:00
|
|
|
void cancel(bool kill) override;
|
2015-12-13 15:07:01 +00:00
|
|
|
|
2018-02-18 03:23:48 +00:00
|
|
|
Block getHeader() const override;
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2015-07-30 23:41:02 +00:00
|
|
|
protected:
|
2017-04-01 07:20:54 +00:00
|
|
|
Block readImpl() override;
|
2015-07-30 23:41:02 +00:00
|
|
|
|
|
|
|
private:
|
2018-02-21 07:34:40 +00:00
|
|
|
static constexpr int NUM_BUCKETS = 256;
|
2015-12-13 15:07:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Aggregator aggregator;
|
|
|
|
bool final;
|
|
|
|
size_t reading_threads;
|
|
|
|
size_t merging_threads;
|
2015-07-30 23:41:02 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
bool started = false;
|
|
|
|
bool all_read = false;
|
|
|
|
std::atomic<bool> has_two_level {false};
|
|
|
|
std::atomic<bool> has_overflows {false};
|
|
|
|
int current_bucket_num = -1;
|
2015-09-07 07:40:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Logger * log = &Logger::get("MergingAggregatedMemoryEfficientBlockInputStream");
|
2015-12-13 15:07:01 +00:00
|
|
|
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
struct Input
|
|
|
|
{
|
|
|
|
BlockInputStreamPtr stream;
|
|
|
|
Block block;
|
|
|
|
Block overflow_block;
|
|
|
|
std::vector<Block> splitted_blocks;
|
|
|
|
bool is_exhausted = false;
|
2015-09-07 07:40:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Input(BlockInputStreamPtr & stream_) : stream(stream_) {}
|
|
|
|
};
|
2015-07-30 23:41:02 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::vector<Input> inputs;
|
2015-12-01 22:35:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using BlocksToMerge = std::unique_ptr<BlocksList>;
|
2015-12-01 22:35:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void start();
|
2015-12-05 04:20:37 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Get blocks that you can merge. This allows you to merge them in parallel in separate threads.
|
2017-04-01 07:20:54 +00:00
|
|
|
BlocksToMerge getNextBlocksToMerge();
|
2015-12-01 22:35:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::unique_ptr<ThreadPool> reading_pool;
|
2015-12-05 04:20:37 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// For a parallel merge.
|
2015-12-01 22:35:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
struct ParallelMergeData
|
|
|
|
{
|
|
|
|
ThreadPool pool;
|
2016-09-19 05:01:32 +00:00
|
|
|
|
2017-05-13 22:19:04 +00:00
|
|
|
/// Now one of the merging threads receives next blocks for the merge. This operation must be done sequentially.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::mutex get_next_blocks_mutex;
|
2016-09-19 05:01:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::atomic<bool> exhausted {false}; /// No more source data.
|
|
|
|
std::atomic<bool> finish {false}; /// Need to terminate early.
|
2015-12-01 22:35:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::exception_ptr exception;
|
2017-05-13 22:19:04 +00:00
|
|
|
/// It is necessary to give out blocks in the order of the key (bucket_num).
|
|
|
|
/// If the value is an empty block, you need to wait for its merge.
|
2017-09-08 03:47:27 +00:00
|
|
|
/// (This means the promise that there will be data here, which is important because the data should be given out
|
2017-05-13 22:19:04 +00:00
|
|
|
/// in the order of the key - bucket_num)
|
2017-04-01 07:20:54 +00:00
|
|
|
std::map<int, Block> merged_blocks;
|
|
|
|
std::mutex merged_blocks_mutex;
|
2017-05-13 22:19:04 +00:00
|
|
|
/// An event that is used by merging threads to tell the main thread that the new block is ready.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::condition_variable merged_blocks_changed;
|
2017-05-13 22:19:04 +00:00
|
|
|
/// An event by which the main thread is telling merging threads that it is possible to process the next group of blocks.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::condition_variable have_space;
|
2015-12-13 15:07:01 +00:00
|
|
|
|
2017-09-07 21:04:48 +00:00
|
|
|
explicit ParallelMergeData(size_t max_threads) : pool(max_threads) {}
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
2015-12-01 22:35:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::unique_ptr<ParallelMergeData> parallel_merge_data;
|
2015-12-01 22:35:48 +00:00
|
|
|
|
2018-06-19 20:30:35 +00:00
|
|
|
void mergeThread(ThreadGroupStatusPtr main_thread);
|
2015-12-13 15:07:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void finalize();
|
2015-07-30 23:41:02 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|