be/src/exec/exchange/vdata_stream_recvr.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 <gen_cpp/Types_types.h> |
21 | | #include <gen_cpp/data.pb.h> |
22 | | #include <glog/logging.h> |
23 | | #include <google/protobuf/stubs/callback.h> |
24 | | |
25 | | #include <atomic> |
26 | | #include <condition_variable> |
27 | | #include <cstddef> |
28 | | #include <cstdint> |
29 | | #include <deque> |
30 | | #include <list> |
31 | | #include <memory> |
32 | | #include <mutex> |
33 | | #include <ostream> |
34 | | #include <sstream> |
35 | | #include <thread> |
36 | | #include <unordered_map> |
37 | | #include <unordered_set> |
38 | | #include <utility> |
39 | | #include <vector> |
40 | | |
41 | | #include "common/config.h" |
42 | | #include "common/global_types.h" |
43 | | #include "common/object_pool.h" |
44 | | #include "common/status.h" |
45 | | #include "core/block/block.h" |
46 | | #include "exprs/vexpr_fwd.h" |
47 | | #include "runtime/descriptors.h" |
48 | | #include "runtime/runtime_profile.h" |
49 | | #include "runtime/task_execution_context.h" |
50 | | #include "runtime/thread_context.h" |
51 | | #include "runtime/workload_group/workload_group.h" |
52 | | #include "util/stopwatch.hpp" |
53 | | |
54 | | namespace doris { |
55 | | #include "common/compile_check_begin.h" |
56 | | class MemTracker; |
57 | | class PBlock; |
58 | | class MemTrackerLimiter; |
59 | | class RuntimeState; |
60 | | |
61 | | class Dependency; |
62 | | class ExchangeLocalState; |
63 | | |
64 | | class VDataStreamMgr; |
65 | | class VSortedRunMerger; |
66 | | |
67 | | class VDataStreamRecvr; |
68 | | |
69 | | class VDataStreamRecvr : public HasTaskExecutionCtx { |
70 | | public: |
71 | | class SenderQueue; |
72 | | VDataStreamRecvr(VDataStreamMgr* stream_mgr, RuntimeProfile::HighWaterMarkCounter* counter, |
73 | | RuntimeState* state, const TUniqueId& fragment_instance_id, |
74 | | PlanNodeId dest_node_id, int num_senders, bool is_merging, |
75 | | RuntimeProfile* profile, size_t data_queue_capacity); |
76 | | |
77 | | ~VDataStreamRecvr() override; |
78 | | |
79 | | MOCK_FUNCTION Status create_merger(const VExprContextSPtrs& ordering_expr, |
80 | | const std::vector<bool>& is_asc_order, |
81 | | const std::vector<bool>& nulls_first, size_t batch_size, |
82 | | int64_t limit, size_t offset); |
83 | | |
84 | 43 | std::vector<SenderQueue*> sender_queues() const { return _sender_queues; } |
85 | | |
86 | | Status add_block(std::unique_ptr<PBlock> pblock, int sender_id, int be_number, |
87 | | int64_t packet_seq, ::google::protobuf::Closure** done, |
88 | | const int64_t wait_for_worker, const uint64_t time_to_find_recvr); |
89 | | |
90 | | Status add_blocks(const PTransmitDataParams* request, ::google::protobuf::Closure** done, |
91 | | const int64_t wait_for_worker, const uint64_t time_to_find_recvr); |
92 | | |
93 | | void add_block(Block* block, int sender_id, bool use_move); |
94 | | std::string debug_string(); |
95 | | |
96 | | MOCK_FUNCTION Status get_next(Block* block, bool* eos); |
97 | | |
98 | 24 | const TUniqueId& fragment_instance_id() const { return _fragment_instance_id; } |
99 | 24 | PlanNodeId dest_node_id() const { return _dest_node_id; } |
100 | | |
101 | | // Indicate that a particular sender is done. Delegated to the appropriate |
102 | | // sender queue. Called from DataStreamMgr. |
103 | | void remove_sender(int sender_id, int be_number, Status exec_status); |
104 | | |
105 | | void cancel_stream(Status exec_status); |
106 | | |
107 | | MOCK_FUNCTION void close(); |
108 | | |
109 | | // When the source reaches eos = true |
110 | | void set_sink_dep_always_ready() const; |
111 | | |
112 | | // Careful: stream sender will call this function for a local receiver, |
113 | | // accessing members of receiver that are allocated by Object pool |
114 | | // in this function is not safe. |
115 | | MOCK_FUNCTION bool exceeds_limit(size_t block_byte_size); |
116 | | bool queue_exceeds_limit(size_t byte_size) const; |
117 | 7 | bool is_closed() const { return _is_closed; } |
118 | | |
119 | | std::shared_ptr<Dependency> get_local_channel_dependency(int sender_id); |
120 | | |
121 | 0 | void set_low_memory_mode() { _sender_queue_mem_limit = 1012 * 1024; } |
122 | | |
123 | | private: |
124 | | friend struct BlockSupplierSortCursorImpl; |
125 | | |
126 | | // DataStreamMgr instance used to create this recvr. (Not owned) |
127 | | VDataStreamMgr* _mgr = nullptr; |
128 | | |
129 | | RuntimeProfile::HighWaterMarkCounter* _memory_used_counter = nullptr; |
130 | | |
131 | | std::shared_ptr<ResourceContext> _resource_ctx; |
132 | | |
133 | | std::shared_ptr<QueryContext> _query_context; |
134 | | |
135 | | // Fragment and node id of the destination exchange node this receiver is used by. |
136 | | TUniqueId _fragment_instance_id; |
137 | | PlanNodeId _dest_node_id; |
138 | | |
139 | | // Row schema, copied from the caller of CreateRecvr(). |
140 | | RowDescriptor _row_desc; |
141 | | |
142 | | // True if this reciver merges incoming rows from different senders. Per-sender |
143 | | // row batch queues are maintained in this case. |
144 | | bool _is_merging; |
145 | | bool _is_closed; |
146 | | |
147 | | std::unique_ptr<MemTracker> _mem_tracker; |
148 | | // Managed by object pool |
149 | | std::vector<SenderQueue*> _sender_queues; |
150 | | |
151 | | std::atomic<size_t> _sender_queue_mem_limit; |
152 | | |
153 | | std::unique_ptr<VSortedRunMerger> _merger; |
154 | | |
155 | | ObjectPool _sender_queue_pool; |
156 | | RuntimeProfile* _profile = nullptr; |
157 | | |
158 | | RuntimeProfile::Counter* _remote_bytes_received_counter = nullptr; |
159 | | RuntimeProfile::Counter* _local_bytes_received_counter = nullptr; |
160 | | RuntimeProfile::Counter* _deserialize_row_batch_timer = nullptr; |
161 | | RuntimeProfile::Counter* _first_batch_wait_total_timer = nullptr; |
162 | | RuntimeProfile::Counter* _buffer_full_total_timer = nullptr; |
163 | | RuntimeProfile::Counter* _data_arrival_timer = nullptr; |
164 | | RuntimeProfile::Counter* _decompress_timer = nullptr; |
165 | | RuntimeProfile::Counter* _decompress_bytes = nullptr; |
166 | | |
167 | | // Number of blocks received |
168 | | RuntimeProfile::Counter* _blocks_produced_counter = nullptr; |
169 | | RuntimeProfile::Counter* _max_wait_worker_time = nullptr; |
170 | | RuntimeProfile::Counter* _max_wait_to_process_time = nullptr; |
171 | | RuntimeProfile::Counter* _max_find_recvr_time = nullptr; |
172 | | |
173 | | std::vector<std::shared_ptr<Dependency>> _sender_to_local_channel_dependency; |
174 | | }; |
175 | | |
176 | | class VDataStreamRecvr::SenderQueue { |
177 | | public: |
178 | | SenderQueue(VDataStreamRecvr* parent_recvr, int num_senders, |
179 | | std::shared_ptr<Dependency> local_channel_dependency); |
180 | | |
181 | | ~SenderQueue(); |
182 | | |
183 | | Status get_batch(Block* next_block, bool* eos); |
184 | | |
185 | | Status add_block(std::unique_ptr<PBlock> pblock, int be_number, int64_t packet_seq, |
186 | | ::google::protobuf::Closure** done, const int64_t wait_for_worker, |
187 | | const uint64_t time_to_find_recvr); |
188 | | |
189 | | Status add_blocks(const PTransmitDataParams* request, ::google::protobuf::Closure** done, |
190 | | const int64_t wait_for_worker, const uint64_t time_to_find_recvr); |
191 | | |
192 | | std::string debug_string(); |
193 | | |
194 | | void add_block(Block* block, bool use_move); |
195 | | |
196 | | void decrement_senders(int sender_id); |
197 | | |
198 | | void cancel(Status cancel_status); |
199 | | |
200 | | void close(); |
201 | | |
202 | 16 | void set_dependency(std::shared_ptr<Dependency> dependency) { _source_dependency = dependency; } |
203 | | |
204 | | protected: |
205 | | void add_blocks_memory_usage(int64_t size); |
206 | | |
207 | | void sub_blocks_memory_usage(int64_t size); |
208 | | |
209 | | bool exceeds_limit(); |
210 | | friend class ExchangeLocalState; |
211 | | |
212 | | void set_source_ready(std::lock_guard<std::mutex>&); |
213 | | |
214 | | // Not managed by this class |
215 | | VDataStreamRecvr* _recvr = nullptr; |
216 | | std::mutex _lock; |
217 | | bool _is_cancelled; |
218 | | Status _cancel_status; |
219 | | int _num_remaining_senders; |
220 | | std::unique_ptr<MemTracker> _queue_mem_tracker; |
221 | | |
222 | | // `BlockItem` is used in `_block_queue` to handle both local and remote exchange blocks. |
223 | | // For local exchange blocks, `BlockUPtr` is used directly without any modification. |
224 | | // For remote exchange blocks, the `pblock` is stored in `BlockItem`. |
225 | | // When `getBlock` is called, the `pblock` is deserialized into a usable block. |
226 | | struct BlockItem { |
227 | 911 | Status get_block(BlockUPtr& block) { |
228 | 911 | if (!_block) { |
229 | 501 | DCHECK(_pblock); |
230 | 501 | SCOPED_RAW_TIMER(&_deserialize_time); |
231 | 501 | _block = Block::create_unique(); |
232 | 501 | RETURN_IF_ERROR_OR_CATCH_EXCEPTION( |
233 | 501 | _block->deserialize(*_pblock, &_decompress_bytes, &_decompress_time)); |
234 | 501 | } |
235 | 911 | block.swap(_block); |
236 | 911 | _block.reset(); |
237 | 911 | return Status::OK(); |
238 | 911 | } |
239 | | |
240 | 911 | size_t block_byte_size() const { return _block_byte_size; } |
241 | 911 | int64_t deserialize_time() const { return _deserialize_time; } |
242 | 911 | int64_t decompress_time() const { return _decompress_time; } |
243 | 911 | size_t decompress_bytes() const { return _decompress_bytes; } |
244 | 921 | BlockItem() = default; |
245 | | BlockItem(BlockUPtr&& block, size_t block_byte_size) |
246 | 414 | : _block(std::move(block)), _block_byte_size(block_byte_size) {} |
247 | | |
248 | | BlockItem(std::unique_ptr<PBlock>&& pblock, size_t block_byte_size) |
249 | 504 | : _block(nullptr), _pblock(std::move(pblock)), _block_byte_size(block_byte_size) {} |
250 | | |
251 | 2 | void set_done(google::protobuf::Closure* done) { |
252 | | // The done callback is only set when the queue memory limit is exceeded. |
253 | 2 | _done_cb = done; |
254 | 2 | _wait_timer.start(); |
255 | 2 | } |
256 | | |
257 | 918 | void call_done(VDataStreamRecvr* recvr) { |
258 | 918 | if (_done_cb != nullptr) { |
259 | 2 | _done_cb->Run(); |
260 | 2 | _done_cb = nullptr; |
261 | 2 | _wait_timer.stop(); |
262 | 2 | int64_t elapse_time = _wait_timer.elapsed_time(); |
263 | 2 | if (recvr->_max_wait_to_process_time->value() < elapse_time) { |
264 | 2 | recvr->_max_wait_to_process_time->set(elapse_time); |
265 | 2 | } |
266 | 2 | recvr->_buffer_full_total_timer->update(elapse_time); |
267 | 2 | } |
268 | 918 | } |
269 | | |
270 | | private: |
271 | | BlockUPtr _block; |
272 | | std::unique_ptr<PBlock> _pblock; |
273 | | size_t _block_byte_size = 0; |
274 | | int64_t _deserialize_time = 0; |
275 | | int64_t _decompress_time = 0; |
276 | | size_t _decompress_bytes = 0; |
277 | | |
278 | | google::protobuf::Closure* _done_cb = nullptr; |
279 | | MonotonicStopWatch _wait_timer; |
280 | | }; |
281 | | |
282 | | std::list<BlockItem> _block_queue; |
283 | | |
284 | | // sender_id |
285 | | std::unordered_set<int> _sender_eos_set; |
286 | | // be_number => packet_seq |
287 | | std::unordered_map<int, int64_t> _packet_seq_map; |
288 | | |
289 | | std::shared_ptr<Dependency> _source_dependency; |
290 | | std::shared_ptr<Dependency> _local_channel_dependency; |
291 | | }; |
292 | | |
293 | | } // namespace doris |
294 | | |
295 | | #include "common/compile_check_end.h" |