| 1 | #pragma once |
| 2 | |
| 3 | #include <Interpreters/Aggregator.h> |
| 4 | #include <DataStreams/IBlockInputStream.h> |
| 5 | #include <Common/ConcurrentBoundedQueue.h> |
| 6 | #include <Common/CurrentThread.h> |
| 7 | #include <Common/ThreadPool.h> |
| 8 | #include <condition_variable> |
| 9 | |
| 10 | |
| 11 | namespace DB |
| 12 | { |
| 13 | |
| 14 | |
| 15 | /** Pre-aggregates block streams, holding in RAM only one or more (up to merging_threads) blocks from each source. |
| 16 | * 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. |
| 17 | * |
| 18 | * Aggregate functions in blocks should not be finalized so that their states can be combined. |
| 19 | * |
| 20 | * Used to solve two tasks: |
| 21 | * |
| 22 | * 1. External aggregation with data flush to disk. |
| 23 | * Partially aggregated data (previously divided into 256 buckets) is flushed to some number of files on the disk. |
| 24 | * We need to read them and merge them by buckets - keeping only a few buckets from each file in RAM simultaneously. |
| 25 | * |
| 26 | * 2. Merge aggregation results for distributed query processing. |
| 27 | * Partially aggregated data arrives from different servers, which can be splitted down or not, into 256 buckets, |
| 28 | * and these buckets are passed to us by the network from each server in sequence, one by one. |
| 29 | * You should also read and merge by the buckets. |
| 30 | * |
| 31 | * The essence of the work: |
| 32 | * |
| 33 | * There are a number of sources. They give out blocks with partially aggregated data. |
| 34 | * Each source can return one of the following block sequences: |
| 35 | * 1. "unsplitted" block with bucket_num = -1; |
| 36 | * 2. "splitted" (two_level) blocks with bucket_num from 0 to 255; |
| 37 | * In both cases, there may also be a block of "overflows" with bucket_num = -1 and is_overflows = true; |
| 38 | * |
| 39 | * We start from the convention that splitted blocks are always passed in the order of bucket_num. |
| 40 | * That is, if a < b, then the bucket_num = a block goes before bucket_num = b. |
| 41 | * This is needed for a memory-efficient merge |
| 42 | * - so that you do not need to read the blocks up front, but go all the way up by bucket_num. |
| 43 | * |
| 44 | * In this case, not all bucket_num from the range of 0..255 can be present. |
| 45 | * The overflow block can be presented in any order relative to other blocks (but it can be only one). |
| 46 | * |
| 47 | * It is necessary to combine these sequences of blocks and return the result as a sequence with the same properties. |
| 48 | * That is, at the output, if there are "splitted" blocks in the sequence, then they should go in the order of bucket_num. |
| 49 | * |
| 50 | * The merge can be performed using several (merging_threads) threads. |
| 51 | * For this, receiving of a set of blocks for the next bucket_num should be done sequentially, |
| 52 | * and then, when we have several received sets, they can be merged in parallel. |
| 53 | * |
| 54 | * When you receive next blocks from different sources, |
| 55 | * data from sources can also be read in several threads (reading_threads) |
| 56 | * for optimal performance in the presence of a fast network or disks (from where these blocks are read). |
| 57 | */ |
| 58 | class MergingAggregatedMemoryEfficientBlockInputStream final : public IBlockInputStream |
| 59 | { |
| 60 | public: |
| 61 | MergingAggregatedMemoryEfficientBlockInputStream( |
| 62 | BlockInputStreams inputs_, const Aggregator::Params & params, bool final_, |
| 63 | size_t reading_threads_, size_t merging_threads_); |
| 64 | |
| 65 | ~MergingAggregatedMemoryEfficientBlockInputStream() override; |
| 66 | |
| 67 | String getName() const override { return "MergingAggregatedMemoryEfficient" ; } |
| 68 | |
| 69 | /// Sends the request (initiates calculations) earlier than `read`. |
| 70 | void readPrefix() override; |
| 71 | |
| 72 | /// Called either after everything is read, or after cancel. |
| 73 | void readSuffix() override; |
| 74 | |
| 75 | /** Different from the default implementation by trying to stop all sources, |
| 76 | * skipping failed by execution. |
| 77 | */ |
| 78 | void cancel(bool kill) override; |
| 79 | |
| 80 | Block () const override; |
| 81 | |
| 82 | protected: |
| 83 | Block readImpl() override; |
| 84 | |
| 85 | private: |
| 86 | static constexpr int NUM_BUCKETS = 256; |
| 87 | |
| 88 | Aggregator aggregator; |
| 89 | bool final; |
| 90 | size_t reading_threads; |
| 91 | size_t merging_threads; |
| 92 | |
| 93 | bool started = false; |
| 94 | bool all_read = false; |
| 95 | std::atomic<bool> has_two_level {false}; |
| 96 | std::atomic<bool> has_overflows {false}; |
| 97 | int current_bucket_num = -1; |
| 98 | |
| 99 | Logger * log = &Logger::get("MergingAggregatedMemoryEfficientBlockInputStream" ); |
| 100 | |
| 101 | |
| 102 | struct Input |
| 103 | { |
| 104 | BlockInputStreamPtr stream; |
| 105 | Block block; |
| 106 | Block overflow_block; |
| 107 | std::vector<Block> splitted_blocks; |
| 108 | bool is_exhausted = false; |
| 109 | |
| 110 | Input(BlockInputStreamPtr & stream_) : stream(stream_) {} |
| 111 | }; |
| 112 | |
| 113 | std::vector<Input> inputs; |
| 114 | |
| 115 | using BlocksToMerge = std::unique_ptr<BlocksList>; |
| 116 | |
| 117 | void start(); |
| 118 | |
| 119 | /// Get blocks that you can merge. This allows you to merge them in parallel in separate threads. |
| 120 | BlocksToMerge getNextBlocksToMerge(); |
| 121 | |
| 122 | std::unique_ptr<ThreadPool> reading_pool; |
| 123 | |
| 124 | /// For a parallel merge. |
| 125 | |
| 126 | struct ParallelMergeData |
| 127 | { |
| 128 | ThreadPool pool; |
| 129 | |
| 130 | /// Now one of the merging threads receives next blocks for the merge. This operation must be done sequentially. |
| 131 | std::mutex get_next_blocks_mutex; |
| 132 | |
| 133 | std::atomic<bool> exhausted {false}; /// No more source data. |
| 134 | std::atomic<bool> finish {false}; /// Need to terminate early. |
| 135 | |
| 136 | std::exception_ptr exception; |
| 137 | /// It is necessary to give out blocks in the order of the key (bucket_num). |
| 138 | /// If the value is an empty block, you need to wait for its merge. |
| 139 | /// (This means the promise that there will be data here, which is important because the data should be given out |
| 140 | /// in the order of the key - bucket_num) |
| 141 | std::map<int, Block> merged_blocks; |
| 142 | std::mutex merged_blocks_mutex; |
| 143 | /// An event that is used by merging threads to tell the main thread that the new block is ready. |
| 144 | std::condition_variable merged_blocks_changed; |
| 145 | /// An event by which the main thread is telling merging threads that it is possible to process the next group of blocks. |
| 146 | std::condition_variable have_space; |
| 147 | |
| 148 | explicit ParallelMergeData(size_t max_threads) : pool(max_threads) {} |
| 149 | }; |
| 150 | |
| 151 | std::unique_ptr<ParallelMergeData> parallel_merge_data; |
| 152 | |
| 153 | void mergeThread(ThreadGroupStatusPtr main_thread); |
| 154 | |
| 155 | void finalize(); |
| 156 | }; |
| 157 | |
| 158 | } |
| 159 | |