/root/doris/be/src/pipeline/dependency.h
Line | Count | Source |
1 | | // Licensed to the Apache Software Foundation (ASF) under one |
2 | | // or more contributor license agreements. See the NOTICE file |
3 | | // distributed with this work for additional information |
4 | | // regarding copyright ownership. The ASF licenses this file |
5 | | // to you under the Apache License, Version 2.0 (the |
6 | | // "License"); you may not use this file except in compliance |
7 | | // with the License. You may obtain a copy of the License at |
8 | | // |
9 | | // http://www.apache.org/licenses/LICENSE-2.0 |
10 | | // |
11 | | // Unless required by applicable law or agreed to in writing, |
12 | | // software distributed under the License is distributed on an |
13 | | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
14 | | // KIND, either express or implied. See the License for the |
15 | | // specific language governing permissions and limitations |
16 | | // under the License. |
17 | | |
18 | | #pragma once |
19 | | |
20 | | #include <concurrentqueue.h> |
21 | | #include <sqltypes.h> |
22 | | |
23 | | #include <atomic> |
24 | | #include <functional> |
25 | | #include <memory> |
26 | | #include <mutex> |
27 | | #include <thread> |
28 | | #include <utility> |
29 | | |
30 | | #include "common/config.h" |
31 | | #include "common/logging.h" |
32 | | #include "gen_cpp/internal_service.pb.h" |
33 | | #include "pipeline/common/agg_utils.h" |
34 | | #include "pipeline/common/join_utils.h" |
35 | | #include "pipeline/common/set_utils.h" |
36 | | #include "pipeline/exec/data_queue.h" |
37 | | #include "pipeline/exec/join/process_hash_table_probe.h" |
38 | | #include "util/brpc_closure.h" |
39 | | #include "util/stack_util.h" |
40 | | #include "vec/common/sort/partition_sorter.h" |
41 | | #include "vec/common/sort/sorter.h" |
42 | | #include "vec/core/block.h" |
43 | | #include "vec/core/types.h" |
44 | | #include "vec/spill/spill_stream.h" |
45 | | |
46 | | namespace doris::vectorized { |
47 | | class AggFnEvaluator; |
48 | | class VSlotRef; |
49 | | } // namespace doris::vectorized |
50 | | |
51 | | namespace doris::pipeline { |
52 | | #include "common/compile_check_begin.h" |
53 | | class Dependency; |
54 | | class PipelineTask; |
55 | | struct BasicSharedState; |
56 | | using DependencySPtr = std::shared_ptr<Dependency>; |
57 | | class LocalExchangeSourceLocalState; |
58 | | |
59 | | static constexpr auto SLOW_DEPENDENCY_THRESHOLD = 60 * 1000L * 1000L * 1000L; |
60 | | static constexpr auto TIME_UNIT_DEPENDENCY_LOG = 30 * 1000L * 1000L * 1000L; |
61 | | static_assert(TIME_UNIT_DEPENDENCY_LOG < SLOW_DEPENDENCY_THRESHOLD); |
62 | | |
63 | | struct BasicSharedState { |
64 | | ENABLE_FACTORY_CREATOR(BasicSharedState) |
65 | | |
66 | | template <class TARGET> |
67 | 5.69M | TARGET* cast() { |
68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) |
69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() |
70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); |
71 | 5.69M | return reinterpret_cast<TARGET*>(this); |
72 | 5.69M | } _ZN5doris8pipeline16BasicSharedState4castINS0_26MaterializationSharedStateEEEPT_v Line | Count | Source | 67 | 3.02k | TARGET* cast() { | 68 | 3.02k | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 0 | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 0 | << " and expect type is" << typeid(TARGET).name(); | 71 | 3.02k | return reinterpret_cast<TARGET*>(this); | 72 | 3.02k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_19HashJoinSharedStateEEEPT_v Line | Count | Source | 67 | 317k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 317k | return reinterpret_cast<TARGET*>(this); | 72 | 317k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_30PartitionedHashJoinSharedStateEEEPT_v Line | Count | Source | 67 | 60.7k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 60.7k | return reinterpret_cast<TARGET*>(this); | 72 | 60.7k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_15SortSharedStateEEEPT_v Line | Count | Source | 67 | 1.12M | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 1.12M | return reinterpret_cast<TARGET*>(this); | 72 | 1.12M | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_20SpillSortSharedStateEEEPT_v Line | Count | Source | 67 | 947k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 947k | return reinterpret_cast<TARGET*>(this); | 72 | 947k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_25NestedLoopJoinSharedStateEEEPT_v Line | Count | Source | 67 | 33.0k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 33.0k | return reinterpret_cast<TARGET*>(this); | 72 | 33.0k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_19AnalyticSharedStateEEEPT_v Line | Count | Source | 67 | 56.0k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 56.0k | return reinterpret_cast<TARGET*>(this); | 72 | 56.0k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_14AggSharedStateEEEPT_v Line | Count | Source | 67 | 434k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 434k | return reinterpret_cast<TARGET*>(this); | 72 | 434k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_25PartitionedAggSharedStateEEEPT_v Line | Count | Source | 67 | 132k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 132k | return reinterpret_cast<TARGET*>(this); | 72 | 132k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_16UnionSharedStateEEEPT_v Line | Count | Source | 67 | 18.7k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 18.7k | return reinterpret_cast<TARGET*>(this); | 72 | 18.7k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_28PartitionSortNodeSharedStateEEEPT_v Line | Count | Source | 67 | 1.90k | TARGET* cast() { | 68 | 1.90k | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 8 | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 8 | << " and expect type is" << typeid(TARGET).name(); | 71 | 1.90k | return reinterpret_cast<TARGET*>(this); | 72 | 1.90k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_20MultiCastSharedStateEEEPT_v Line | Count | Source | 67 | 15.9k | TARGET* cast() { | 68 | 15.9k | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 2 | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 2 | << " and expect type is" << typeid(TARGET).name(); | 71 | 15.9k | return reinterpret_cast<TARGET*>(this); | 72 | 15.9k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_14SetSharedStateEEEPT_v Line | Count | Source | 67 | 6.39k | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 6.39k | return reinterpret_cast<TARGET*>(this); | 72 | 6.39k | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_24LocalExchangeSharedStateEEEPT_v Line | Count | Source | 67 | 1.44M | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 1.44M | return reinterpret_cast<TARGET*>(this); | 72 | 1.44M | } |
_ZN5doris8pipeline16BasicSharedState4castIS1_EEPT_v Line | Count | Source | 67 | 1.09M | TARGET* cast() { | 68 | 18.4E | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 18.4E | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 18.4E | << " and expect type is" << typeid(TARGET).name(); | 71 | 1.09M | return reinterpret_cast<TARGET*>(this); | 72 | 1.09M | } |
_ZN5doris8pipeline16BasicSharedState4castINS0_20DataQueueSharedStateEEEPT_v Line | Count | Source | 67 | 356 | TARGET* cast() { | 68 | 356 | DCHECK(dynamic_cast<TARGET*>(this)) | 69 | 0 | << " Mismatch type! Current type is " << typeid(*this).name() | 70 | 0 | << " and expect type is" << typeid(TARGET).name(); | 71 | 356 | return reinterpret_cast<TARGET*>(this); | 72 | 356 | } |
|
73 | | template <class TARGET> |
74 | | const TARGET* cast() const { |
75 | | DCHECK(dynamic_cast<const TARGET*>(this)) |
76 | | << " Mismatch type! Current type is " << typeid(*this).name() |
77 | | << " and expect type is" << typeid(TARGET).name(); |
78 | | return reinterpret_cast<const TARGET*>(this); |
79 | | } |
80 | | std::vector<DependencySPtr> source_deps; |
81 | | std::vector<DependencySPtr> sink_deps; |
82 | | int id = 0; |
83 | | std::set<int> related_op_ids; |
84 | | |
85 | 3.74M | virtual ~BasicSharedState() = default; |
86 | | |
87 | | void create_source_dependencies(int num_sources, int operator_id, int node_id, |
88 | | const std::string& name); |
89 | | Dependency* create_source_dependency(int operator_id, int node_id, const std::string& name); |
90 | | |
91 | | Dependency* create_sink_dependency(int dest_id, int node_id, const std::string& name); |
92 | 1.03M | std::vector<DependencySPtr> get_dep_by_channel_id(int channel_id) { |
93 | 1.03M | DCHECK_LT(channel_id, source_deps.size()); |
94 | 1.03M | return {source_deps[channel_id]}; |
95 | 1.03M | } |
96 | | }; |
97 | | |
98 | | class Dependency : public std::enable_shared_from_this<Dependency> { |
99 | | public: |
100 | | ENABLE_FACTORY_CREATOR(Dependency); |
101 | | Dependency(int id, int node_id, std::string name, bool ready = false) |
102 | 14.5M | : _id(id), _node_id(node_id), _name(std::move(name)), _ready(ready) {} |
103 | 14.5M | virtual ~Dependency() = default; |
104 | | |
105 | | [[nodiscard]] int id() const { return _id; } |
106 | 17.3M | [[nodiscard]] virtual std::string name() const { return _name; } |
107 | 485k | BasicSharedState* shared_state() { return _shared_state; } |
108 | 5.45M | void set_shared_state(BasicSharedState* shared_state) { _shared_state = shared_state; } |
109 | | virtual std::string debug_string(int indentation_level = 0); |
110 | 664M | bool ready() const { return _ready; } |
111 | | |
112 | | // Start the watcher. We use it to count how long this dependency block the current pipeline task. |
113 | 10.3M | void start_watcher() { _watcher.start(); } |
114 | 11.2M | [[nodiscard]] int64_t watcher_elapse_time() { return _watcher.elapsed_time(); } |
115 | | |
116 | | // Which dependency current pipeline task is blocked by. `nullptr` if this dependency is ready. |
117 | | [[nodiscard]] Dependency* is_blocked_by(std::shared_ptr<PipelineTask> task = nullptr); |
118 | | // Notify downstream pipeline tasks this dependency is ready. |
119 | | void set_ready(); |
120 | 1.03M | void set_ready_to_read(int channel_id = 0) { |
121 | 1.03M | DCHECK_LT(channel_id, _shared_state->source_deps.size()) << debug_string(); |
122 | 1.03M | _shared_state->source_deps[channel_id]->set_ready(); |
123 | 1.03M | } |
124 | 1.57k | void set_ready_to_write() { |
125 | 1.57k | DCHECK_EQ(_shared_state->sink_deps.size(), 1) << debug_string(); |
126 | 1.57k | _shared_state->sink_deps.front()->set_ready(); |
127 | 1.57k | } |
128 | | |
129 | | // Notify downstream pipeline tasks this dependency is blocked. |
130 | 5.28M | void block() { |
131 | 5.28M | if (_always_ready) { |
132 | 520k | return; |
133 | 520k | } |
134 | 4.76M | std::unique_lock<std::mutex> lc(_always_ready_lock); |
135 | 4.76M | if (_always_ready) { |
136 | 6 | return; |
137 | 6 | } |
138 | 4.76M | _ready = false; |
139 | 4.76M | } |
140 | | |
141 | 7.82M | void set_always_ready() { |
142 | 7.82M | if (_always_ready) { |
143 | 4.10M | return; |
144 | 4.10M | } |
145 | 3.71M | std::unique_lock<std::mutex> lc(_always_ready_lock); |
146 | 3.71M | if (_always_ready) { |
147 | 0 | return; |
148 | 0 | } |
149 | 3.71M | _always_ready = true; |
150 | 3.71M | set_ready(); |
151 | 3.71M | } |
152 | | |
153 | | protected: |
154 | | void _add_block_task(std::shared_ptr<PipelineTask> task); |
155 | | |
156 | | const int _id; |
157 | | const int _node_id; |
158 | | const std::string _name; |
159 | | std::atomic<bool> _ready; |
160 | | |
161 | | BasicSharedState* _shared_state = nullptr; |
162 | | MonotonicStopWatch _watcher; |
163 | | |
164 | | std::mutex _task_lock; |
165 | | std::vector<std::weak_ptr<PipelineTask>> _blocked_task; |
166 | | |
167 | | // If `_always_ready` is true, `block()` will never block tasks. |
168 | | std::atomic<bool> _always_ready = false; |
169 | | std::mutex _always_ready_lock; |
170 | | }; |
171 | | |
172 | | struct FakeSharedState final : public BasicSharedState { |
173 | | ENABLE_FACTORY_CREATOR(FakeSharedState) |
174 | | }; |
175 | | |
176 | | class CountedFinishDependency final : public Dependency { |
177 | | public: |
178 | | using SharedState = FakeSharedState; |
179 | | CountedFinishDependency(int id, int node_id, std::string name) |
180 | 188k | : Dependency(id, node_id, std::move(name), true) {} |
181 | | |
182 | 11.4k | void add(uint32_t count = 1) { |
183 | 11.4k | std::unique_lock<std::mutex> l(_mtx); |
184 | 11.4k | if (!_counter) { |
185 | 11.4k | block(); |
186 | 11.4k | } |
187 | 11.4k | _counter += count; |
188 | 11.4k | } |
189 | | |
190 | 9.89k | void sub() { |
191 | 9.89k | std::unique_lock<std::mutex> l(_mtx); |
192 | 9.89k | _counter--; |
193 | 9.89k | if (!_counter) { |
194 | 9.89k | set_ready(); |
195 | 9.89k | } |
196 | 9.89k | } |
197 | | |
198 | | std::string debug_string(int indentation_level = 0) override; |
199 | | |
200 | | private: |
201 | | std::mutex _mtx; |
202 | | uint32_t _counter = 0; |
203 | | }; |
204 | | |
205 | | struct RuntimeFilterTimerQueue; |
206 | | class RuntimeFilterTimer { |
207 | | public: |
208 | | RuntimeFilterTimer(int64_t registration_time, int32_t wait_time_ms, |
209 | | std::shared_ptr<Dependency> parent, bool force_wait_timeout = false) |
210 | 56.7k | : _parent(std::move(parent)), |
211 | 56.7k | _registration_time(registration_time), |
212 | 56.7k | _wait_time_ms(wait_time_ms), |
213 | 56.7k | _force_wait_timeout(force_wait_timeout) {} |
214 | | |
215 | | // Called by runtime filter producer. |
216 | | void call_ready(); |
217 | | |
218 | | // Called by RuntimeFilterTimerQueue which is responsible for checking if this rf is timeout. |
219 | | void call_timeout(); |
220 | | |
221 | 1.06M | int64_t registration_time() const { return _registration_time; } |
222 | 1.06M | int32_t wait_time_ms() const { return _wait_time_ms; } |
223 | | |
224 | | void set_local_runtime_filter_dependencies( |
225 | 23.4k | const std::vector<std::shared_ptr<Dependency>>& deps) { |
226 | 23.4k | _local_runtime_filter_dependencies = deps; |
227 | 23.4k | } |
228 | | |
229 | | bool should_be_check_timeout(); |
230 | | |
231 | 1.10M | bool force_wait_timeout() { return _force_wait_timeout; } |
232 | | |
233 | | private: |
234 | | friend struct RuntimeFilterTimerQueue; |
235 | | std::shared_ptr<Dependency> _parent = nullptr; |
236 | | std::vector<std::shared_ptr<Dependency>> _local_runtime_filter_dependencies; |
237 | | std::mutex _lock; |
238 | | int64_t _registration_time; |
239 | | const int32_t _wait_time_ms; |
240 | | // true only for group_commit_scan_operator |
241 | | bool _force_wait_timeout; |
242 | | }; |
243 | | |
244 | | struct RuntimeFilterTimerQueue { |
245 | | constexpr static int64_t interval = 10; |
246 | 9 | void run() { _thread.detach(); } |
247 | | void start(); |
248 | | |
249 | 4 | void stop() { |
250 | 4 | _stop = true; |
251 | 4 | cv.notify_all(); |
252 | 4 | wait_for_shutdown(); |
253 | 4 | } |
254 | | |
255 | 4 | void wait_for_shutdown() const { |
256 | 8 | while (!_shutdown) { |
257 | 4 | std::this_thread::sleep_for(std::chrono::milliseconds(interval)); |
258 | 4 | } |
259 | 4 | } |
260 | | |
261 | 4 | ~RuntimeFilterTimerQueue() = default; |
262 | 9 | RuntimeFilterTimerQueue() { _thread = std::thread(&RuntimeFilterTimerQueue::start, this); } |
263 | 35.6k | void push_filter_timer(std::vector<std::shared_ptr<pipeline::RuntimeFilterTimer>>&& filter) { |
264 | 35.6k | std::unique_lock<std::mutex> lc(_que_lock); |
265 | 35.6k | _que.insert(_que.end(), filter.begin(), filter.end()); |
266 | 35.6k | cv.notify_all(); |
267 | 35.6k | } |
268 | | |
269 | | std::thread _thread; |
270 | | std::condition_variable cv; |
271 | | std::mutex cv_m; |
272 | | std::mutex _que_lock; |
273 | | std::atomic_bool _stop = false; |
274 | | std::atomic_bool _shutdown = false; |
275 | | std::list<std::shared_ptr<pipeline::RuntimeFilterTimer>> _que; |
276 | | }; |
277 | | |
278 | | struct AggSharedState : public BasicSharedState { |
279 | | ENABLE_FACTORY_CREATOR(AggSharedState) |
280 | | public: |
281 | 218k | AggSharedState() { agg_data = std::make_unique<AggregatedDataVariants>(); } |
282 | 218k | ~AggSharedState() override { |
283 | 218k | if (!probe_expr_ctxs.empty()) { |
284 | 78.0k | _close_with_serialized_key(); |
285 | 140k | } else { |
286 | 140k | _close_without_key(); |
287 | 140k | } |
288 | 218k | } |
289 | | |
290 | | Status reset_hash_table(); |
291 | | |
292 | | bool do_limit_filter(vectorized::Block* block, size_t num_rows, |
293 | | const std::vector<int>* key_locs = nullptr); |
294 | | void build_limit_heap(size_t hash_table_size); |
295 | | |
296 | | // We should call this function only at 1st phase. |
297 | | // 1st phase: is_merge=true, only have one SlotRef. |
298 | | // 2nd phase: is_merge=false, maybe have multiple exprs. |
299 | | static int get_slot_column_id(const vectorized::AggFnEvaluator* evaluator); |
300 | | |
301 | | AggregatedDataVariantsUPtr agg_data = nullptr; |
302 | | std::unique_ptr<AggregateDataContainer> aggregate_data_container; |
303 | | std::vector<vectorized::AggFnEvaluator*> aggregate_evaluators; |
304 | | // group by k1,k2 |
305 | | vectorized::VExprContextSPtrs probe_expr_ctxs; |
306 | | size_t input_num_rows = 0; |
307 | | std::vector<vectorized::AggregateDataPtr> values; |
308 | | /// The total size of the row from the aggregate functions. |
309 | | size_t total_size_of_aggregate_states = 0; |
310 | | size_t align_aggregate_states = 1; |
311 | | /// The offset to the n-th aggregate function in a row of aggregate functions. |
312 | | vectorized::Sizes offsets_of_aggregate_states; |
313 | | std::vector<size_t> make_nullable_keys; |
314 | | |
315 | | bool agg_data_created_without_key = false; |
316 | | bool enable_spill = false; |
317 | | bool reach_limit = false; |
318 | | |
319 | | int64_t limit = -1; |
320 | | bool do_sort_limit = false; |
321 | | vectorized::MutableColumns limit_columns; |
322 | | int limit_columns_min = -1; |
323 | | vectorized::PaddedPODArray<uint8_t> need_computes; |
324 | | std::vector<uint8_t> cmp_res; |
325 | | std::vector<int> order_directions; |
326 | | std::vector<int> null_directions; |
327 | | |
328 | | struct HeapLimitCursor { |
329 | | HeapLimitCursor(int row_id, vectorized::MutableColumns& limit_columns, |
330 | | std::vector<int>& order_directions, std::vector<int>& null_directions) |
331 | 56.1k | : _row_id(row_id), |
332 | 56.1k | _limit_columns(limit_columns), |
333 | 56.1k | _order_directions(order_directions), |
334 | 56.1k | _null_directions(null_directions) {} |
335 | | |
336 | | HeapLimitCursor(const HeapLimitCursor& other) = default; |
337 | | |
338 | | HeapLimitCursor(HeapLimitCursor&& other) noexcept |
339 | 332k | : _row_id(other._row_id), |
340 | 332k | _limit_columns(other._limit_columns), |
341 | 332k | _order_directions(other._order_directions), |
342 | 332k | _null_directions(other._null_directions) {} |
343 | | |
344 | 0 | HeapLimitCursor& operator=(const HeapLimitCursor& other) noexcept { |
345 | 0 | _row_id = other._row_id; |
346 | 0 | return *this; |
347 | 0 | } |
348 | | |
349 | 664k | HeapLimitCursor& operator=(HeapLimitCursor&& other) noexcept { |
350 | 664k | _row_id = other._row_id; |
351 | 664k | return *this; |
352 | 664k | } |
353 | | |
354 | 598k | bool operator<(const HeapLimitCursor& rhs) const { |
355 | 1.12M | for (int i = 0; i < _limit_columns.size(); ++i) { |
356 | 1.12M | const auto& _limit_column = _limit_columns[i]; |
357 | 1.12M | auto res = _limit_column->compare_at(_row_id, rhs._row_id, *_limit_column, |
358 | 1.12M | _null_directions[i]) * |
359 | 1.12M | _order_directions[i]; |
360 | 1.12M | if (res < 0) { |
361 | 307k | return true; |
362 | 813k | } else if (res > 0) { |
363 | 296k | return false; |
364 | 296k | } |
365 | 1.12M | } |
366 | 18.4E | return false; |
367 | 598k | } |
368 | | |
369 | | int _row_id; |
370 | | vectorized::MutableColumns& _limit_columns; |
371 | | std::vector<int>& _order_directions; |
372 | | std::vector<int>& _null_directions; |
373 | | }; |
374 | | |
375 | | std::priority_queue<HeapLimitCursor> limit_heap; |
376 | | |
377 | | // Refresh the top limit heap with a new row |
378 | | void refresh_top_limit(size_t row_id, const vectorized::ColumnRawPtrs& key_columns); |
379 | | |
380 | | private: |
381 | | vectorized::MutableColumns _get_keys_hash_table(); |
382 | | |
383 | 77.6k | void _close_with_serialized_key() { |
384 | 77.6k | std::visit(vectorized::Overload {[&](std::monostate& arg) -> void { |
385 | | // Do nothing |
386 | 0 | }, |
387 | 77.8k | [&](auto& agg_method) -> void { |
388 | 77.8k | auto& data = *agg_method.hash_table; |
389 | 18.9M | data.for_each_mapped([&](auto& mapped) { |
390 | 18.9M | if (mapped) { |
391 | 18.9M | static_cast<void>(_destroy_agg_status(mapped)); |
392 | 18.9M | mapped = nullptr; |
393 | 18.9M | } |
394 | 18.9M | }); _ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapINS6_7UInt136EPc9HashCRC32IS9_EEEEEEvS3_ENKUlS3_E_clISA_EEDaS3_ Line | Count | Source | 389 | 4.79k | data.for_each_mapped([&](auto& mapped) { | 390 | 4.80k | if (mapped) { | 391 | 4.80k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 4.80k | mapped = nullptr; | 393 | 4.80k | } | 394 | 4.79k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapIN4wide7integerILm256EjEEPc9HashCRC32ISB_EEEEEEvS3_ENKUlS3_E_clISC_EEDaS3_ Line | Count | Source | 389 | 6.80k | data.for_each_mapped([&](auto& mapped) { | 390 | 6.80k | if (mapped) { | 391 | 6.79k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 6.79k | mapped = nullptr; | 393 | 6.79k | } | 394 | 6.80k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapIN4wide7integerILm128EjEEPc9HashCRC32ISB_EEEEEEvS3_ENKUlS3_E_clISC_EEDaS3_ Line | Count | Source | 389 | 99.5k | data.for_each_mapped([&](auto& mapped) { | 390 | 99.5k | if (mapped) { | 391 | 99.5k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 99.5k | mapped = nullptr; | 393 | 99.5k | } | 394 | 99.5k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapImPc9HashCRC32ImEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 173k | data.for_each_mapped([&](auto& mapped) { | 390 | 174k | if (mapped) { | 391 | 174k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 174k | mapped = nullptr; | 393 | 174k | } | 394 | 173k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_19MethodStringNoCacheINS6_15DataWithNullKeyINS_13StringHashMapIPcNS_9AllocatorILb1ELb1ELb0ENS_22DefaultMemoryAllocatorELb1EEEEEEEEEEEEEvS3_ENKUlS3_E_clISB_EEDaS3_ Line | Count | Source | 389 | 6.23k | data.for_each_mapped([&](auto& mapped) { | 390 | 6.23k | if (mapped) { | 391 | 6.23k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 6.23k | mapped = nullptr; | 393 | 6.23k | } | 394 | 6.23k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIN4wide7integerILm256EjEENS6_15DataWithNullKeyI9PHHashMapISB_Pc9HashCRC32ISB_EEEEEEEEEEvS3_ENKUlS3_E_clISE_EEDaS3_ Line | Count | Source | 389 | 14 | data.for_each_mapped([&](auto& mapped) { | 390 | 14 | if (mapped) { | 391 | 14 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 14 | mapped = nullptr; | 393 | 14 | } | 394 | 14 | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIN4wide7integerILm128EjEENS6_15DataWithNullKeyI9PHHashMapISB_Pc9HashCRC32ISB_EEEEEEEEEEvS3_ENKUlS3_E_clISE_EEDaS3_ Line | Count | Source | 389 | 2.41M | data.for_each_mapped([&](auto& mapped) { | 390 | 2.41M | if (mapped) { | 391 | 2.41M | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 2.41M | mapped = nullptr; | 393 | 2.41M | } | 394 | 2.41M | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberImNS6_15DataWithNullKeyI9PHHashMapImPc14HashMixWrapperIm9HashCRC32ImEEEEEEEEEEEvS3_ENKUlS3_E_clISB_EEDaS3_ Line | Count | Source | 389 | 5.24M | data.for_each_mapped([&](auto& mapped) { | 390 | 5.24M | if (mapped) { | 391 | 5.24M | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 5.24M | mapped = nullptr; | 393 | 5.24M | } | 394 | 5.24M | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIjNS6_15DataWithNullKeyI9PHHashMapIjPc14HashMixWrapperIj9HashCRC32IjEEEEEEEEEEEvS3_ENKUlS3_E_clISB_EEDaS3_ Line | Count | Source | 389 | 805k | data.for_each_mapped([&](auto& mapped) { | 390 | 805k | if (mapped) { | 391 | 805k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 805k | mapped = nullptr; | 393 | 805k | } | 394 | 805k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberImNS6_15DataWithNullKeyI9PHHashMapImPc9HashCRC32ImEEEEEEEEEEvS3_ENKUlS3_E_clISB_EEDaS3_ Line | Count | Source | 389 | 3.84M | data.for_each_mapped([&](auto& mapped) { | 390 | 3.84M | if (mapped) { | 391 | 3.84M | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 3.84M | mapped = nullptr; | 393 | 3.84M | } | 394 | 3.84M | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIjNS6_15DataWithNullKeyI9PHHashMapIjPc9HashCRC32IjEEEEEEEEEEvS3_ENKUlS3_E_clISB_EEDaS3_ Line | Count | Source | 389 | 395k | data.for_each_mapped([&](auto& mapped) { | 390 | 395k | if (mapped) { | 391 | 395k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 395k | mapped = nullptr; | 393 | 395k | } | 394 | 395k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberItNS6_15DataWithNullKeyI9PHHashMapItPc9HashCRC32ItEEEEEEEEEEvS3_ENKUlS3_E_clISB_EEDaS3_ Line | Count | Source | 389 | 314k | data.for_each_mapped([&](auto& mapped) { | 390 | 314k | if (mapped) { | 391 | 314k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 314k | mapped = nullptr; | 393 | 314k | } | 394 | 314k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIhNS6_15DataWithNullKeyI9PHHashMapIhPc9HashCRC32IhEEEEEEEEEEvS3_ENKUlS3_E_clISB_EEDaS3_ Line | Count | Source | 389 | 4.60k | data.for_each_mapped([&](auto& mapped) { | 390 | 4.60k | if (mapped) { | 391 | 4.60k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 4.60k | mapped = nullptr; | 393 | 4.60k | } | 394 | 4.60k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIm9PHHashMapImPc14HashMixWrapperIm9HashCRC32ImEEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 2.22M | data.for_each_mapped([&](auto& mapped) { | 390 | 2.22M | if (mapped) { | 391 | 2.22M | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 2.22M | mapped = nullptr; | 393 | 2.22M | } | 394 | 2.22M | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIj9PHHashMapIjPc14HashMixWrapperIj9HashCRC32IjEEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 343k | data.for_each_mapped([&](auto& mapped) { | 390 | 343k | if (mapped) { | 391 | 343k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 343k | mapped = nullptr; | 393 | 343k | } | 394 | 343k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIN4wide7integerILm256EjEE9PHHashMapISA_Pc9HashCRC32ISA_EEEEEEvS3_ENKUlS3_E_clISC_EEDaS3_ Line | Count | Source | 389 | 2 | data.for_each_mapped([&](auto& mapped) { | 390 | 2 | if (mapped) { | 391 | 2 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 2 | mapped = nullptr; | 393 | 2 | } | 394 | 2 | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIN4wide7integerILm128EjEE9PHHashMapISA_Pc9HashCRC32ISA_EEEEEEvS3_ENKUlS3_E_clISC_EEDaS3_ Line | Count | Source | 389 | 168 | data.for_each_mapped([&](auto& mapped) { | 390 | 168 | if (mapped) { | 391 | 168 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 168 | mapped = nullptr; | 393 | 168 | } | 394 | 168 | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized19MethodStringNoCacheINS_13StringHashMapIPcNS_9AllocatorILb1ELb1ELb0ENS_22DefaultMemoryAllocatorELb1EEEEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 5.15k | data.for_each_mapped([&](auto& mapped) { | 390 | 5.15k | if (mapped) { | 391 | 5.15k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 5.15k | mapped = nullptr; | 393 | 5.15k | } | 394 | 5.15k | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIm9PHHashMapImPc9HashCRC32ImEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 1.44M | data.for_each_mapped([&](auto& mapped) { | 390 | 1.44M | if (mapped) { | 391 | 1.44M | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 1.44M | mapped = nullptr; | 393 | 1.44M | } | 394 | 1.44M | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIj9PHHashMapIjPc9HashCRC32IjEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 1.46M | data.for_each_mapped([&](auto& mapped) { | 390 | 1.46M | if (mapped) { | 391 | 1.46M | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 1.46M | mapped = nullptr; | 393 | 1.46M | } | 394 | 1.46M | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIt9PHHashMapItPc9HashCRC32ItEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 80 | data.for_each_mapped([&](auto& mapped) { | 390 | 80 | if (mapped) { | 391 | 80 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 80 | mapped = nullptr; | 393 | 80 | } | 394 | 80 | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIh9PHHashMapIhPc9HashCRC32IhEEEEEEvS3_ENKUlS3_E_clIS9_EEDaS3_ Line | Count | Source | 389 | 510 | data.for_each_mapped([&](auto& mapped) { | 390 | 510 | if (mapped) { | 391 | 510 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 510 | mapped = nullptr; | 393 | 510 | } | 394 | 510 | }); |
_ZZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized16MethodSerializedI9PHHashMapINS_9StringRefEPc11DefaultHashIS9_vEEEEEEvS3_ENKUlS3_E_clISA_EEDaS3_ Line | Count | Source | 389 | 169k | data.for_each_mapped([&](auto& mapped) { | 390 | 169k | if (mapped) { | 391 | 169k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 169k | mapped = nullptr; | 393 | 169k | } | 394 | 169k | }); |
|
395 | 77.8k | if (data.has_null_key_data()) { |
396 | 3.53k | auto st = _destroy_agg_status( |
397 | 3.53k | data.template get_null_key_data< |
398 | 3.53k | vectorized::AggregateDataPtr>()); |
399 | 3.53k | if (!st) { |
400 | 0 | throw Exception(st.code(), st.to_string()); |
401 | 0 | } |
402 | 3.53k | } |
403 | 77.8k | }}, _ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapINS6_7UInt136EPc9HashCRC32IS9_EEEEEEvS3_ Line | Count | Source | 387 | 5.61k | [&](auto& agg_method) -> void { | 388 | 5.61k | auto& data = *agg_method.hash_table; | 389 | 5.61k | data.for_each_mapped([&](auto& mapped) { | 390 | 5.61k | if (mapped) { | 391 | 5.61k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 5.61k | mapped = nullptr; | 393 | 5.61k | } | 394 | 5.61k | }); | 395 | 5.61k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 5.61k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapIN4wide7integerILm256EjEEPc9HashCRC32ISB_EEEEEEvS3_ Line | Count | Source | 387 | 2.12k | [&](auto& agg_method) -> void { | 388 | 2.12k | auto& data = *agg_method.hash_table; | 389 | 2.12k | data.for_each_mapped([&](auto& mapped) { | 390 | 2.12k | if (mapped) { | 391 | 2.12k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 2.12k | mapped = nullptr; | 393 | 2.12k | } | 394 | 2.12k | }); | 395 | 2.12k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 2.12k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapIN4wide7integerILm128EjEEPc9HashCRC32ISB_EEEEEEvS3_ Line | Count | Source | 387 | 8.08k | [&](auto& agg_method) -> void { | 388 | 8.08k | auto& data = *agg_method.hash_table; | 389 | 8.08k | data.for_each_mapped([&](auto& mapped) { | 390 | 8.08k | if (mapped) { | 391 | 8.08k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 8.08k | mapped = nullptr; | 393 | 8.08k | } | 394 | 8.08k | }); | 395 | 8.08k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 8.08k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodKeysFixedI9PHHashMapImPc9HashCRC32ImEEEEEEvS3_ Line | Count | Source | 387 | 1.67k | [&](auto& agg_method) -> void { | 388 | 1.67k | auto& data = *agg_method.hash_table; | 389 | 1.67k | data.for_each_mapped([&](auto& mapped) { | 390 | 1.67k | if (mapped) { | 391 | 1.67k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 1.67k | mapped = nullptr; | 393 | 1.67k | } | 394 | 1.67k | }); | 395 | 1.67k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 1.67k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_19MethodStringNoCacheINS6_15DataWithNullKeyINS_13StringHashMapIPcNS_9AllocatorILb1ELb1ELb0ENS_22DefaultMemoryAllocatorELb1EEEEEEEEEEEEEvS3_ Line | Count | Source | 387 | 3.66k | [&](auto& agg_method) -> void { | 388 | 3.66k | auto& data = *agg_method.hash_table; | 389 | 3.66k | data.for_each_mapped([&](auto& mapped) { | 390 | 3.66k | if (mapped) { | 391 | 3.66k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 3.66k | mapped = nullptr; | 393 | 3.66k | } | 394 | 3.66k | }); | 395 | 3.66k | if (data.has_null_key_data()) { | 396 | 360 | auto st = _destroy_agg_status( | 397 | 360 | data.template get_null_key_data< | 398 | 360 | vectorized::AggregateDataPtr>()); | 399 | 360 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 360 | } | 403 | 3.66k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIN4wide7integerILm256EjEENS6_15DataWithNullKeyI9PHHashMapISB_Pc9HashCRC32ISB_EEEEEEEEEEvS3_ Line | Count | Source | 387 | 66 | [&](auto& agg_method) -> void { | 388 | 66 | auto& data = *agg_method.hash_table; | 389 | 66 | data.for_each_mapped([&](auto& mapped) { | 390 | 66 | if (mapped) { | 391 | 66 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 66 | mapped = nullptr; | 393 | 66 | } | 394 | 66 | }); | 395 | 66 | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 66 | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIN4wide7integerILm128EjEENS6_15DataWithNullKeyI9PHHashMapISB_Pc9HashCRC32ISB_EEEEEEEEEEvS3_ Line | Count | Source | 387 | 268 | [&](auto& agg_method) -> void { | 388 | 268 | auto& data = *agg_method.hash_table; | 389 | 268 | data.for_each_mapped([&](auto& mapped) { | 390 | 268 | if (mapped) { | 391 | 268 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 268 | mapped = nullptr; | 393 | 268 | } | 394 | 268 | }); | 395 | 268 | if (data.has_null_key_data()) { | 396 | 4 | auto st = _destroy_agg_status( | 397 | 4 | data.template get_null_key_data< | 398 | 4 | vectorized::AggregateDataPtr>()); | 399 | 4 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 4 | } | 403 | 268 | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberImNS6_15DataWithNullKeyI9PHHashMapImPc14HashMixWrapperIm9HashCRC32ImEEEEEEEEEEEvS3_ Line | Count | Source | 387 | 1.72k | [&](auto& agg_method) -> void { | 388 | 1.72k | auto& data = *agg_method.hash_table; | 389 | 1.72k | data.for_each_mapped([&](auto& mapped) { | 390 | 1.72k | if (mapped) { | 391 | 1.72k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 1.72k | mapped = nullptr; | 393 | 1.72k | } | 394 | 1.72k | }); | 395 | 1.72k | if (data.has_null_key_data()) { | 396 | 135 | auto st = _destroy_agg_status( | 397 | 135 | data.template get_null_key_data< | 398 | 135 | vectorized::AggregateDataPtr>()); | 399 | 135 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 135 | } | 403 | 1.72k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIjNS6_15DataWithNullKeyI9PHHashMapIjPc14HashMixWrapperIj9HashCRC32IjEEEEEEEEEEEvS3_ Line | Count | Source | 387 | 9.01k | [&](auto& agg_method) -> void { | 388 | 9.01k | auto& data = *agg_method.hash_table; | 389 | 9.01k | data.for_each_mapped([&](auto& mapped) { | 390 | 9.01k | if (mapped) { | 391 | 9.01k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 9.01k | mapped = nullptr; | 393 | 9.01k | } | 394 | 9.01k | }); | 395 | 9.01k | if (data.has_null_key_data()) { | 396 | 794 | auto st = _destroy_agg_status( | 397 | 794 | data.template get_null_key_data< | 398 | 794 | vectorized::AggregateDataPtr>()); | 399 | 794 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 794 | } | 403 | 9.01k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberImNS6_15DataWithNullKeyI9PHHashMapImPc9HashCRC32ImEEEEEEEEEEvS3_ Line | Count | Source | 387 | 2.17k | [&](auto& agg_method) -> void { | 388 | 2.17k | auto& data = *agg_method.hash_table; | 389 | 2.17k | data.for_each_mapped([&](auto& mapped) { | 390 | 2.17k | if (mapped) { | 391 | 2.17k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 2.17k | mapped = nullptr; | 393 | 2.17k | } | 394 | 2.17k | }); | 395 | 2.17k | if (data.has_null_key_data()) { | 396 | 122 | auto st = _destroy_agg_status( | 397 | 122 | data.template get_null_key_data< | 398 | 122 | vectorized::AggregateDataPtr>()); | 399 | 122 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 122 | } | 403 | 2.17k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIjNS6_15DataWithNullKeyI9PHHashMapIjPc9HashCRC32IjEEEEEEEEEEvS3_ Line | Count | Source | 387 | 16.8k | [&](auto& agg_method) -> void { | 388 | 16.8k | auto& data = *agg_method.hash_table; | 389 | 16.8k | data.for_each_mapped([&](auto& mapped) { | 390 | 16.8k | if (mapped) { | 391 | 16.8k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 16.8k | mapped = nullptr; | 393 | 16.8k | } | 394 | 16.8k | }); | 395 | 16.8k | if (data.has_null_key_data()) { | 396 | 1.31k | auto st = _destroy_agg_status( | 397 | 1.31k | data.template get_null_key_data< | 398 | 1.31k | vectorized::AggregateDataPtr>()); | 399 | 1.31k | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 1.31k | } | 403 | 16.8k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberItNS6_15DataWithNullKeyI9PHHashMapItPc9HashCRC32ItEEEEEEEEEEvS3_ Line | Count | Source | 387 | 1.23k | [&](auto& agg_method) -> void { | 388 | 1.23k | auto& data = *agg_method.hash_table; | 389 | 1.23k | data.for_each_mapped([&](auto& mapped) { | 390 | 1.23k | if (mapped) { | 391 | 1.23k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 1.23k | mapped = nullptr; | 393 | 1.23k | } | 394 | 1.23k | }); | 395 | 1.23k | if (data.has_null_key_data()) { | 396 | 60 | auto st = _destroy_agg_status( | 397 | 60 | data.template get_null_key_data< | 398 | 60 | vectorized::AggregateDataPtr>()); | 399 | 60 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 60 | } | 403 | 1.23k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized26MethodSingleNullableColumnINS6_15MethodOneNumberIhNS6_15DataWithNullKeyI9PHHashMapIhPc9HashCRC32IhEEEEEEEEEEvS3_ Line | Count | Source | 387 | 4.50k | [&](auto& agg_method) -> void { | 388 | 4.50k | auto& data = *agg_method.hash_table; | 389 | 4.50k | data.for_each_mapped([&](auto& mapped) { | 390 | 4.50k | if (mapped) { | 391 | 4.50k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 4.50k | mapped = nullptr; | 393 | 4.50k | } | 394 | 4.50k | }); | 395 | 4.50k | if (data.has_null_key_data()) { | 396 | 750 | auto st = _destroy_agg_status( | 397 | 750 | data.template get_null_key_data< | 398 | 750 | vectorized::AggregateDataPtr>()); | 399 | 750 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 750 | } | 403 | 4.50k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIm9PHHashMapImPc14HashMixWrapperIm9HashCRC32ImEEEEEEEvS3_ Line | Count | Source | 387 | 992 | [&](auto& agg_method) -> void { | 388 | 992 | auto& data = *agg_method.hash_table; | 389 | 992 | data.for_each_mapped([&](auto& mapped) { | 390 | 992 | if (mapped) { | 391 | 992 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 992 | mapped = nullptr; | 393 | 992 | } | 394 | 992 | }); | 395 | 992 | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 992 | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIj9PHHashMapIjPc14HashMixWrapperIj9HashCRC32IjEEEEEEEvS3_ Line | Count | Source | 387 | 3.00k | [&](auto& agg_method) -> void { | 388 | 3.00k | auto& data = *agg_method.hash_table; | 389 | 3.00k | data.for_each_mapped([&](auto& mapped) { | 390 | 3.00k | if (mapped) { | 391 | 3.00k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 3.00k | mapped = nullptr; | 393 | 3.00k | } | 394 | 3.00k | }); | 395 | 3.00k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 3.00k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIN4wide7integerILm256EjEE9PHHashMapISA_Pc9HashCRC32ISA_EEEEEEvS3_ Line | Count | Source | 387 | 2 | [&](auto& agg_method) -> void { | 388 | 2 | auto& data = *agg_method.hash_table; | 389 | 2 | data.for_each_mapped([&](auto& mapped) { | 390 | 2 | if (mapped) { | 391 | 2 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 2 | mapped = nullptr; | 393 | 2 | } | 394 | 2 | }); | 395 | 2 | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 2 | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIN4wide7integerILm128EjEE9PHHashMapISA_Pc9HashCRC32ISA_EEEEEEvS3_ Line | Count | Source | 387 | 134 | [&](auto& agg_method) -> void { | 388 | 134 | auto& data = *agg_method.hash_table; | 389 | 134 | data.for_each_mapped([&](auto& mapped) { | 390 | 134 | if (mapped) { | 391 | 134 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 134 | mapped = nullptr; | 393 | 134 | } | 394 | 134 | }); | 395 | 134 | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 134 | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized19MethodStringNoCacheINS_13StringHashMapIPcNS_9AllocatorILb1ELb1ELb0ENS_22DefaultMemoryAllocatorELb1EEEEEEEEEvS3_ Line | Count | Source | 387 | 1.57k | [&](auto& agg_method) -> void { | 388 | 1.57k | auto& data = *agg_method.hash_table; | 389 | 1.57k | data.for_each_mapped([&](auto& mapped) { | 390 | 1.57k | if (mapped) { | 391 | 1.57k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 1.57k | mapped = nullptr; | 393 | 1.57k | } | 394 | 1.57k | }); | 395 | 1.57k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 1.57k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIm9PHHashMapImPc9HashCRC32ImEEEEEEvS3_ Line | Count | Source | 387 | 746 | [&](auto& agg_method) -> void { | 388 | 746 | auto& data = *agg_method.hash_table; | 389 | 746 | data.for_each_mapped([&](auto& mapped) { | 390 | 746 | if (mapped) { | 391 | 746 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 746 | mapped = nullptr; | 393 | 746 | } | 394 | 746 | }); | 395 | 746 | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 746 | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIj9PHHashMapIjPc9HashCRC32IjEEEEEEvS3_ Line | Count | Source | 387 | 3.97k | [&](auto& agg_method) -> void { | 388 | 3.97k | auto& data = *agg_method.hash_table; | 389 | 3.97k | data.for_each_mapped([&](auto& mapped) { | 390 | 3.97k | if (mapped) { | 391 | 3.97k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 3.97k | mapped = nullptr; | 393 | 3.97k | } | 394 | 3.97k | }); | 395 | 3.97k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 3.97k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIt9PHHashMapItPc9HashCRC32ItEEEEEEvS3_ Line | Count | Source | 387 | 98 | [&](auto& agg_method) -> void { | 388 | 98 | auto& data = *agg_method.hash_table; | 389 | 98 | data.for_each_mapped([&](auto& mapped) { | 390 | 98 | if (mapped) { | 391 | 98 | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 98 | mapped = nullptr; | 393 | 98 | } | 394 | 98 | }); | 395 | 98 | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 98 | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized15MethodOneNumberIh9PHHashMapIhPc9HashCRC32IhEEEEEEvS3_ Line | Count | Source | 387 | 1.33k | [&](auto& agg_method) -> void { | 388 | 1.33k | auto& data = *agg_method.hash_table; | 389 | 1.33k | data.for_each_mapped([&](auto& mapped) { | 390 | 1.33k | if (mapped) { | 391 | 1.33k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 1.33k | mapped = nullptr; | 393 | 1.33k | } | 394 | 1.33k | }); | 395 | 1.33k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 1.33k | }}, |
_ZZN5doris8pipeline14AggSharedState26_close_with_serialized_keyEvENKUlRT_E_clINS_10vectorized16MethodSerializedI9PHHashMapINS_9StringRefEPc11DefaultHashIS9_vEEEEEEvS3_ Line | Count | Source | 387 | 9.00k | [&](auto& agg_method) -> void { | 388 | 9.00k | auto& data = *agg_method.hash_table; | 389 | 9.00k | data.for_each_mapped([&](auto& mapped) { | 390 | 9.00k | if (mapped) { | 391 | 9.00k | static_cast<void>(_destroy_agg_status(mapped)); | 392 | 9.00k | mapped = nullptr; | 393 | 9.00k | } | 394 | 9.00k | }); | 395 | 9.00k | if (data.has_null_key_data()) { | 396 | 0 | auto st = _destroy_agg_status( | 397 | 0 | data.template get_null_key_data< | 398 | 0 | vectorized::AggregateDataPtr>()); | 399 | 0 | if (!st) { | 400 | 0 | throw Exception(st.code(), st.to_string()); | 401 | 0 | } | 402 | 0 | } | 403 | 9.00k | }}, |
|
404 | 77.6k | agg_data->method_variant); |
405 | 77.6k | } |
406 | | |
407 | 139k | void _close_without_key() { |
408 | | //because prepare maybe failed, and couldn't create agg data. |
409 | | //but finally call close to destory agg data, if agg data has bitmapValue |
410 | | //will be core dump, it's not initialized |
411 | 140k | if (agg_data_created_without_key) { |
412 | 140k | static_cast<void>(_destroy_agg_status(agg_data->without_key)); |
413 | 140k | agg_data_created_without_key = false; |
414 | 140k | } |
415 | 139k | } |
416 | | Status _destroy_agg_status(vectorized::AggregateDataPtr data); |
417 | | }; |
418 | | |
419 | | struct BasicSpillSharedState { |
420 | 575k | virtual ~BasicSpillSharedState() = default; |
421 | | |
422 | | // These two counters are shared to spill source operators as the initial value |
423 | | // of 'SpillWriteFileCurrentBytes' and 'SpillWriteFileCurrentCount'. |
424 | | // Total bytes of spill data written to disk file(after serialized) |
425 | | RuntimeProfile::Counter* _spill_write_file_total_size = nullptr; |
426 | | RuntimeProfile::Counter* _spill_file_total_count = nullptr; |
427 | | |
428 | 581k | void setup_shared_profile(RuntimeProfile* sink_profile) { |
429 | 581k | _spill_file_total_count = |
430 | 581k | ADD_COUNTER_WITH_LEVEL(sink_profile, "SpillWriteFileTotalCount", TUnit::UNIT, 1); |
431 | 581k | _spill_write_file_total_size = |
432 | 581k | ADD_COUNTER_WITH_LEVEL(sink_profile, "SpillWriteFileBytes", TUnit::BYTES, 1); |
433 | 581k | } |
434 | | |
435 | | virtual void update_spill_stream_profiles(RuntimeProfile* source_profile) = 0; |
436 | | }; |
437 | | |
438 | | struct AggSpillPartition; |
439 | | struct PartitionedAggSharedState : public BasicSharedState, |
440 | | public BasicSpillSharedState, |
441 | | public std::enable_shared_from_this<PartitionedAggSharedState> { |
442 | | ENABLE_FACTORY_CREATOR(PartitionedAggSharedState) |
443 | | |
444 | 66.3k | PartitionedAggSharedState() = default; |
445 | 65.9k | ~PartitionedAggSharedState() override = default; |
446 | | |
447 | | void update_spill_stream_profiles(RuntimeProfile* source_profile) override; |
448 | | |
449 | | void init_spill_params(size_t spill_partition_count); |
450 | | |
451 | | void close(); |
452 | | |
453 | | AggSharedState* in_mem_shared_state = nullptr; |
454 | | std::shared_ptr<BasicSharedState> in_mem_shared_state_sptr; |
455 | | |
456 | | size_t partition_count; |
457 | | size_t max_partition_index; |
458 | | bool is_spilled = false; |
459 | | std::atomic_bool is_closed = false; |
460 | | std::deque<std::shared_ptr<AggSpillPartition>> spill_partitions; |
461 | | |
462 | 10.3M | size_t get_partition_index(size_t hash_value) const { return hash_value % partition_count; } |
463 | | }; |
464 | | |
465 | | struct AggSpillPartition { |
466 | | static constexpr int64_t AGG_SPILL_FILE_SIZE = 1024 * 1024 * 1024; // 1G |
467 | | |
468 | 2.12M | AggSpillPartition() = default; |
469 | | |
470 | | void close(); |
471 | | |
472 | | Status get_spill_stream(RuntimeState* state, int node_id, RuntimeProfile* profile, |
473 | | vectorized::SpillStreamSPtr& spilling_stream); |
474 | | |
475 | 176k | Status flush_if_full() { |
476 | 176k | DCHECK(spilling_stream_); |
477 | 176k | Status status; |
478 | | // avoid small spill files |
479 | 176k | if (spilling_stream_->get_written_bytes() >= AGG_SPILL_FILE_SIZE) { |
480 | 0 | status = spilling_stream_->spill_eof(); |
481 | 0 | spilling_stream_.reset(); |
482 | 0 | } |
483 | 176k | return status; |
484 | 176k | } |
485 | | |
486 | 537k | Status finish_current_spilling(bool eos = false) { |
487 | 537k | if (spilling_stream_) { |
488 | 215k | if (eos || spilling_stream_->get_written_bytes() >= AGG_SPILL_FILE_SIZE) { |
489 | 20.2k | auto status = spilling_stream_->spill_eof(); |
490 | 20.2k | spilling_stream_.reset(); |
491 | 20.2k | return status; |
492 | 20.2k | } |
493 | 215k | } |
494 | 517k | return Status::OK(); |
495 | 537k | } |
496 | | |
497 | | std::deque<vectorized::SpillStreamSPtr> spill_streams_; |
498 | | vectorized::SpillStreamSPtr spilling_stream_; |
499 | | }; |
500 | | using AggSpillPartitionSPtr = std::shared_ptr<AggSpillPartition>; |
501 | | struct SortSharedState : public BasicSharedState { |
502 | | ENABLE_FACTORY_CREATOR(SortSharedState) |
503 | | public: |
504 | | std::shared_ptr<vectorized::Sorter> sorter; |
505 | | }; |
506 | | |
507 | | struct SpillSortSharedState : public BasicSharedState, |
508 | | public BasicSpillSharedState, |
509 | | public std::enable_shared_from_this<SpillSortSharedState> { |
510 | | ENABLE_FACTORY_CREATOR(SpillSortSharedState) |
511 | | |
512 | 479k | SpillSortSharedState() = default; |
513 | 472k | ~SpillSortSharedState() override = default; |
514 | | |
515 | 200k | void update_spill_block_batch_row_count(RuntimeState* state, const vectorized::Block* block) { |
516 | 200k | auto rows = block->rows(); |
517 | 200k | if (rows > 0 && 0 == avg_row_bytes) { |
518 | 152k | avg_row_bytes = std::max((std::size_t)1, block->bytes() / rows); |
519 | 152k | spill_block_batch_row_count = |
520 | 152k | (state->spill_sort_batch_bytes() + avg_row_bytes - 1) / avg_row_bytes; |
521 | 152k | LOG(INFO) << "spill sort block batch row count: " << spill_block_batch_row_count; |
522 | 152k | } |
523 | 200k | } |
524 | | |
525 | | void update_spill_stream_profiles(RuntimeProfile* source_profile) override; |
526 | | |
527 | | void close(); |
528 | | |
529 | | SortSharedState* in_mem_shared_state = nullptr; |
530 | | bool enable_spill = false; |
531 | | bool is_spilled = false; |
532 | | std::atomic_bool is_closed = false; |
533 | | std::shared_ptr<BasicSharedState> in_mem_shared_state_sptr; |
534 | | |
535 | | std::deque<vectorized::SpillStreamSPtr> sorted_streams; |
536 | | size_t avg_row_bytes = 0; |
537 | | size_t spill_block_batch_row_count; |
538 | | }; |
539 | | |
540 | | struct UnionSharedState : public BasicSharedState { |
541 | | ENABLE_FACTORY_CREATOR(UnionSharedState) |
542 | | |
543 | | public: |
544 | 5.24k | UnionSharedState(int child_count = 1) : data_queue(child_count), _child_count(child_count) {}; |
545 | 0 | int child_count() const { return _child_count; } |
546 | | DataQueue data_queue; |
547 | | const int _child_count; |
548 | | }; |
549 | | |
550 | | struct DataQueueSharedState : public BasicSharedState { |
551 | | ENABLE_FACTORY_CREATOR(DataQueueSharedState) |
552 | | public: |
553 | | DataQueue data_queue; |
554 | | }; |
555 | | |
556 | | class MultiCastDataStreamer; |
557 | | |
558 | | struct MultiCastSharedState : public BasicSharedState, |
559 | | public BasicSpillSharedState, |
560 | | public std::enable_shared_from_this<MultiCastSharedState> { |
561 | | MultiCastSharedState(ObjectPool* pool, int cast_sender_count, int node_id); |
562 | | std::unique_ptr<pipeline::MultiCastDataStreamer> multi_cast_data_streamer; |
563 | | |
564 | | void update_spill_stream_profiles(RuntimeProfile* source_profile) override; |
565 | | }; |
566 | | |
567 | | struct AnalyticSharedState : public BasicSharedState { |
568 | | ENABLE_FACTORY_CREATOR(AnalyticSharedState) |
569 | | |
570 | | public: |
571 | 28.1k | AnalyticSharedState() = default; |
572 | | std::queue<vectorized::Block> blocks_buffer; |
573 | | std::mutex buffer_mutex; |
574 | | bool sink_eos = false; |
575 | | std::mutex sink_eos_lock; |
576 | | }; |
577 | | |
578 | | struct JoinSharedState : public BasicSharedState { |
579 | | // For some join case, we can apply a short circuit strategy |
580 | | // 1. _has_null_in_build_side = true |
581 | | // 2. build side rows is empty, Join op is: inner join/right outer join/left semi/right semi/right anti |
582 | | bool _has_null_in_build_side = false; |
583 | | bool short_circuit_for_probe = false; |
584 | | // for some join, when build side rows is empty, we could return directly by add some additional null data in probe table. |
585 | | bool empty_right_table_need_probe_dispose = false; |
586 | | JoinOpVariants join_op_variants; |
587 | | }; |
588 | | |
589 | | struct HashJoinSharedState : public JoinSharedState { |
590 | | ENABLE_FACTORY_CREATOR(HashJoinSharedState) |
591 | 184k | HashJoinSharedState() { |
592 | 184k | hash_table_variant_vector.push_back(std::make_shared<JoinDataVariants>()); |
593 | 184k | } |
594 | 6.37k | HashJoinSharedState(int num_instances) { |
595 | 6.37k | source_deps.resize(num_instances, nullptr); |
596 | 6.37k | hash_table_variant_vector.resize(num_instances, nullptr); |
597 | 38.3k | for (int i = 0; i < num_instances; i++) { |
598 | 31.9k | hash_table_variant_vector[i] = std::make_shared<JoinDataVariants>(); |
599 | 31.9k | } |
600 | 6.37k | } |
601 | | std::shared_ptr<vectorized::Arena> arena = std::make_shared<vectorized::Arena>(); |
602 | | |
603 | | const std::vector<TupleDescriptor*> build_side_child_desc; |
604 | | size_t build_exprs_size = 0; |
605 | | std::shared_ptr<vectorized::Block> build_block; |
606 | | std::shared_ptr<std::vector<uint32_t>> build_indexes_null; |
607 | | |
608 | | // Used by shared hash table |
609 | | // For probe operator, hash table in _hash_table_variants is read-only if visited flags is not |
610 | | // used. (visited flags will be used only in right / full outer join). |
611 | | // |
612 | | // For broadcast join, although hash table is read-only, some states in `_hash_table_variants` |
613 | | // are still could be written. For example, serialized keys will be written in a continuous |
614 | | // memory in `_hash_table_variants`. So before execution, we should use a local _hash_table_variants |
615 | | // which has a shared hash table in it. |
616 | | std::vector<std::shared_ptr<JoinDataVariants>> hash_table_variant_vector; |
617 | | }; |
618 | | |
619 | | struct PartitionedHashJoinSharedState |
620 | | : public HashJoinSharedState, |
621 | | public BasicSpillSharedState, |
622 | | public std::enable_shared_from_this<PartitionedHashJoinSharedState> { |
623 | | ENABLE_FACTORY_CREATOR(PartitionedHashJoinSharedState) |
624 | | |
625 | 30.4k | void update_spill_stream_profiles(RuntimeProfile* source_profile) override { |
626 | 974k | for (auto& stream : spilled_streams) { |
627 | 974k | if (stream) { |
628 | 974k | stream->update_shared_profiles(source_profile); |
629 | 974k | } |
630 | 974k | } |
631 | 30.4k | } |
632 | | |
633 | | std::unique_ptr<RuntimeState> inner_runtime_state; |
634 | | std::shared_ptr<HashJoinSharedState> inner_shared_state; |
635 | | std::vector<std::unique_ptr<vectorized::MutableBlock>> partitioned_build_blocks; |
636 | | std::vector<vectorized::SpillStreamSPtr> spilled_streams; |
637 | | bool need_to_spill = false; |
638 | | }; |
639 | | |
640 | | struct NestedLoopJoinSharedState : public JoinSharedState { |
641 | | ENABLE_FACTORY_CREATOR(NestedLoopJoinSharedState) |
642 | | // if true, left child has no more rows to process |
643 | | bool left_side_eos = false; |
644 | | // Visited flags for each row in build side. |
645 | | vectorized::MutableColumns build_side_visited_flags; |
646 | | // List of build blocks, constructed in prepare() |
647 | | vectorized::Blocks build_blocks; |
648 | | }; |
649 | | |
650 | | struct PartitionSortNodeSharedState : public BasicSharedState { |
651 | | ENABLE_FACTORY_CREATOR(PartitionSortNodeSharedState) |
652 | | public: |
653 | | std::queue<vectorized::Block> blocks_buffer; |
654 | | std::mutex buffer_mutex; |
655 | | std::vector<std::unique_ptr<vectorized::PartitionSorter>> partition_sorts; |
656 | | bool sink_eos = false; |
657 | | std::mutex sink_eos_lock; |
658 | | std::mutex prepared_finish_lock; |
659 | | }; |
660 | | |
661 | | struct SetSharedState : public BasicSharedState { |
662 | | ENABLE_FACTORY_CREATOR(SetSharedState) |
663 | | public: |
664 | | /// default init |
665 | | vectorized::Block build_block; // build to source |
666 | | //record element size in hashtable |
667 | | int64_t valid_element_in_hash_tbl = 0; |
668 | | //first: idx mapped to column types |
669 | | //second: column_id, could point to origin column or cast column |
670 | | std::unordered_map<int, int> build_col_idx; |
671 | | |
672 | | //// shared static states (shared, decided in prepare/open...) |
673 | | |
674 | | /// init in setup_local_state |
675 | | std::unique_ptr<SetDataVariants> hash_table_variants = |
676 | | std::make_unique<SetDataVariants>(); // the real data HERE. |
677 | | std::vector<bool> build_not_ignore_null; |
678 | | |
679 | | // The SET operator's child might have different nullable attributes. |
680 | | // If a calculation involves both nullable and non-nullable columns, the final output should be a nullable column |
681 | | Status update_build_not_ignore_null(const vectorized::VExprContextSPtrs& ctxs); |
682 | | |
683 | | size_t get_hash_table_size() const; |
684 | | /// init in both upstream side. |
685 | | //The i-th result expr list refers to the i-th child. |
686 | | std::vector<vectorized::VExprContextSPtrs> child_exprs_lists; |
687 | | |
688 | | /// init in build side |
689 | | size_t child_quantity; |
690 | | vectorized::VExprContextSPtrs build_child_exprs; |
691 | | std::vector<Dependency*> probe_finished_children_dependency; |
692 | | |
693 | | /// init in probe side |
694 | | std::vector<vectorized::VExprContextSPtrs> probe_child_exprs_lists; |
695 | | |
696 | | std::atomic<bool> ready_for_read = false; |
697 | | |
698 | | /// called in setup_local_state |
699 | | Status hash_table_init(); |
700 | | }; |
701 | | |
702 | | enum class ExchangeType : uint8_t { |
703 | | NOOP = 0, |
704 | | // Shuffle data by Crc32HashPartitioner<LocalExchangeChannelIds>. |
705 | | HASH_SHUFFLE = 1, |
706 | | // Round-robin passthrough data blocks. |
707 | | PASSTHROUGH = 2, |
708 | | // Shuffle data by Crc32HashPartitioner<ShuffleChannelIds> (e.g. same as storage engine). |
709 | | BUCKET_HASH_SHUFFLE = 3, |
710 | | // Passthrough data blocks to all channels. |
711 | | BROADCAST = 4, |
712 | | // Passthrough data to channels evenly in an adaptive way. |
713 | | ADAPTIVE_PASSTHROUGH = 5, |
714 | | // Send all data to the first channel. |
715 | | PASS_TO_ONE = 6, |
716 | | }; |
717 | | |
718 | 307k | inline std::string get_exchange_type_name(ExchangeType idx) { |
719 | 307k | switch (idx) { |
720 | 13 | case ExchangeType::NOOP: |
721 | 13 | return "NOOP"; |
722 | 16.7k | case ExchangeType::HASH_SHUFFLE: |
723 | 16.7k | return "HASH_SHUFFLE"; |
724 | 271k | case ExchangeType::PASSTHROUGH: |
725 | 271k | return "PASSTHROUGH"; |
726 | 4.76k | case ExchangeType::BUCKET_HASH_SHUFFLE: |
727 | 4.76k | return "BUCKET_HASH_SHUFFLE"; |
728 | 1.04k | case ExchangeType::BROADCAST: |
729 | 1.04k | return "BROADCAST"; |
730 | 5.50k | case ExchangeType::ADAPTIVE_PASSTHROUGH: |
731 | 5.50k | return "ADAPTIVE_PASSTHROUGH"; |
732 | 7.99k | case ExchangeType::PASS_TO_ONE: |
733 | 7.99k | return "PASS_TO_ONE"; |
734 | 307k | } |
735 | 0 | throw Exception(Status::FatalError("__builtin_unreachable")); |
736 | 307k | } |
737 | | |
738 | | struct DataDistribution { |
739 | 5.42M | DataDistribution(ExchangeType type) : distribution_type(type) {} |
740 | | DataDistribution(ExchangeType type, const std::vector<TExpr>& partition_exprs_) |
741 | 148k | : distribution_type(type), partition_exprs(partition_exprs_) {} |
742 | 325k | DataDistribution(const DataDistribution& other) = default; |
743 | 1.61M | bool need_local_exchange() const { return distribution_type != ExchangeType::NOOP; } |
744 | 1.76M | DataDistribution& operator=(const DataDistribution& other) = default; |
745 | | ExchangeType distribution_type; |
746 | | std::vector<TExpr> partition_exprs; |
747 | | }; |
748 | | |
749 | | class ExchangerBase; |
750 | | |
751 | | struct LocalExchangeSharedState : public BasicSharedState { |
752 | | public: |
753 | | ENABLE_FACTORY_CREATOR(LocalExchangeSharedState); |
754 | | LocalExchangeSharedState(int num_instances); |
755 | | ~LocalExchangeSharedState() override; |
756 | | std::unique_ptr<ExchangerBase> exchanger {}; |
757 | | std::vector<RuntimeProfile::Counter*> mem_counters; |
758 | | std::atomic<int64_t> mem_usage = 0; |
759 | | std::atomic<size_t> _buffer_mem_limit = config::local_exchange_buffer_mem_limit; |
760 | | // We need to make sure to add mem_usage first and then enqueue, otherwise sub mem_usage may cause negative mem_usage during concurrent dequeue. |
761 | | std::mutex le_lock; |
762 | | void sub_running_sink_operators(); |
763 | | void sub_running_source_operators(); |
764 | 307k | void _set_always_ready() { |
765 | 2.02M | for (auto& dep : source_deps) { |
766 | 2.02M | DCHECK(dep); |
767 | 2.02M | dep->set_always_ready(); |
768 | 2.02M | } |
769 | 307k | for (auto& dep : sink_deps) { |
770 | 307k | DCHECK(dep); |
771 | 307k | dep->set_always_ready(); |
772 | 307k | } |
773 | 307k | } |
774 | | |
775 | 443k | Dependency* get_sink_dep_by_channel_id(int channel_id) { return nullptr; } |
776 | | |
777 | 550k | void set_ready_to_read(int channel_id) { |
778 | 550k | auto& dep = source_deps[channel_id]; |
779 | 18.4E | DCHECK(dep) << channel_id; |
780 | 550k | dep->set_ready(); |
781 | 550k | } |
782 | | |
783 | 550k | void add_mem_usage(int channel_id, size_t delta) { mem_counters[channel_id]->update(delta); } |
784 | | |
785 | 550k | void sub_mem_usage(int channel_id, size_t delta) { |
786 | 550k | mem_counters[channel_id]->update(-(int64_t)delta); |
787 | 550k | } |
788 | | |
789 | 467k | void add_total_mem_usage(size_t delta) { |
790 | 467k | if (cast_set<int64_t>(mem_usage.fetch_add(delta) + delta) > _buffer_mem_limit) { |
791 | 17 | sink_deps.front()->block(); |
792 | 17 | } |
793 | 467k | } |
794 | | |
795 | 467k | void sub_total_mem_usage(size_t delta) { |
796 | 467k | auto prev_usage = mem_usage.fetch_sub(delta); |
797 | 467k | DCHECK_GE(prev_usage - delta, 0) << "prev_usage: " << prev_usage << " delta: " << delta; |
798 | 467k | if (cast_set<int64_t>(prev_usage - delta) <= _buffer_mem_limit) { |
799 | 467k | sink_deps.front()->set_ready(); |
800 | 467k | } |
801 | 467k | } |
802 | | |
803 | 51.1k | void set_low_memory_mode(RuntimeState* state) { |
804 | 51.1k | _buffer_mem_limit = std::min<int64_t>(config::local_exchange_buffer_mem_limit, |
805 | 51.1k | state->low_memory_mode_buffer_limit()); |
806 | 51.1k | } |
807 | | }; |
808 | | |
809 | | struct FetchRpcStruct { |
810 | | std::shared_ptr<PBackendService_Stub> stub; |
811 | | PMultiGetRequestV2 request; |
812 | | std::shared_ptr<doris::DummyBrpcCallback<PMultiGetResponseV2>> callback; |
813 | | MonotonicStopWatch rpc_timer; |
814 | | }; |
815 | | |
816 | | struct MaterializationSharedState : public BasicSharedState { |
817 | | ENABLE_FACTORY_CREATOR(MaterializationSharedState) |
818 | | public: |
819 | 1.51k | MaterializationSharedState() = default; |
820 | | |
821 | | Status init_multi_requests(const TMaterializationNode& tnode, RuntimeState* state); |
822 | | Status create_muiltget_result(const vectorized::Columns& columns, bool eos, bool gc_id_map); |
823 | | Status merge_multi_response(vectorized::Block* block); |
824 | | |
825 | | void create_counter_dependency(int operator_id, int node_id, const std::string& name); |
826 | | |
827 | | private: |
828 | | void _update_profile_info(int64_t backend_id, RuntimeProfile* response_profile); |
829 | | |
830 | | public: |
831 | | bool rpc_struct_inited = false; |
832 | | AtomicStatus rpc_status; |
833 | | |
834 | | bool last_block = false; |
835 | | // empty materialization sink block not need to merge block |
836 | | bool need_merge_block = true; |
837 | | vectorized::Block origin_block; |
838 | | // The rowid column of the origin block. should be replaced by the column of the result block. |
839 | | std::vector<int> rowid_locs; |
840 | | std::vector<vectorized::MutableBlock> response_blocks; |
841 | | std::map<int64_t, FetchRpcStruct> rpc_struct_map; |
842 | | // Register each line in which block to ensure the order of the result. |
843 | | // Zero means NULL value. |
844 | | std::vector<std::vector<int64_t>> block_order_results; |
845 | | // backend id => <rpc profile info string key, rpc profile info string value>. |
846 | | std::map<int64_t, std::map<std::string, fmt::memory_buffer>> backend_profile_info_string; |
847 | | }; |
848 | | #include "common/compile_check_end.h" |
849 | | } // namespace doris::pipeline |