be/src/exec/exchange/vdata_stream_recvr.cpp
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 | | #include "exec/exchange/vdata_stream_recvr.h" |
19 | | |
20 | | #include <fmt/format.h> |
21 | | #include <gen_cpp/Metrics_types.h> |
22 | | #include <gen_cpp/Types_types.h> |
23 | | #include <gen_cpp/data.pb.h> |
24 | | |
25 | | #include <algorithm> |
26 | | #include <functional> |
27 | | #include <string> |
28 | | |
29 | | #include "common/logging.h" |
30 | | #include "core/block/block.h" |
31 | | #include "core/block/materialize_block.h" |
32 | | #include "exec/exchange/vdata_stream_mgr.h" |
33 | | #include "exec/operator/exchange_sink_operator.h" |
34 | | #include "exec/operator/exchange_source_operator.h" |
35 | | #include "exec/sort/sort_cursor.h" |
36 | | #include "exec/sort/vsorted_run_merger.h" |
37 | | #include "runtime/memory/mem_tracker.h" |
38 | | #include "runtime/runtime_state.h" |
39 | | #include "runtime/thread_context.h" |
40 | | #include "util/defer_op.h" |
41 | | #include "util/uid_util.h" |
42 | | |
43 | | namespace doris { |
44 | | #include "common/compile_check_begin.h" |
45 | | |
46 | | VDataStreamRecvr::SenderQueue::SenderQueue(VDataStreamRecvr* parent_recvr, int num_senders, |
47 | | std::shared_ptr<Dependency> local_channel_dependency) |
48 | 594k | : _recvr(parent_recvr), |
49 | 594k | _is_cancelled(false), |
50 | 594k | _num_remaining_senders(num_senders), |
51 | 594k | _local_channel_dependency(local_channel_dependency) { |
52 | 594k | _cancel_status = Status::OK(); |
53 | 594k | _queue_mem_tracker = std::make_unique<MemTracker>("local data queue mem tracker"); |
54 | 594k | } |
55 | | |
56 | 601k | VDataStreamRecvr::SenderQueue::~SenderQueue() { |
57 | 601k | for (auto& block_item : _block_queue) { |
58 | 0 | block_item.call_done(_recvr); |
59 | 0 | } |
60 | 601k | _block_queue.clear(); |
61 | 601k | } |
62 | | |
63 | 768k | Status VDataStreamRecvr::SenderQueue::get_batch(Block* block, bool* eos) { |
64 | 768k | #ifndef NDEBUG |
65 | 768k | if (!_is_cancelled && _block_queue.empty() && _num_remaining_senders > 0) { |
66 | 0 | throw doris::Exception(ErrorCode::INTERNAL_ERROR, |
67 | 0 | "_is_cancelled: {}, _block_queue_empty: {}, " |
68 | 0 | "_num_remaining_senders: {}", |
69 | 0 | _is_cancelled, _block_queue.empty(), _num_remaining_senders); |
70 | 0 | } |
71 | 768k | #endif |
72 | 768k | BlockItem block_item; |
73 | 768k | { |
74 | 768k | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
75 | | //check and get block_item from data_queue |
76 | 768k | if (_is_cancelled) { |
77 | 1 | RETURN_IF_ERROR(_cancel_status); |
78 | 1 | return Status::Cancelled("Cancelled"); |
79 | 1 | } |
80 | | |
81 | 768k | if (_block_queue.empty()) { |
82 | 588k | if (_num_remaining_senders != 0) { |
83 | 0 | return Status::InternalError( |
84 | 0 | "Data queue is empty but there are still remaining senders. " |
85 | 0 | "_num_remaining_senders: {}", |
86 | 0 | _num_remaining_senders); |
87 | 0 | } |
88 | 588k | *eos = true; |
89 | 588k | return Status::OK(); |
90 | 588k | } |
91 | | |
92 | 768k | DCHECK(!_block_queue.empty()); |
93 | 179k | block_item = std::move(_block_queue.front()); |
94 | 179k | _block_queue.pop_front(); |
95 | 179k | } |
96 | 0 | BlockUPtr next_block; |
97 | 179k | RETURN_IF_ERROR(block_item.get_block(next_block)); |
98 | 179k | size_t block_byte_size = block_item.block_byte_size(); |
99 | 179k | COUNTER_UPDATE(_recvr->_deserialize_row_batch_timer, block_item.deserialize_time()); |
100 | 179k | COUNTER_UPDATE(_recvr->_decompress_timer, block_item.decompress_time()); |
101 | 179k | COUNTER_UPDATE(_recvr->_decompress_bytes, block_item.decompress_bytes()); |
102 | 179k | _recvr->_memory_used_counter->update(-(int64_t)block_byte_size); |
103 | 179k | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
104 | 179k | sub_blocks_memory_usage(block_byte_size); |
105 | 179k | if (_block_queue.empty() && _source_dependency) { |
106 | 140k | if (!_is_cancelled && _num_remaining_senders > 0) { |
107 | 52.9k | _source_dependency->block(); |
108 | 52.9k | } |
109 | 140k | } |
110 | | |
111 | 179k | block_item.call_done(_recvr); |
112 | | |
113 | 179k | DCHECK(block->empty()); |
114 | 179k | block->swap(*next_block); |
115 | 179k | *eos = false; |
116 | 179k | return Status::OK(); |
117 | 179k | } |
118 | | |
119 | 1.37M | void VDataStreamRecvr::SenderQueue::set_source_ready(std::lock_guard<std::mutex>&) { |
120 | | // Here, it is necessary to check if _source_dependency is not nullptr. |
121 | | // This is because the queue might be closed before setting the source dependency. |
122 | 1.37M | if (!_source_dependency) { |
123 | 11 | return; |
124 | 11 | } |
125 | 1.37M | const bool should_wait = !_is_cancelled && _block_queue.empty() && _num_remaining_senders > 0; |
126 | 1.37M | if (!should_wait) { |
127 | 1.37M | _source_dependency->set_ready(); |
128 | 1.37M | } |
129 | 1.37M | } |
130 | | |
131 | 15 | std::string VDataStreamRecvr::SenderQueue::debug_string() { |
132 | 15 | fmt::memory_buffer debug_string_buffer; |
133 | 15 | fmt::format_to(debug_string_buffer, |
134 | 15 | "_num_remaining_senders = {}, block_queue size = {}, _is_cancelled: {}, " |
135 | 15 | "_cancel_status: {}, _sender_eos_set: (", |
136 | 15 | _num_remaining_senders, _block_queue.size(), _is_cancelled, |
137 | 15 | _cancel_status.to_string()); |
138 | 15 | std::lock_guard<std::mutex> l(_lock); |
139 | 15 | for (auto& i : _sender_eos_set) { |
140 | 0 | fmt::format_to(debug_string_buffer, "{}, ", i); |
141 | 0 | } |
142 | 15 | fmt::format_to(debug_string_buffer, ")"); |
143 | 15 | return fmt::to_string(debug_string_buffer); |
144 | 15 | } |
145 | | |
146 | | Status VDataStreamRecvr::SenderQueue::add_block(std::unique_ptr<PBlock> pblock, int be_number, |
147 | | int64_t packet_seq, |
148 | | ::google::protobuf::Closure** done, |
149 | | const int64_t wait_for_worker, |
150 | 493 | const uint64_t time_to_find_recvr) { |
151 | 493 | { |
152 | 493 | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
153 | 493 | if (_is_cancelled) { |
154 | 0 | return Status::OK(); |
155 | 0 | } |
156 | 493 | auto iter = _packet_seq_map.find(be_number); |
157 | 495 | if (iter != _packet_seq_map.end()) { |
158 | 495 | if (iter->second >= packet_seq) { |
159 | 0 | return Status::InternalError( |
160 | 0 | "packet already exist [cur_packet_id= {} receive_packet_id={}]", |
161 | 0 | iter->second, packet_seq); |
162 | 0 | } |
163 | 495 | iter->second = packet_seq; |
164 | 18.4E | } else { |
165 | 18.4E | _packet_seq_map.emplace(be_number, packet_seq); |
166 | 18.4E | } |
167 | | |
168 | 493 | DCHECK(_num_remaining_senders >= 0); |
169 | 493 | if (_num_remaining_senders == 0) { |
170 | 0 | DCHECK(_sender_eos_set.contains(be_number)); |
171 | 0 | return Status::OK(); |
172 | 0 | } |
173 | 493 | } |
174 | | |
175 | 493 | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
176 | 493 | if (_is_cancelled) { |
177 | 0 | return Status::OK(); |
178 | 0 | } |
179 | | |
180 | 493 | const auto block_byte_size = pblock->ByteSizeLong(); |
181 | 493 | COUNTER_UPDATE(_recvr->_blocks_produced_counter, 1); |
182 | 493 | if (_recvr->_max_wait_worker_time->value() < wait_for_worker) { |
183 | 0 | _recvr->_max_wait_worker_time->set(wait_for_worker); |
184 | 0 | } |
185 | | |
186 | 493 | if (_recvr->_max_find_recvr_time->value() < time_to_find_recvr) { |
187 | 0 | _recvr->_max_find_recvr_time->set((int64_t)time_to_find_recvr); |
188 | 0 | } |
189 | | |
190 | 493 | _block_queue.emplace_back(std::move(pblock), block_byte_size); |
191 | 493 | COUNTER_UPDATE(_recvr->_remote_bytes_received_counter, block_byte_size); |
192 | 493 | set_source_ready(l); |
193 | | |
194 | | // if done is nullptr, this function can't delay this response |
195 | 502 | if (done != nullptr && _recvr->exceeds_limit(block_byte_size)) { |
196 | 1 | _block_queue.back().set_done(*done); |
197 | 1 | *done = nullptr; |
198 | 1 | } |
199 | 493 | _recvr->_memory_used_counter->update(block_byte_size); |
200 | 493 | add_blocks_memory_usage(block_byte_size); |
201 | 493 | return Status::OK(); |
202 | 493 | } |
203 | | |
204 | | Status VDataStreamRecvr::SenderQueue::add_blocks(const PTransmitDataParams* request, |
205 | | ::google::protobuf::Closure** done, |
206 | | const int64_t wait_for_worker, |
207 | 85.3k | const uint64_t time_to_find_recvr) { |
208 | 85.3k | { |
209 | 85.3k | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
210 | 85.3k | if (_is_cancelled) { |
211 | 2 | return Status::OK(); |
212 | 2 | } |
213 | 85.3k | const int be_number = request->be_number(); |
214 | | // In the request, the packet_seq for blocks is [request->packet_seq() - blocks_size(), request->packet_seq()) |
215 | | // Note this is a left-closed, right-open interval; the packet_seq of the last block is request->packet_seq() - 1 |
216 | | // We store the packet_seq of the last block in _packet_seq_map so we can compare it with the packet_seq of the next received packet |
217 | 85.3k | const int64_t packet_seq = request->packet_seq() - 1; |
218 | 85.3k | auto iter = _packet_seq_map.find(be_number); |
219 | 85.3k | if (iter != _packet_seq_map.end()) { |
220 | 13.8k | if (iter->second > (packet_seq - request->blocks_size())) { |
221 | 0 | return Status::InternalError( |
222 | 0 | "packet already exist [cur_packet_id= {} receive_packet_id={}]", |
223 | 0 | iter->second, packet_seq); |
224 | 0 | } |
225 | 13.8k | iter->second = packet_seq; |
226 | 71.4k | } else { |
227 | 71.4k | _packet_seq_map.emplace(be_number, packet_seq); |
228 | 71.4k | } |
229 | | |
230 | 85.3k | DCHECK(_num_remaining_senders >= 0); |
231 | 85.3k | if (_num_remaining_senders == 0) { |
232 | 0 | DCHECK(_sender_eos_set.end() != _sender_eos_set.find(be_number)); |
233 | 0 | return Status::OK(); |
234 | 0 | } |
235 | 85.3k | } |
236 | | |
237 | 85.3k | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
238 | 85.3k | if (_is_cancelled) { |
239 | 0 | return Status::OK(); |
240 | 0 | } |
241 | | |
242 | 85.3k | int64_t total_block_byte_size = 0; |
243 | 170k | for (int i = 0; i < request->blocks_size(); i++) { |
244 | 85.3k | std::unique_ptr<PBlock> pblock = std::make_unique<PBlock>(); |
245 | 85.3k | pblock->CopyFrom(request->blocks(i)); |
246 | | |
247 | 85.3k | const auto block_byte_size = pblock->ByteSizeLong(); |
248 | 85.3k | COUNTER_UPDATE(_recvr->_blocks_produced_counter, 1); |
249 | 85.3k | if (_recvr->_max_wait_worker_time->value() < wait_for_worker) { |
250 | 1 | _recvr->_max_wait_worker_time->set(wait_for_worker); |
251 | 1 | } |
252 | | |
253 | 85.3k | if (_recvr->_max_find_recvr_time->value() < time_to_find_recvr) { |
254 | 62.1k | _recvr->_max_find_recvr_time->set((int64_t)time_to_find_recvr); |
255 | 62.1k | } |
256 | | |
257 | 85.3k | _block_queue.emplace_back(std::move(pblock), block_byte_size); |
258 | 85.3k | COUNTER_UPDATE(_recvr->_remote_bytes_received_counter, block_byte_size); |
259 | 85.3k | total_block_byte_size += block_byte_size; |
260 | 85.3k | } |
261 | | |
262 | 85.3k | set_source_ready(l); |
263 | | |
264 | | // if done is nullptr, this function can't delay this response |
265 | 85.3k | if (done != nullptr && _recvr->exceeds_limit(total_block_byte_size)) { |
266 | 3 | _block_queue.back().set_done(*done); |
267 | 3 | *done = nullptr; |
268 | 3 | } |
269 | 85.3k | _recvr->_memory_used_counter->update(total_block_byte_size); |
270 | 85.3k | add_blocks_memory_usage(total_block_byte_size); |
271 | 85.3k | return Status::OK(); |
272 | 85.3k | } |
273 | | |
274 | 96.0k | void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) { |
275 | 96.0k | if (block->rows() == 0) { |
276 | 0 | return; |
277 | 0 | } |
278 | 96.0k | { |
279 | 96.0k | INJECT_MOCK_SLEEP(std::unique_lock<std::mutex> l(_lock)); |
280 | 96.0k | if (_is_cancelled) { |
281 | 298 | return; |
282 | 298 | } |
283 | 96.0k | DCHECK(_num_remaining_senders >= 0); |
284 | 95.7k | if (_num_remaining_senders == 0) { |
285 | 1 | return; |
286 | 1 | } |
287 | 95.7k | } |
288 | 95.7k | BlockUPtr nblock = Block::create_unique(block->get_columns_with_type_and_name()); |
289 | | |
290 | | // local exchange should copy the block contented if use move == false |
291 | 95.7k | if (use_move) { |
292 | 87.8k | block->clear(); |
293 | 87.8k | } else { |
294 | 7.82k | auto rows = block->rows(); |
295 | 25.0k | for (int i = 0; i < nblock->columns(); ++i) { |
296 | 17.2k | nblock->get_by_position(i).column = |
297 | 17.2k | nblock->get_by_position(i).column->clone_resized(rows); |
298 | 17.2k | } |
299 | 7.82k | } |
300 | 95.7k | materialize_block_inplace(*nblock); |
301 | | |
302 | 95.7k | auto block_mem_size = nblock->allocated_bytes(); |
303 | 95.7k | { |
304 | 95.7k | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
305 | 95.7k | if (_is_cancelled) { |
306 | 1 | return; |
307 | 1 | } |
308 | 95.7k | _block_queue.emplace_back(std::move(nblock), block_mem_size); |
309 | 95.7k | set_source_ready(l); |
310 | 95.7k | COUNTER_UPDATE(_recvr->_local_bytes_received_counter, block_mem_size); |
311 | 95.7k | _recvr->_memory_used_counter->update(block_mem_size); |
312 | 95.7k | add_blocks_memory_usage(block_mem_size); |
313 | 95.7k | } |
314 | 95.7k | } |
315 | | |
316 | 5.28M | void VDataStreamRecvr::SenderQueue::decrement_senders(int be_number) { |
317 | 5.28M | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
318 | 5.28M | if (_sender_eos_set.end() != _sender_eos_set.find(be_number)) { |
319 | 0 | return; |
320 | 0 | } |
321 | 5.28M | _sender_eos_set.insert(be_number); |
322 | 5.28M | DCHECK_GT(_num_remaining_senders, 0); |
323 | 5.28M | _num_remaining_senders--; |
324 | 18.4E | VLOG_FILE << "decremented senders: fragment_instance_id=" |
325 | 18.4E | << print_id(_recvr->fragment_instance_id()) << " node_id=" << _recvr->dest_node_id() |
326 | 18.4E | << " #senders=" << _num_remaining_senders; |
327 | 5.28M | if (_num_remaining_senders == 0) { |
328 | 589k | set_source_ready(l); |
329 | 589k | } |
330 | 5.28M | } |
331 | | |
332 | 601k | void VDataStreamRecvr::SenderQueue::cancel(Status cancel_status) { |
333 | 601k | { |
334 | 601k | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
335 | 601k | if (_is_cancelled) { |
336 | 601k | return; |
337 | 601k | } |
338 | 18.4E | _is_cancelled = true; |
339 | 18.4E | _cancel_status = cancel_status; |
340 | 18.4E | set_source_ready(l); |
341 | 18.4E | VLOG_QUERY << "cancelled stream: _fragment_instance_id=" |
342 | 18.4E | << print_id(_recvr->fragment_instance_id()) |
343 | 18.4E | << " node_id=" << _recvr->dest_node_id(); |
344 | 18.4E | } |
345 | 0 | { |
346 | 18.4E | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
347 | 18.4E | for (auto& block_item : _block_queue) { |
348 | 0 | block_item.call_done(_recvr); |
349 | 0 | } |
350 | 18.4E | _block_queue.clear(); |
351 | 18.4E | } |
352 | 18.4E | } |
353 | | |
354 | 600k | void VDataStreamRecvr::SenderQueue::close() { |
355 | | // If _is_cancelled is not set to true, there may be concurrent send |
356 | | // which add batch to _block_queue. The batch added after _block_queue |
357 | | // is clear will be memory leak |
358 | 600k | INJECT_MOCK_SLEEP(std::lock_guard<std::mutex> l(_lock)); |
359 | 600k | _is_cancelled = true; |
360 | 600k | set_source_ready(l); |
361 | | |
362 | 600k | for (auto& block_item : _block_queue) { |
363 | 1.75k | block_item.call_done(_recvr); |
364 | 1.75k | } |
365 | | // Delete any batches queued in _block_queue |
366 | 600k | _block_queue.clear(); |
367 | 600k | } |
368 | | |
369 | | VDataStreamRecvr::VDataStreamRecvr(VDataStreamMgr* stream_mgr, |
370 | | RuntimeProfile::HighWaterMarkCounter* memory_used_counter, |
371 | | RuntimeState* state, const TUniqueId& fragment_instance_id, |
372 | | PlanNodeId dest_node_id, int num_senders, bool is_merging, |
373 | | RuntimeProfile* profile, size_t data_queue_capacity) |
374 | 433k | : HasTaskExecutionCtx(state), |
375 | 433k | _mgr(stream_mgr), |
376 | 433k | _memory_used_counter(memory_used_counter), |
377 | 433k | _resource_ctx(state->get_query_ctx()->resource_ctx()), |
378 | 433k | _query_context(state->get_query_ctx()->shared_from_this()), |
379 | 433k | _fragment_instance_id(fragment_instance_id), |
380 | 433k | _dest_node_id(dest_node_id), |
381 | 433k | _is_merging(is_merging), |
382 | 433k | _is_closed(false), |
383 | 433k | _sender_queue_mem_limit(data_queue_capacity), |
384 | 433k | _profile(profile) { |
385 | | // DataStreamRecvr may be destructed after the instance execution thread ends. |
386 | 433k | _mem_tracker = |
387 | 433k | std::make_unique<MemTracker>("VDataStreamRecvr:" + print_id(_fragment_instance_id)); |
388 | 433k | SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); |
389 | | |
390 | | // Create one queue per sender if is_merging is true. |
391 | 433k | int num_queues = is_merging ? num_senders : 1; |
392 | 433k | _sender_to_local_channel_dependency.resize(num_queues); |
393 | 1.03M | for (size_t i = 0; i < num_queues; i++) { |
394 | 597k | _sender_to_local_channel_dependency[i] = Dependency::create_shared( |
395 | 597k | _dest_node_id, _dest_node_id, fmt::format("LocalExchangeChannelDependency_{}", i), |
396 | 597k | true); |
397 | 597k | } |
398 | 433k | _sender_queues.reserve(num_queues); |
399 | 433k | int num_sender_per_queue = is_merging ? 1 : num_senders; |
400 | 1.03M | for (int i = 0; i < num_queues; ++i) { |
401 | 596k | SenderQueue* queue = nullptr; |
402 | 596k | queue = _sender_queue_pool.add(new SenderQueue(this, num_sender_per_queue, |
403 | 596k | _sender_to_local_channel_dependency[i])); |
404 | 596k | _sender_queues.push_back(queue); |
405 | 596k | } |
406 | | |
407 | | // Initialize the counters |
408 | 433k | _remote_bytes_received_counter = ADD_COUNTER(_profile, "RemoteBytesReceived", TUnit::BYTES); |
409 | 433k | _local_bytes_received_counter = ADD_COUNTER(_profile, "LocalBytesReceived", TUnit::BYTES); |
410 | | |
411 | 433k | _deserialize_row_batch_timer = ADD_TIMER(_profile, "DeserializeRowBatchTimer"); |
412 | 433k | _data_arrival_timer = ADD_TIMER(_profile, "DataArrivalWaitTime"); |
413 | 433k | _buffer_full_total_timer = ADD_TIMER(_profile, "SendersBlockedTotalTimer(*)"); |
414 | 433k | _first_batch_wait_total_timer = ADD_TIMER(_profile, "FirstBatchArrivalWaitTime"); |
415 | 433k | _decompress_timer = ADD_TIMER(_profile, "DecompressTime"); |
416 | 433k | _decompress_bytes = ADD_COUNTER(_profile, "DecompressBytes", TUnit::BYTES); |
417 | 433k | _blocks_produced_counter = ADD_COUNTER(_profile, "BlocksProduced", TUnit::UNIT); |
418 | 433k | _max_wait_worker_time = ADD_COUNTER(_profile, "MaxWaitForWorkerTime", TUnit::UNIT); |
419 | 433k | _max_wait_to_process_time = ADD_COUNTER(_profile, "MaxWaitToProcessTime", TUnit::UNIT); |
420 | 433k | _max_find_recvr_time = ADD_COUNTER(_profile, "MaxFindRecvrTime(NS)", TUnit::UNIT); |
421 | 433k | } |
422 | | |
423 | 436k | VDataStreamRecvr::~VDataStreamRecvr() { |
424 | 18.4E | DCHECK(_mgr == nullptr) << "Must call close()"; |
425 | 436k | } |
426 | | |
427 | | Status VDataStreamRecvr::create_merger(const VExprContextSPtrs& ordering_expr, |
428 | | const std::vector<bool>& is_asc_order, |
429 | | const std::vector<bool>& nulls_first, size_t batch_size, |
430 | 29.0k | int64_t limit, size_t offset) { |
431 | 29.0k | DCHECK(_is_merging); |
432 | 29.0k | SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); |
433 | 29.0k | std::vector<BlockSupplier> child_block_suppliers; |
434 | | // Create the merger that will a single stream of sorted rows. |
435 | 29.0k | _merger.reset(new VSortedRunMerger(ordering_expr, is_asc_order, nulls_first, batch_size, limit, |
436 | 29.0k | offset, _profile)); |
437 | | |
438 | 222k | for (int i = 0; i < _sender_queues.size(); ++i) { |
439 | 193k | child_block_suppliers.emplace_back(std::bind(std::mem_fn(&SenderQueue::get_batch), |
440 | 193k | _sender_queues[i], std::placeholders::_1, |
441 | 193k | std::placeholders::_2)); |
442 | 193k | } |
443 | 29.0k | RETURN_IF_ERROR(_merger->prepare(child_block_suppliers)); |
444 | 29.0k | return Status::OK(); |
445 | 29.0k | } |
446 | | |
447 | | Status VDataStreamRecvr::add_block(std::unique_ptr<PBlock> pblock, int sender_id, int be_number, |
448 | | int64_t packet_seq, ::google::protobuf::Closure** done, |
449 | | const int64_t wait_for_worker, |
450 | 0 | const uint64_t time_to_find_recvr) { |
451 | 0 | SCOPED_ATTACH_TASK(_resource_ctx); |
452 | 0 | if (_query_context->low_memory_mode()) { |
453 | 0 | set_low_memory_mode(); |
454 | 0 | } |
455 | |
|
456 | 0 | int use_sender_id = _is_merging ? sender_id : 0; |
457 | 0 | return _sender_queues[use_sender_id]->add_block(std::move(pblock), be_number, packet_seq, done, |
458 | 0 | wait_for_worker, time_to_find_recvr); |
459 | 0 | } |
460 | | |
461 | | Status VDataStreamRecvr::add_blocks(const PTransmitDataParams* request, |
462 | | ::google::protobuf::Closure** done, |
463 | | const int64_t wait_for_worker, |
464 | 85.3k | const uint64_t time_to_find_recvr) { |
465 | 85.3k | SCOPED_ATTACH_TASK(_resource_ctx); |
466 | 85.3k | if (_query_context->low_memory_mode()) { |
467 | 763 | set_low_memory_mode(); |
468 | 763 | } |
469 | 85.3k | int use_sender_id = _is_merging ? request->sender_id() : 0; |
470 | 85.3k | return _sender_queues[use_sender_id]->add_blocks(request, done, wait_for_worker, |
471 | 85.3k | time_to_find_recvr); |
472 | 85.3k | } |
473 | | |
474 | 95.3k | void VDataStreamRecvr::add_block(Block* block, int sender_id, bool use_move) { |
475 | 95.3k | if (_query_context->low_memory_mode()) { |
476 | 1.09k | set_low_memory_mode(); |
477 | 1.09k | } |
478 | 95.3k | int use_sender_id = _is_merging ? sender_id : 0; |
479 | 95.3k | _sender_queues[use_sender_id]->add_block(block, use_move); |
480 | 95.3k | } |
481 | | |
482 | 15 | std::string VDataStreamRecvr::debug_string() { |
483 | 15 | fmt::memory_buffer debug_string_buffer; |
484 | 15 | fmt::format_to(debug_string_buffer, |
485 | 15 | "fragment_instance_id: {}, _dest_node_id: {}, _is_merging: {}, _is_closed: {}", |
486 | 15 | print_id(_fragment_instance_id), _dest_node_id, _is_merging, _is_closed); |
487 | 30 | for (size_t i = 0; i < _sender_queues.size(); i++) { |
488 | 15 | fmt::format_to(debug_string_buffer, "No. {} queue: {}", i, |
489 | 15 | _sender_queues[i]->debug_string()); |
490 | 15 | } |
491 | 15 | return fmt::to_string(debug_string_buffer); |
492 | 15 | } |
493 | | |
494 | 1.72M | std::shared_ptr<Dependency> VDataStreamRecvr::get_local_channel_dependency(int sender_id) { |
495 | 1.72M | DCHECK(_sender_to_local_channel_dependency[_is_merging ? sender_id : 0] != nullptr); |
496 | 1.72M | return _sender_to_local_channel_dependency[_is_merging ? sender_id : 0]; |
497 | 1.72M | } |
498 | | |
499 | 603k | Status VDataStreamRecvr::get_next(Block* block, bool* eos) { |
500 | 603k | if (!_is_merging) { |
501 | 529k | block->clear(); |
502 | 529k | return _sender_queues[0]->get_batch(block, eos); |
503 | 529k | } else { |
504 | 73.4k | return _merger->get_next(block, eos); |
505 | 73.4k | } |
506 | 603k | } |
507 | | |
508 | 5.29M | void VDataStreamRecvr::remove_sender(int sender_id, int be_number, Status exec_status) { |
509 | 5.29M | if (!exec_status.ok()) { |
510 | 0 | cancel_stream(exec_status); |
511 | 0 | return; |
512 | 0 | } |
513 | 5.29M | int use_sender_id = _is_merging ? sender_id : 0; |
514 | 5.29M | _sender_queues[use_sender_id]->decrement_senders(be_number); |
515 | 5.29M | } |
516 | | |
517 | 436k | void VDataStreamRecvr::cancel_stream(Status exec_status) { |
518 | 18.4E | VLOG_QUERY << "cancel_stream: fragment_instance_id=" << print_id(_fragment_instance_id) |
519 | 18.4E | << exec_status; |
520 | | |
521 | 1.03M | for (int i = 0; i < _sender_queues.size(); ++i) { |
522 | 601k | _sender_queues[i]->cancel(exec_status); |
523 | 601k | } |
524 | 436k | } |
525 | | |
526 | 181k | void VDataStreamRecvr::SenderQueue::add_blocks_memory_usage(int64_t size) { |
527 | 181k | DCHECK(size >= 0); |
528 | 181k | _recvr->_mem_tracker->consume(size); |
529 | 181k | _queue_mem_tracker->consume(size); |
530 | 181k | if (_local_channel_dependency && exceeds_limit()) { |
531 | 915 | _local_channel_dependency->block(); |
532 | 915 | } |
533 | 181k | } |
534 | | |
535 | 179k | void VDataStreamRecvr::SenderQueue::sub_blocks_memory_usage(int64_t size) { |
536 | 179k | DCHECK(size >= 0); |
537 | 179k | _recvr->_mem_tracker->release(size); |
538 | 179k | _queue_mem_tracker->release(size); |
539 | 179k | if (_local_channel_dependency && (!exceeds_limit())) { |
540 | 179k | _local_channel_dependency->set_ready(); |
541 | 179k | } |
542 | 179k | } |
543 | | |
544 | 361k | bool VDataStreamRecvr::SenderQueue::exceeds_limit() { |
545 | 361k | const size_t queue_byte_size = _queue_mem_tracker->consumption(); |
546 | 361k | return _recvr->queue_exceeds_limit(queue_byte_size); |
547 | 361k | } |
548 | | |
549 | 85.8k | bool VDataStreamRecvr::exceeds_limit(size_t block_byte_size) { |
550 | 85.8k | return _mem_tracker->consumption() + block_byte_size > config::exchg_node_buffer_size_bytes; |
551 | 85.8k | } |
552 | | |
553 | 361k | bool VDataStreamRecvr::queue_exceeds_limit(size_t queue_byte_size) const { |
554 | 361k | return queue_byte_size >= _sender_queue_mem_limit; |
555 | 361k | } |
556 | | |
557 | 869k | void VDataStreamRecvr::close() { |
558 | 869k | if (_is_closed) { |
559 | 434k | return; |
560 | 434k | } |
561 | 435k | _is_closed = true; |
562 | 600k | for (auto& it : _sender_to_local_channel_dependency) { |
563 | 600k | it->set_always_ready(); |
564 | 600k | } |
565 | 1.03M | for (int i = 0; i < _sender_queues.size(); ++i) { |
566 | 601k | _sender_queues[i]->close(); |
567 | 601k | } |
568 | | // Remove this receiver from the DataStreamMgr that created it. |
569 | | // TODO: log error msg |
570 | 436k | if (_mgr) { |
571 | 436k | static_cast<void>(_mgr->deregister_recvr(fragment_instance_id(), dest_node_id())); |
572 | 436k | } |
573 | 435k | _mgr = nullptr; |
574 | | |
575 | 435k | _merger.reset(); |
576 | 435k | } |
577 | | |
578 | 425k | void VDataStreamRecvr::set_sink_dep_always_ready() const { |
579 | 590k | for (auto dep : _sender_to_local_channel_dependency) { |
580 | 590k | dep->set_always_ready(); |
581 | 590k | } |
582 | 425k | } |
583 | | |
584 | | } // namespace doris |