/root/doris/be/src/runtime/tablets_channel.cpp
Line | Count | Source (jump to first uncovered line) |
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 "runtime/tablets_channel.h" |
19 | | |
20 | | #include <bvar/bvar.h> |
21 | | #include <fmt/format.h> |
22 | | #include <gen_cpp/internal_service.pb.h> |
23 | | #include <gen_cpp/types.pb.h> |
24 | | |
25 | | #include <ctime> |
26 | | |
27 | | #include "common/compiler_util.h" // IWYU pragma: keep |
28 | | #include "common/status.h" |
29 | | // IWYU pragma: no_include <bits/chrono.h> |
30 | | #include <chrono> // IWYU pragma: keep |
31 | | #include <initializer_list> |
32 | | #include <set> |
33 | | #include <thread> |
34 | | #include <utility> |
35 | | |
36 | | #ifdef DEBUG |
37 | | #include <unordered_set> |
38 | | #endif |
39 | | |
40 | | #include "common/logging.h" |
41 | | #include "exec/tablet_info.h" |
42 | | #include "olap/delta_writer.h" |
43 | | #include "olap/storage_engine.h" |
44 | | #include "olap/txn_manager.h" |
45 | | #include "runtime/load_channel.h" |
46 | | #include "util/defer_op.h" |
47 | | #include "util/doris_metrics.h" |
48 | | #include "util/metrics.h" |
49 | | #include "vec/core/block.h" |
50 | | |
51 | | namespace doris { |
52 | | class SlotDescriptor; |
53 | | |
54 | | bvar::Adder<int64_t> g_tablets_channel_send_data_allocated_size( |
55 | | "tablets_channel_send_data_allocated_size"); |
56 | | |
57 | | DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(tablet_writer_count, MetricUnit::NOUNIT); |
58 | | |
59 | | std::atomic<uint64_t> BaseTabletsChannel::_s_tablet_writer_count; |
60 | | |
61 | | BaseTabletsChannel::BaseTabletsChannel(const TabletsChannelKey& key, const UniqueId& load_id, |
62 | | bool is_high_priority, RuntimeProfile* profile) |
63 | | : _key(key), |
64 | | _state(kInitialized), |
65 | | _load_id(load_id), |
66 | | _closed_senders(64), |
67 | 0 | _is_high_priority(is_high_priority) { |
68 | 0 | static std::once_flag once_flag; |
69 | 0 | _init_profile(profile); |
70 | 0 | std::call_once(once_flag, [] { |
71 | 0 | REGISTER_HOOK_METRIC(tablet_writer_count, [&]() { return _s_tablet_writer_count.load(); }); |
72 | 0 | }); |
73 | 0 | } |
74 | | |
75 | | TabletsChannel::TabletsChannel(StorageEngine& engine, const TabletsChannelKey& key, |
76 | | const UniqueId& load_id, bool is_high_priority, |
77 | | RuntimeProfile* profile) |
78 | 0 | : BaseTabletsChannel(key, load_id, is_high_priority, profile), _engine(engine) {} |
79 | | |
80 | 0 | BaseTabletsChannel::~BaseTabletsChannel() { |
81 | 0 | _s_tablet_writer_count -= _tablet_writers.size(); |
82 | 0 | } |
83 | | |
84 | 0 | TabletsChannel::~TabletsChannel() = default; |
85 | | |
86 | | Status BaseTabletsChannel::_get_current_seq(int64_t& cur_seq, |
87 | 0 | const PTabletWriterAddBlockRequest& request) { |
88 | 0 | std::lock_guard<std::mutex> l(_lock); |
89 | 0 | if (_state != kOpened) { |
90 | 0 | return _state == kFinished ? _close_status |
91 | 0 | : Status::InternalError("TabletsChannel {} state: {}", |
92 | 0 | _key.to_string(), _state); |
93 | 0 | } |
94 | 0 | cur_seq = _next_seqs[request.sender_id()]; |
95 | | // check packet |
96 | 0 | if (request.packet_seq() > cur_seq) { |
97 | 0 | LOG(WARNING) << "lost data packet, expect_seq=" << cur_seq |
98 | 0 | << ", recept_seq=" << request.packet_seq(); |
99 | 0 | return Status::InternalError("lost data packet"); |
100 | 0 | } |
101 | 0 | return Status::OK(); |
102 | 0 | } |
103 | | |
104 | 0 | void BaseTabletsChannel::_init_profile(RuntimeProfile* profile) { |
105 | 0 | _profile = |
106 | 0 | profile->create_child(fmt::format("TabletsChannel {}", _key.to_string()), true, true); |
107 | 0 | _add_batch_number_counter = ADD_COUNTER(_profile, "NumberBatchAdded", TUnit::UNIT); |
108 | |
|
109 | 0 | auto* memory_usage = _profile->create_child("PeakMemoryUsage", true, true); |
110 | 0 | _add_batch_timer = ADD_TIMER(_profile, "AddBatchTime"); |
111 | 0 | _write_block_timer = ADD_TIMER(_profile, "WriteBlockTime"); |
112 | 0 | _incremental_open_timer = ADD_TIMER(_profile, "IncrementalOpenTabletTime"); |
113 | 0 | _memory_usage_counter = memory_usage->AddHighWaterMarkCounter("Total", TUnit::BYTES); |
114 | 0 | _write_memory_usage_counter = memory_usage->AddHighWaterMarkCounter("Write", TUnit::BYTES); |
115 | 0 | _flush_memory_usage_counter = memory_usage->AddHighWaterMarkCounter("Flush", TUnit::BYTES); |
116 | 0 | _max_tablet_memory_usage_counter = |
117 | 0 | memory_usage->AddHighWaterMarkCounter("MaxTablet", TUnit::BYTES); |
118 | 0 | _max_tablet_write_memory_usage_counter = |
119 | 0 | memory_usage->AddHighWaterMarkCounter("MaxTabletWrite", TUnit::BYTES); |
120 | 0 | _max_tablet_flush_memory_usage_counter = |
121 | 0 | memory_usage->AddHighWaterMarkCounter("MaxTabletFlush", TUnit::BYTES); |
122 | 0 | } |
123 | | |
124 | 0 | void TabletsChannel::_init_profile(RuntimeProfile* profile) { |
125 | 0 | BaseTabletsChannel::_init_profile(profile); |
126 | 0 | _slave_replica_timer = ADD_TIMER(_profile, "SlaveReplicaTime"); |
127 | 0 | } |
128 | | |
129 | 0 | Status BaseTabletsChannel::open(const PTabletWriterOpenRequest& request) { |
130 | 0 | std::lock_guard<std::mutex> l(_lock); |
131 | | // if _state is kOpened, it's a normal case, already open by other sender |
132 | | // if _state is kFinished, already cancelled by other sender |
133 | 0 | if (_state == kOpened || _state == kFinished) { |
134 | 0 | return Status::OK(); |
135 | 0 | } |
136 | 0 | LOG(INFO) << fmt::format("open tablets channel {}, tablets num: {} timeout(s): {}", |
137 | 0 | _key.to_string(), request.tablets().size(), |
138 | 0 | request.load_channel_timeout_s()); |
139 | 0 | _txn_id = request.txn_id(); |
140 | 0 | _index_id = request.index_id(); |
141 | 0 | _schema = std::make_shared<OlapTableSchemaParam>(); |
142 | 0 | RETURN_IF_ERROR(_schema->init(request.schema())); |
143 | 0 | _tuple_desc = _schema->tuple_desc(); |
144 | |
|
145 | 0 | int max_sender = request.num_senders(); |
146 | | /* |
147 | | * a tablets channel in reciever is related to a bulk of VNodeChannel of sender. each instance one or none. |
148 | | * there are two possibilities: |
149 | | * 1. there's partitions originally broadcasted by FE. so all sender(instance) know it at start. and open() will be |
150 | | * called directly, not by incremental_open(). and after _state changes to kOpened. _open_by_incremental will never |
151 | | * be true. in this case, _num_remaining_senders will keep same with senders number. when all sender sent close rpc, |
152 | | * the tablets channel will close. and if for auto partition table, these channel's closing will hang on reciever and |
153 | | * return together to avoid close-then-incremental-open problem. |
154 | | * 2. this tablets channel is opened by incremental_open of sender's sink node. so only this sender will know this partition |
155 | | * (this TabletsChannel) at that time. and we are not sure how many sender will know in the end. it depends on data |
156 | | * distribution. in this situation open() is called by incremental_open() at first time. so _open_by_incremental is true. |
157 | | * then _num_remaining_senders will not be set here. but inc every time when incremental_open() called. so it's dynamic |
158 | | * and also need same number of senders' close to close. but will not hang. |
159 | | */ |
160 | 0 | if (_open_by_incremental) { |
161 | 0 | DCHECK(_num_remaining_senders == 0) << _num_remaining_senders; |
162 | 0 | } else { |
163 | 0 | _num_remaining_senders = max_sender; |
164 | 0 | } |
165 | 0 | LOG(INFO) << fmt::format( |
166 | 0 | "txn {}: TabletsChannel of index {} init senders {} with incremental {}", _txn_id, |
167 | 0 | _index_id, _num_remaining_senders, _open_by_incremental ? "on" : "off"); |
168 | | // just use max_sender no matter incremental or not cuz we dont know how many senders will open. |
169 | 0 | _next_seqs.resize(max_sender, 0); |
170 | 0 | _closed_senders.Reset(max_sender); |
171 | |
|
172 | 0 | RETURN_IF_ERROR(_open_all_writers(request)); |
173 | | |
174 | 0 | _state = kOpened; |
175 | 0 | return Status::OK(); |
176 | 0 | } |
177 | | |
178 | 0 | Status BaseTabletsChannel::incremental_open(const PTabletWriterOpenRequest& params) { |
179 | 0 | SCOPED_TIMER(_incremental_open_timer); |
180 | | |
181 | | // current node first opened by incremental open |
182 | 0 | if (_state == kInitialized) { |
183 | 0 | _open_by_incremental = true; |
184 | 0 | RETURN_IF_ERROR(open(params)); |
185 | 0 | } |
186 | | |
187 | 0 | std::lock_guard<std::mutex> l(_lock); |
188 | | |
189 | | // one sender may incremental_open many times. but only close one time. so dont count duplicately. |
190 | 0 | if (_open_by_incremental) { |
191 | 0 | if (params.has_sender_id() && !_recieved_senders.contains(params.sender_id())) { |
192 | 0 | _recieved_senders.insert(params.sender_id()); |
193 | 0 | _num_remaining_senders++; |
194 | 0 | } else if (!params.has_sender_id()) { // for compatible |
195 | 0 | _num_remaining_senders++; |
196 | 0 | } |
197 | 0 | VLOG_DEBUG << fmt::format("txn {}: TabletsChannel {} inc senders to {}", _txn_id, _index_id, |
198 | 0 | _num_remaining_senders); |
199 | 0 | } |
200 | |
|
201 | 0 | std::vector<SlotDescriptor*>* index_slots = nullptr; |
202 | 0 | int32_t schema_hash = 0; |
203 | 0 | for (const auto& index : _schema->indexes()) { |
204 | 0 | if (index->index_id == _index_id) { |
205 | 0 | index_slots = &index->slots; |
206 | 0 | schema_hash = index->schema_hash; |
207 | 0 | break; |
208 | 0 | } |
209 | 0 | } |
210 | 0 | if (index_slots == nullptr) { |
211 | 0 | return Status::InternalError("unknown index id, key={}", _key.to_string()); |
212 | 0 | } |
213 | | // update tablets |
214 | 0 | size_t incremental_tablet_num = 0; |
215 | 0 | std::stringstream ss; |
216 | 0 | ss << "LocalTabletsChannel txn_id: " << _txn_id << " load_id: " << print_id(params.id()) |
217 | 0 | << " incremental open delta writer: "; |
218 | | |
219 | | // every change will hold _lock. this find in under _lock too. so no need _tablet_writers_lock again. |
220 | 0 | for (const auto& tablet : params.tablets()) { |
221 | 0 | if (_tablet_writers.find(tablet.tablet_id()) != _tablet_writers.end()) { |
222 | 0 | continue; |
223 | 0 | } |
224 | 0 | incremental_tablet_num++; |
225 | |
|
226 | 0 | WriteRequest wrequest; |
227 | 0 | wrequest.index_id = params.index_id(); |
228 | 0 | wrequest.tablet_id = tablet.tablet_id(); |
229 | 0 | wrequest.schema_hash = schema_hash; |
230 | 0 | wrequest.txn_id = _txn_id; |
231 | 0 | wrequest.partition_id = tablet.partition_id(); |
232 | 0 | wrequest.load_id = params.id(); |
233 | 0 | wrequest.tuple_desc = _tuple_desc; |
234 | 0 | wrequest.slots = index_slots; |
235 | 0 | wrequest.is_high_priority = _is_high_priority; |
236 | 0 | wrequest.table_schema_param = _schema; |
237 | 0 | wrequest.txn_expiration = params.txn_expiration(); // Required by CLOUD. |
238 | 0 | wrequest.storage_vault_id = params.storage_vault_id(); |
239 | |
|
240 | 0 | auto delta_writer = create_delta_writer(wrequest); |
241 | 0 | { |
242 | | // here we modify _tablet_writers. so need lock. |
243 | 0 | std::lock_guard<SpinLock> l(_tablet_writers_lock); |
244 | 0 | _tablet_writers.emplace(tablet.tablet_id(), std::move(delta_writer)); |
245 | 0 | } |
246 | |
|
247 | 0 | ss << "[" << tablet.tablet_id() << "]"; |
248 | 0 | } |
249 | |
|
250 | 0 | _s_tablet_writer_count += incremental_tablet_num; |
251 | 0 | LOG(INFO) << ss.str(); |
252 | |
|
253 | 0 | _state = kOpened; |
254 | 0 | return Status::OK(); |
255 | 0 | } |
256 | | |
257 | 0 | std::unique_ptr<BaseDeltaWriter> TabletsChannel::create_delta_writer(const WriteRequest& request) { |
258 | 0 | return std::make_unique<DeltaWriter>(_engine, request, _profile, _load_id); |
259 | 0 | } |
260 | | |
261 | | Status TabletsChannel::close(LoadChannel* parent, const PTabletWriterAddBlockRequest& req, |
262 | 0 | PTabletWriterAddBlockResult* res, bool* finished) { |
263 | 0 | int sender_id = req.sender_id(); |
264 | 0 | int64_t backend_id = req.backend_id(); |
265 | 0 | const auto& partition_ids = req.partition_ids(); |
266 | 0 | auto* tablet_errors = res->mutable_tablet_errors(); |
267 | 0 | std::lock_guard<std::mutex> l(_lock); |
268 | 0 | if (_state == kFinished) { |
269 | 0 | return _close_status; |
270 | 0 | } |
271 | 0 | if (_closed_senders.Get(sender_id)) { |
272 | | // Double close from one sender, just return OK |
273 | 0 | *finished = (_num_remaining_senders == 0); |
274 | 0 | return _close_status; |
275 | 0 | } |
276 | | |
277 | 0 | for (auto pid : partition_ids) { |
278 | 0 | _partition_ids.emplace(pid); |
279 | 0 | } |
280 | 0 | _closed_senders.Set(sender_id, true); |
281 | 0 | _num_remaining_senders--; |
282 | 0 | *finished = (_num_remaining_senders == 0); |
283 | |
|
284 | 0 | LOG(INFO) << fmt::format( |
285 | 0 | "txn {}: close tablets channel of index {} , sender id: {}, backend {}, remain " |
286 | 0 | "senders: {}", |
287 | 0 | _txn_id, _index_id, sender_id, backend_id, _num_remaining_senders); |
288 | |
|
289 | 0 | if (!*finished) { |
290 | 0 | return Status::OK(); |
291 | 0 | } |
292 | | |
293 | 0 | _state = kFinished; |
294 | | // All senders are closed |
295 | | // 1. close all delta writers |
296 | 0 | std::set<DeltaWriter*> need_wait_writers; |
297 | | // under _lock. no need _tablet_writers_lock again. |
298 | 0 | for (auto&& [tablet_id, writer] : _tablet_writers) { |
299 | 0 | if (_partition_ids.contains(writer->partition_id())) { |
300 | 0 | auto st = writer->close(); |
301 | 0 | if (!st.ok()) { |
302 | 0 | auto err_msg = fmt::format( |
303 | 0 | "close tablet writer failed, tablet_id={}, " |
304 | 0 | "transaction_id={}, err={}", |
305 | 0 | tablet_id, _txn_id, st.to_string()); |
306 | 0 | LOG(WARNING) << err_msg; |
307 | 0 | PTabletError* tablet_error = tablet_errors->Add(); |
308 | 0 | tablet_error->set_tablet_id(tablet_id); |
309 | 0 | tablet_error->set_msg(st.to_string()); |
310 | | // just skip this tablet(writer) and continue to close others |
311 | 0 | continue; |
312 | 0 | } |
313 | | // tablet writer in `_broken_tablets` should not call `build_rowset` and |
314 | | // `commit_txn` method, after that, the publish-version task will success, |
315 | | // which can cause the replica inconsistency. |
316 | 0 | if (_is_broken_tablet(writer->tablet_id())) { |
317 | 0 | LOG(WARNING) << "SHOULD NOT HAPPEN, tablet writer is broken but not cancelled" |
318 | 0 | << ", tablet_id=" << tablet_id << ", transaction_id=" << _txn_id; |
319 | 0 | continue; |
320 | 0 | } |
321 | 0 | need_wait_writers.insert(static_cast<DeltaWriter*>(writer.get())); |
322 | 0 | } else { |
323 | 0 | auto st = writer->cancel(); |
324 | 0 | if (!st.ok()) { |
325 | 0 | LOG(WARNING) << "cancel tablet writer failed, tablet_id=" << tablet_id |
326 | 0 | << ", transaction_id=" << _txn_id; |
327 | | // just skip this tablet(writer) and continue to close others |
328 | 0 | continue; |
329 | 0 | } |
330 | 0 | VLOG_PROGRESS << "cancel tablet writer successfully, tablet_id=" << tablet_id |
331 | 0 | << ", transaction_id=" << _txn_id; |
332 | 0 | } |
333 | 0 | } |
334 | |
|
335 | 0 | _write_single_replica = req.write_single_replica(); |
336 | | |
337 | | // 2. wait all writer finished flush. |
338 | 0 | for (auto* writer : need_wait_writers) { |
339 | 0 | RETURN_IF_ERROR((writer->wait_flush())); |
340 | 0 | } |
341 | | |
342 | | // 3. build rowset |
343 | 0 | for (auto it = need_wait_writers.begin(); it != need_wait_writers.end();) { |
344 | 0 | Status st = (*it)->build_rowset(); |
345 | 0 | if (!st.ok()) { |
346 | 0 | _add_error_tablet(tablet_errors, (*it)->tablet_id(), st); |
347 | 0 | it = need_wait_writers.erase(it); |
348 | 0 | continue; |
349 | 0 | } |
350 | | // 3.1 calculate delete bitmap for Unique Key MoW tables |
351 | 0 | st = (*it)->submit_calc_delete_bitmap_task(); |
352 | 0 | if (!st.ok()) { |
353 | 0 | _add_error_tablet(tablet_errors, (*it)->tablet_id(), st); |
354 | 0 | it = need_wait_writers.erase(it); |
355 | 0 | continue; |
356 | 0 | } |
357 | 0 | it++; |
358 | 0 | } |
359 | | |
360 | | // 4. wait for delete bitmap calculation complete if necessary |
361 | 0 | for (auto it = need_wait_writers.begin(); it != need_wait_writers.end();) { |
362 | 0 | Status st = (*it)->wait_calc_delete_bitmap(); |
363 | 0 | if (!st.ok()) { |
364 | 0 | _add_error_tablet(tablet_errors, (*it)->tablet_id(), st); |
365 | 0 | it = need_wait_writers.erase(it); |
366 | 0 | continue; |
367 | 0 | } |
368 | 0 | it++; |
369 | 0 | } |
370 | | |
371 | | // 5. commit all writers |
372 | |
|
373 | 0 | for (auto* writer : need_wait_writers) { |
374 | | // close may return failed, but no need to handle it here. |
375 | | // tablet_vec will only contains success tablet, and then let FE judge it. |
376 | 0 | _commit_txn(writer, req, res); |
377 | 0 | } |
378 | |
|
379 | 0 | if (_write_single_replica) { |
380 | 0 | auto* success_slave_tablet_node_ids = res->mutable_success_slave_tablet_node_ids(); |
381 | | // The operation waiting for all slave replicas to complete must end before the timeout, |
382 | | // so that there is enough time to collect completed replica. Otherwise, the task may |
383 | | // timeout and fail even though most of the replicas are completed. Here we set 0.9 |
384 | | // times the timeout as the maximum waiting time. |
385 | 0 | SCOPED_TIMER(_slave_replica_timer); |
386 | 0 | while (!need_wait_writers.empty() && |
387 | 0 | (time(nullptr) - parent->last_updated_time()) < (parent->timeout() * 0.9)) { |
388 | 0 | std::set<DeltaWriter*>::iterator it; |
389 | 0 | for (it = need_wait_writers.begin(); it != need_wait_writers.end();) { |
390 | 0 | bool is_done = (*it)->check_slave_replicas_done(success_slave_tablet_node_ids); |
391 | 0 | if (is_done) { |
392 | 0 | need_wait_writers.erase(it++); |
393 | 0 | } else { |
394 | 0 | it++; |
395 | 0 | } |
396 | 0 | } |
397 | 0 | std::this_thread::sleep_for(std::chrono::milliseconds(100)); |
398 | 0 | } |
399 | 0 | for (auto* writer : need_wait_writers) { |
400 | 0 | writer->add_finished_slave_replicas(success_slave_tablet_node_ids); |
401 | 0 | } |
402 | 0 | _engine.txn_manager()->clear_txn_tablet_delta_writer(_txn_id); |
403 | 0 | } |
404 | |
|
405 | 0 | return Status::OK(); |
406 | 0 | } |
407 | | |
408 | | void TabletsChannel::_commit_txn(DeltaWriter* writer, const PTabletWriterAddBlockRequest& req, |
409 | 0 | PTabletWriterAddBlockResult* res) { |
410 | 0 | PSlaveTabletNodes slave_nodes; |
411 | 0 | if (_write_single_replica) { |
412 | 0 | auto& nodes_map = req.slave_tablet_nodes(); |
413 | 0 | auto it = nodes_map.find(writer->tablet_id()); |
414 | 0 | if (it != nodes_map.end()) { |
415 | 0 | slave_nodes = it->second; |
416 | 0 | } |
417 | 0 | } |
418 | 0 | Status st = writer->commit_txn(slave_nodes); |
419 | 0 | if (st.ok()) [[likely]] { |
420 | 0 | auto* tablet_vec = res->mutable_tablet_vec(); |
421 | 0 | PTabletInfo* tablet_info = tablet_vec->Add(); |
422 | 0 | tablet_info->set_tablet_id(writer->tablet_id()); |
423 | | // unused required field. |
424 | 0 | tablet_info->set_schema_hash(0); |
425 | 0 | tablet_info->set_received_rows(writer->total_received_rows()); |
426 | 0 | tablet_info->set_num_rows_filtered(writer->num_rows_filtered()); |
427 | 0 | _total_received_rows += writer->total_received_rows(); |
428 | 0 | _num_rows_filtered += writer->num_rows_filtered(); |
429 | 0 | } else { |
430 | 0 | _add_error_tablet(res->mutable_tablet_errors(), writer->tablet_id(), st); |
431 | 0 | } |
432 | 0 | } |
433 | | |
434 | | void BaseTabletsChannel::_add_error_tablet( |
435 | | google::protobuf::RepeatedPtrField<PTabletError>* tablet_errors, int64_t tablet_id, |
436 | 0 | Status error) const { |
437 | 0 | PTabletError* tablet_error = tablet_errors->Add(); |
438 | 0 | tablet_error->set_tablet_id(tablet_id); |
439 | 0 | tablet_error->set_msg(error.to_string()); |
440 | 0 | VLOG_PROGRESS << "close wait failed tablet " << tablet_id << " transaction_id " << _txn_id |
441 | 0 | << "err msg " << error; |
442 | 0 | } |
443 | | |
444 | 0 | void BaseTabletsChannel::refresh_profile() { |
445 | 0 | int64_t write_mem_usage = 0; |
446 | 0 | int64_t flush_mem_usage = 0; |
447 | 0 | int64_t max_tablet_mem_usage = 0; |
448 | 0 | int64_t max_tablet_write_mem_usage = 0; |
449 | 0 | int64_t max_tablet_flush_mem_usage = 0; |
450 | 0 | { |
451 | 0 | std::lock_guard<SpinLock> l(_tablet_writers_lock); |
452 | 0 | for (auto&& [tablet_id, writer] : _tablet_writers) { |
453 | 0 | int64_t write_mem = writer->mem_consumption(MemType::WRITE_FINISHED); |
454 | 0 | write_mem_usage += write_mem; |
455 | 0 | int64_t flush_mem = writer->mem_consumption(MemType::FLUSH); |
456 | 0 | flush_mem_usage += flush_mem; |
457 | 0 | if (write_mem > max_tablet_write_mem_usage) { |
458 | 0 | max_tablet_write_mem_usage = write_mem; |
459 | 0 | } |
460 | 0 | if (flush_mem > max_tablet_flush_mem_usage) { |
461 | 0 | max_tablet_flush_mem_usage = flush_mem; |
462 | 0 | } |
463 | 0 | if (write_mem + flush_mem > max_tablet_mem_usage) { |
464 | 0 | max_tablet_mem_usage = write_mem + flush_mem; |
465 | 0 | } |
466 | 0 | } |
467 | 0 | } |
468 | 0 | COUNTER_SET(_memory_usage_counter, write_mem_usage + flush_mem_usage); |
469 | 0 | COUNTER_SET(_write_memory_usage_counter, write_mem_usage); |
470 | 0 | COUNTER_SET(_flush_memory_usage_counter, flush_mem_usage); |
471 | 0 | COUNTER_SET(_max_tablet_memory_usage_counter, max_tablet_mem_usage); |
472 | 0 | COUNTER_SET(_max_tablet_write_memory_usage_counter, max_tablet_write_mem_usage); |
473 | 0 | COUNTER_SET(_max_tablet_flush_memory_usage_counter, max_tablet_flush_mem_usage); |
474 | 0 | } |
475 | | |
476 | 0 | Status BaseTabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& request) { |
477 | 0 | std::vector<SlotDescriptor*>* index_slots = nullptr; |
478 | 0 | int32_t schema_hash = 0; |
479 | 0 | for (const auto& index : _schema->indexes()) { |
480 | 0 | if (index->index_id == _index_id) { |
481 | 0 | index_slots = &index->slots; |
482 | 0 | schema_hash = index->schema_hash; |
483 | 0 | break; |
484 | 0 | } |
485 | 0 | } |
486 | 0 | if (index_slots == nullptr) { |
487 | 0 | return Status::InternalError("unknown index id, key={}", _key.to_string()); |
488 | 0 | } |
489 | | |
490 | | #ifdef DEBUG |
491 | | // check: tablet ids should be unique |
492 | | { |
493 | | std::unordered_set<int64_t> tablet_ids; |
494 | | for (const auto& tablet : request.tablets()) { |
495 | | CHECK(tablet_ids.count(tablet.tablet_id()) == 0) |
496 | | << "found duplicate tablet id: " << tablet.tablet_id(); |
497 | | tablet_ids.insert(tablet.tablet_id()); |
498 | | } |
499 | | } |
500 | | #endif |
501 | | |
502 | 0 | int tablet_cnt = 0; |
503 | | // under _lock. no need _tablet_writers_lock again. |
504 | 0 | for (const auto& tablet : request.tablets()) { |
505 | 0 | if (_tablet_writers.find(tablet.tablet_id()) != _tablet_writers.end()) { |
506 | 0 | continue; |
507 | 0 | } |
508 | 0 | tablet_cnt++; |
509 | 0 | WriteRequest wrequest { |
510 | 0 | .tablet_id = tablet.tablet_id(), |
511 | 0 | .schema_hash = schema_hash, |
512 | 0 | .txn_id = _txn_id, |
513 | 0 | .txn_expiration = request.txn_expiration(), // Required by CLOUD. |
514 | 0 | .index_id = request.index_id(), |
515 | 0 | .partition_id = tablet.partition_id(), |
516 | 0 | .load_id = request.id(), |
517 | 0 | .tuple_desc = _tuple_desc, |
518 | 0 | .slots = index_slots, |
519 | 0 | .table_schema_param = _schema, |
520 | 0 | .is_high_priority = _is_high_priority, |
521 | 0 | .write_file_cache = request.write_file_cache(), |
522 | 0 | .storage_vault_id = request.storage_vault_id(), |
523 | 0 | }; |
524 | |
|
525 | 0 | auto delta_writer = create_delta_writer(wrequest); |
526 | 0 | { |
527 | 0 | std::lock_guard<SpinLock> l(_tablet_writers_lock); |
528 | 0 | _tablet_writers.emplace(tablet.tablet_id(), std::move(delta_writer)); |
529 | 0 | } |
530 | 0 | } |
531 | 0 | _s_tablet_writer_count += _tablet_writers.size(); |
532 | 0 | DCHECK_EQ(_tablet_writers.size(), tablet_cnt); |
533 | 0 | return Status::OK(); |
534 | 0 | } |
535 | | |
536 | 0 | Status BaseTabletsChannel::cancel() { |
537 | 0 | std::lock_guard<std::mutex> l(_lock); |
538 | 0 | if (_state == kFinished) { |
539 | 0 | return _close_status; |
540 | 0 | } |
541 | 0 | for (auto& it : _tablet_writers) { |
542 | 0 | static_cast<void>(it.second->cancel()); |
543 | 0 | } |
544 | 0 | _state = kFinished; |
545 | |
|
546 | 0 | return Status::OK(); |
547 | 0 | } |
548 | | |
549 | 0 | Status TabletsChannel::cancel() { |
550 | 0 | RETURN_IF_ERROR(BaseTabletsChannel::cancel()); |
551 | 0 | if (_write_single_replica) { |
552 | 0 | _engine.txn_manager()->clear_txn_tablet_delta_writer(_txn_id); |
553 | 0 | } |
554 | 0 | return Status::OK(); |
555 | 0 | } |
556 | | |
557 | 0 | std::string TabletsChannelKey::to_string() const { |
558 | 0 | std::stringstream ss; |
559 | 0 | ss << *this; |
560 | 0 | return ss.str(); |
561 | 0 | } |
562 | | |
563 | 0 | std::ostream& operator<<(std::ostream& os, const TabletsChannelKey& key) { |
564 | 0 | os << "(load_id=" << key.id << ", index_id=" << key.index_id << ")"; |
565 | 0 | return os; |
566 | 0 | } |
567 | | |
568 | | Status BaseTabletsChannel::_write_block_data( |
569 | | const PTabletWriterAddBlockRequest& request, int64_t cur_seq, |
570 | | std::unordered_map<int64_t, DorisVector<uint32_t>>& tablet_to_rowidxs, |
571 | 0 | PTabletWriterAddBlockResult* response) { |
572 | 0 | vectorized::Block send_data; |
573 | 0 | RETURN_IF_ERROR(send_data.deserialize(request.block())); |
574 | 0 | CHECK(send_data.rows() == request.tablet_ids_size()) |
575 | 0 | << "block rows: " << send_data.rows() |
576 | 0 | << ", tablet_ids_size: " << request.tablet_ids_size(); |
577 | |
|
578 | 0 | g_tablets_channel_send_data_allocated_size << send_data.allocated_bytes(); |
579 | 0 | Defer defer { |
580 | 0 | [&]() { g_tablets_channel_send_data_allocated_size << -send_data.allocated_bytes(); }}; |
581 | |
|
582 | 0 | auto write_tablet_data = [&](int64_t tablet_id, |
583 | 0 | std::function<Status(BaseDeltaWriter * writer)> write_func) { |
584 | 0 | google::protobuf::RepeatedPtrField<PTabletError>* tablet_errors = |
585 | 0 | response->mutable_tablet_errors(); |
586 | | |
587 | | // add_batch may concurrency with inc_open but not under _lock. |
588 | | // so need to protect it with _tablet_writers_lock. |
589 | 0 | decltype(_tablet_writers.find(tablet_id)) tablet_writer_it; |
590 | 0 | { |
591 | 0 | std::lock_guard<SpinLock> l(_tablet_writers_lock); |
592 | 0 | tablet_writer_it = _tablet_writers.find(tablet_id); |
593 | 0 | if (tablet_writer_it == _tablet_writers.end()) { |
594 | 0 | return Status::InternalError("unknown tablet to append data, tablet={}", tablet_id); |
595 | 0 | } |
596 | 0 | } |
597 | | |
598 | 0 | Status st = write_func(tablet_writer_it->second.get()); |
599 | 0 | if (!st.ok()) { |
600 | 0 | auto err_msg = |
601 | 0 | fmt::format("tablet writer write failed, tablet_id={}, txn_id={}, err={}", |
602 | 0 | tablet_id, _txn_id, st.to_string()); |
603 | 0 | LOG(WARNING) << err_msg; |
604 | 0 | PTabletError* error = tablet_errors->Add(); |
605 | 0 | error->set_tablet_id(tablet_id); |
606 | 0 | error->set_msg(err_msg); |
607 | 0 | static_cast<void>(tablet_writer_it->second->cancel_with_status(st)); |
608 | 0 | _add_broken_tablet(tablet_id); |
609 | | // continue write to other tablet. |
610 | | // the error will return back to sender. |
611 | 0 | } |
612 | 0 | return Status::OK(); |
613 | 0 | }; |
614 | |
|
615 | 0 | SCOPED_TIMER(_write_block_timer); |
616 | 0 | for (const auto& tablet_to_rowidxs_it : tablet_to_rowidxs) { |
617 | 0 | RETURN_IF_ERROR(write_tablet_data(tablet_to_rowidxs_it.first, [&](BaseDeltaWriter* writer) { |
618 | 0 | return writer->write(&send_data, tablet_to_rowidxs_it.second); |
619 | 0 | })); |
620 | 0 | } |
621 | | |
622 | 0 | { |
623 | 0 | std::lock_guard<std::mutex> l(_lock); |
624 | 0 | _next_seqs[request.sender_id()] = cur_seq + 1; |
625 | 0 | } |
626 | 0 | return Status::OK(); |
627 | 0 | } |
628 | | |
629 | | Status TabletsChannel::add_batch(const PTabletWriterAddBlockRequest& request, |
630 | 0 | PTabletWriterAddBlockResult* response) { |
631 | 0 | SCOPED_TIMER(_add_batch_timer); |
632 | 0 | int64_t cur_seq = 0; |
633 | 0 | _add_batch_number_counter->update(1); |
634 | |
|
635 | 0 | auto status = _get_current_seq(cur_seq, request); |
636 | 0 | if (UNLIKELY(!status.ok())) { |
637 | 0 | return status; |
638 | 0 | } |
639 | | |
640 | 0 | if (request.packet_seq() < cur_seq) { |
641 | 0 | LOG(INFO) << "packet has already recept before, expect_seq=" << cur_seq |
642 | 0 | << ", recept_seq=" << request.packet_seq(); |
643 | 0 | return Status::OK(); |
644 | 0 | } |
645 | | |
646 | 0 | std::unordered_map<int64_t /* tablet_id */, DorisVector<uint32_t> /* row index */> |
647 | 0 | tablet_to_rowidxs; |
648 | 0 | _build_tablet_to_rowidxs(request, &tablet_to_rowidxs); |
649 | |
|
650 | 0 | return _write_block_data(request, cur_seq, tablet_to_rowidxs, response); |
651 | 0 | } |
652 | | |
653 | 0 | void BaseTabletsChannel::_add_broken_tablet(int64_t tablet_id) { |
654 | 0 | std::unique_lock<std::shared_mutex> wlock(_broken_tablets_lock); |
655 | 0 | _broken_tablets.insert(tablet_id); |
656 | 0 | } |
657 | | |
658 | 0 | bool BaseTabletsChannel::_is_broken_tablet(int64_t tablet_id) const { |
659 | 0 | return _broken_tablets.find(tablet_id) != _broken_tablets.end(); |
660 | 0 | } |
661 | | |
662 | | void BaseTabletsChannel::_build_tablet_to_rowidxs( |
663 | | const PTabletWriterAddBlockRequest& request, |
664 | 0 | std::unordered_map<int64_t, DorisVector<uint32_t>>* tablet_to_rowidxs) { |
665 | | // just add a coarse-grained read lock here rather than each time when visiting _broken_tablets |
666 | | // tests show that a relatively coarse-grained read lock here performs better under multicore scenario |
667 | | // see: https://github.com/apache/doris/pull/28552 |
668 | 0 | std::shared_lock<std::shared_mutex> rlock(_broken_tablets_lock); |
669 | 0 | if (request.is_single_tablet_block()) { |
670 | | // The cloud mode need the tablet ids to prepare rowsets. |
671 | 0 | int64_t tablet_id = request.tablet_ids(0); |
672 | 0 | tablet_to_rowidxs->emplace(tablet_id, std::initializer_list<uint32_t> {0}); |
673 | 0 | return; |
674 | 0 | } |
675 | 0 | for (uint32_t i = 0; i < request.tablet_ids_size(); ++i) { |
676 | 0 | int64_t tablet_id = request.tablet_ids(i); |
677 | 0 | if (_is_broken_tablet(tablet_id)) { |
678 | | // skip broken tablets |
679 | 0 | VLOG_PROGRESS << "skip broken tablet tablet=" << tablet_id; |
680 | 0 | continue; |
681 | 0 | } |
682 | 0 | auto it = tablet_to_rowidxs->find(tablet_id); |
683 | 0 | if (it == tablet_to_rowidxs->end()) { |
684 | 0 | tablet_to_rowidxs->emplace(tablet_id, std::initializer_list<uint32_t> {i}); |
685 | 0 | } else { |
686 | 0 | it->second.emplace_back(i); |
687 | 0 | } |
688 | 0 | } |
689 | 0 | } |
690 | | |
691 | | } // namespace doris |