Coverage Report

Created: 2026-01-30 19:04

next uncovered line (L), next uncovered region (R), next uncovered branch (B)
/root/doris/be/src/runtime/load_stream.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 "runtime/load_stream.h"
19
20
#include <brpc/stream.h>
21
#include <bthread/bthread.h>
22
#include <bthread/condition_variable.h>
23
#include <bthread/mutex.h>
24
#include <olap/rowset/rowset_factory.h>
25
#include <olap/rowset/rowset_meta.h>
26
#include <olap/storage_engine.h>
27
#include <olap/tablet_manager.h>
28
#include <runtime/exec_env.h>
29
30
#include <memory>
31
#include <sstream>
32
33
#include "bvar/bvar.h"
34
#include "cloud/config.h"
35
#include "common/signal_handler.h"
36
#include "exec/tablet_info.h"
37
#include "olap/delta_writer.h"
38
#include "olap/tablet.h"
39
#include "olap/tablet_fwd.h"
40
#include "olap/tablet_schema.h"
41
#include "runtime/exec_env.h"
42
#include "runtime/fragment_mgr.h"
43
#include "runtime/load_channel.h"
44
#include "runtime/load_stream_mgr.h"
45
#include "runtime/load_stream_writer.h"
46
#include "runtime/workload_group/workload_group_manager.h"
47
#include "util/debug_points.h"
48
#include "util/runtime_profile.h"
49
#include "util/thrift_util.h"
50
#include "util/uid_util.h"
51
52
#define UNKNOWN_ID_FOR_TEST 0x7c00
53
54
namespace doris {
55
#include "common/compile_check_begin.h"
56
57
bvar::Adder<int64_t> g_load_stream_cnt("load_stream_count");
58
bvar::LatencyRecorder g_load_stream_flush_wait_ms("load_stream_flush_wait_ms");
59
bvar::Adder<int> g_load_stream_flush_running_threads("load_stream_flush_wait_threads");
60
61
TabletStream::TabletStream(const PUniqueId& load_id, int64_t id, int64_t txn_id,
62
                           LoadStreamMgr* load_stream_mgr, RuntimeProfile* profile)
63
14
        : _id(id),
64
14
          _next_segid(0),
65
14
          _load_id(load_id),
66
14
          _txn_id(txn_id),
67
14
          _load_stream_mgr(load_stream_mgr) {
68
14
    load_stream_mgr->create_token(_flush_token);
69
14
    _profile = profile->create_child(fmt::format("TabletStream {}", id), true, true);
70
14
    _append_data_timer = ADD_TIMER(_profile, "AppendDataTime");
71
14
    _add_segment_timer = ADD_TIMER(_profile, "AddSegmentTime");
72
14
    _close_wait_timer = ADD_TIMER(_profile, "CloseWaitTime");
73
14
}
74
75
10
inline std::ostream& operator<<(std::ostream& ostr, const TabletStream& tablet_stream) {
76
10
    ostr << "load_id=" << print_id(tablet_stream._load_id) << ", txn_id=" << tablet_stream._txn_id
77
10
         << ", tablet_id=" << tablet_stream._id << ", status=" << tablet_stream._status.status();
78
10
    return ostr;
79
10
}
80
81
Status TabletStream::init(std::shared_ptr<OlapTableSchemaParam> schema, int64_t index_id,
82
14
                          int64_t partition_id) {
83
14
    WriteRequest req {
84
14
            .tablet_id = _id,
85
14
            .txn_id = _txn_id,
86
14
            .index_id = index_id,
87
14
            .partition_id = partition_id,
88
14
            .load_id = _load_id,
89
14
            .table_schema_param = schema,
90
            // TODO(plat1ko): write_file_cache
91
14
            .storage_vault_id {},
92
14
    };
93
94
14
    _load_stream_writer = std::make_shared<LoadStreamWriter>(&req, _profile);
95
14
    DBUG_EXECUTE_IF("TabletStream.init.uninited_writer", {
96
14
        _status.update(Status::Uninitialized("fault injection"));
97
14
        return _status.status();
98
14
    });
99
14
    _status.update(_load_stream_writer->init());
100
14
    if (!_status.ok()) {
101
1
        LOG(INFO) << "failed to init rowset builder due to " << *this;
102
1
    }
103
14
    return _status.status();
104
14
}
105
106
27
Status TabletStream::append_data(const PStreamHeader& header, butil::IOBuf* data) {
107
27
    if (!_status.ok()) {
108
1
        return _status.status();
109
1
    }
110
111
    // dispatch add_segment request
112
26
    if (header.opcode() == PStreamHeader::ADD_SEGMENT) {
113
0
        return add_segment(header, data);
114
0
    }
115
116
26
    SCOPED_TIMER(_append_data_timer);
117
118
26
    int64_t src_id = header.src_id();
119
26
    uint32_t segid = header.segment_id();
120
    // Ensure there are enough space and mapping are built.
121
26
    SegIdMapping* mapping = nullptr;
122
26
    {
123
26
        std::lock_guard lock_guard(_lock);
124
26
        if (!_segids_mapping.contains(src_id)) {
125
14
            _segids_mapping[src_id] = std::make_unique<SegIdMapping>();
126
14
        }
127
26
        mapping = _segids_mapping[src_id].get();
128
26
    }
129
26
    if (segid + 1 > mapping->size()) {
130
        // TODO: Each sender lock is enough.
131
15
        std::lock_guard lock_guard(_lock);
132
15
        ssize_t origin_size = mapping->size();
133
15
        if (segid + 1 > origin_size) {
134
15
            mapping->resize(segid + 1, std::numeric_limits<uint32_t>::max());
135
39
            for (size_t index = origin_size; index <= segid; index++) {
136
24
                mapping->at(index) = _next_segid;
137
24
                _next_segid++;
138
24
                VLOG_DEBUG << "src_id=" << src_id << ", segid=" << index << " to "
139
0
                           << " segid=" << _next_segid - 1 << ", " << *this;
140
24
            }
141
15
        }
142
15
    }
143
144
    // Each sender sends data in one segment sequential, so we also do not
145
    // need a lock here.
146
26
    bool eos = header.segment_eos();
147
26
    FileType file_type = header.file_type();
148
26
    uint32_t new_segid = mapping->at(segid);
149
26
    DCHECK(new_segid != std::numeric_limits<uint32_t>::max());
150
26
    butil::IOBuf buf = data->movable();
151
26
    auto flush_func = [this, new_segid, eos, buf, header, file_type]() mutable {
152
26
        signal::set_signal_task_id(_load_id);
153
26
        g_load_stream_flush_running_threads << -1;
154
26
        auto st = _load_stream_writer->append_data(new_segid, header.offset(), buf, file_type);
155
26
        if (!st.ok() && !config::is_cloud_mode()) {
156
1
            auto res = ExecEnv::get_tablet(_id);
157
1
            TabletSharedPtr tablet =
158
1
                    res.has_value() ? std::dynamic_pointer_cast<Tablet>(res.value()) : nullptr;
159
1
            if (tablet) {
160
1
                tablet->report_error(st);
161
1
            }
162
1
        }
163
26
        if (eos && st.ok()) {
164
21
            DBUG_EXECUTE_IF("TabletStream.append_data.unknown_file_type",
165
21
                            { file_type = static_cast<FileType>(-1); });
166
21
            if (file_type == FileType::SEGMENT_FILE || file_type == FileType::INVERTED_INDEX_FILE) {
167
21
                st = _load_stream_writer->close_writer(new_segid, file_type);
168
21
            } else {
169
0
                st = Status::InternalError(
170
0
                        "appent data failed, file type error, file type = {}, "
171
0
                        "segment_id={}",
172
0
                        file_type, new_segid);
173
0
            }
174
21
        }
175
26
        DBUG_EXECUTE_IF("TabletStream.append_data.append_failed",
176
26
                        { st = Status::InternalError("fault injection"); });
177
26
        if (!st.ok()) {
178
2
            _status.update(st);
179
2
            LOG(WARNING) << "write data failed " << st << ", " << *this;
180
2
        }
181
26
    };
182
26
    auto load_stream_flush_token_max_tasks = config::load_stream_flush_token_max_tasks;
183
26
    auto load_stream_max_wait_flush_token_time_ms =
184
26
            config::load_stream_max_wait_flush_token_time_ms;
185
26
    DBUG_EXECUTE_IF("TabletStream.append_data.long_wait", {
186
26
        load_stream_flush_token_max_tasks = 0;
187
26
        load_stream_max_wait_flush_token_time_ms = 1000;
188
26
    });
189
26
    MonotonicStopWatch timer;
190
26
    timer.start();
191
26
    while (_flush_token->num_tasks() >= load_stream_flush_token_max_tasks) {
192
0
        if (timer.elapsed_time() / 1000 / 1000 >= load_stream_max_wait_flush_token_time_ms) {
193
0
            _status.update(
194
0
                    Status::Error<true>("wait flush token back pressure time is more than "
195
0
                                        "load_stream_max_wait_flush_token_time {}",
196
0
                                        load_stream_max_wait_flush_token_time_ms));
197
0
            return _status.status();
198
0
        }
199
0
        bthread_usleep(2 * 1000); // 2ms
200
0
    }
201
26
    timer.stop();
202
26
    int64_t time_ms = timer.elapsed_time() / 1000 / 1000;
203
26
    g_load_stream_flush_wait_ms << time_ms;
204
26
    g_load_stream_flush_running_threads << 1;
205
26
    Status st = Status::OK();
206
26
    DBUG_EXECUTE_IF("TabletStream.append_data.submit_func_failed",
207
26
                    { st = Status::InternalError("fault injection"); });
208
26
    if (st.ok()) {
209
26
        st = _flush_token->submit_func(flush_func);
210
26
    }
211
26
    if (!st.ok()) {
212
0
        _status.update(st);
213
0
    }
214
26
    return _status.status();
215
26
}
216
217
0
Status TabletStream::add_segment(const PStreamHeader& header, butil::IOBuf* data) {
218
0
    if (!_status.ok()) {
219
0
        return _status.status();
220
0
    }
221
222
0
    SCOPED_TIMER(_add_segment_timer);
223
0
    DCHECK(header.has_segment_statistics());
224
0
    SegmentStatistics stat(header.segment_statistics());
225
226
0
    int64_t src_id = header.src_id();
227
0
    uint32_t segid = header.segment_id();
228
0
    uint32_t new_segid;
229
0
    DBUG_EXECUTE_IF("TabletStream.add_segment.unknown_segid", { segid = UNKNOWN_ID_FOR_TEST; });
230
0
    {
231
0
        std::lock_guard lock_guard(_lock);
232
0
        if (!_segids_mapping.contains(src_id)) {
233
0
            _status.update(Status::InternalError(
234
0
                    "add segment failed, no segment written by this src be yet, src_id={}, "
235
0
                    "segment_id={}",
236
0
                    src_id, segid));
237
0
            return _status.status();
238
0
        }
239
0
        DBUG_EXECUTE_IF("TabletStream.add_segment.segid_never_written",
240
0
                        { segid = static_cast<uint32_t>(_segids_mapping[src_id]->size()); });
241
0
        if (segid >= _segids_mapping[src_id]->size()) {
242
0
            _status.update(Status::InternalError(
243
0
                    "add segment failed, segment is never written, src_id={}, segment_id={}",
244
0
                    src_id, segid));
245
0
            return _status.status();
246
0
        }
247
0
        new_segid = _segids_mapping[src_id]->at(segid);
248
0
    }
249
0
    DCHECK(new_segid != std::numeric_limits<uint32_t>::max());
250
251
0
    auto add_segment_func = [this, new_segid, stat]() {
252
0
        signal::set_signal_task_id(_load_id);
253
0
        auto st = _load_stream_writer->add_segment(new_segid, stat);
254
0
        DBUG_EXECUTE_IF("TabletStream.add_segment.add_segment_failed",
255
0
                        { st = Status::InternalError("fault injection"); });
256
0
        if (!st.ok()) {
257
0
            _status.update(st);
258
0
            LOG(INFO) << "add segment failed " << *this;
259
0
        }
260
0
    };
261
0
    Status st = Status::OK();
262
0
    DBUG_EXECUTE_IF("TabletStream.add_segment.submit_func_failed",
263
0
                    { st = Status::InternalError("fault injection"); });
264
0
    if (st.ok()) {
265
0
        st = _flush_token->submit_func(add_segment_func);
266
0
    }
267
0
    if (!st.ok()) {
268
0
        _status.update(st);
269
0
    }
270
0
    return _status.status();
271
0
}
272
273
30
Status TabletStream::_run_in_heavy_work_pool(std::function<Status()> fn) {
274
30
    bthread::Mutex mu;
275
30
    std::unique_lock<bthread::Mutex> lock(mu);
276
30
    bthread::ConditionVariable cv;
277
30
    auto st = Status::OK();
278
30
    auto func = [this, &mu, &cv, &st, &fn] {
279
30
        signal::set_signal_task_id(_load_id);
280
30
        st = fn();
281
30
        std::lock_guard<bthread::Mutex> lock(mu);
282
30
        cv.notify_one();
283
30
    };
284
30
    bool ret = _load_stream_mgr->heavy_work_pool()->try_offer(func);
285
30
    if (!ret) {
286
0
        return Status::Error<ErrorCode::INTERNAL_ERROR>(
287
0
                "there is not enough thread resource for close load");
288
0
    }
289
30
    cv.wait(lock);
290
30
    return st;
291
30
}
292
293
28
void TabletStream::wait_for_flush_tasks() {
294
28
    {
295
28
        std::lock_guard lock_guard(_lock);
296
28
        if (_flush_tasks_done) {
297
14
            return;
298
14
        }
299
14
        _flush_tasks_done = true;
300
14
    }
301
302
14
    if (!_status.ok()) {
303
1
        _flush_token->shutdown();
304
1
        return;
305
1
    }
306
307
    // Use heavy_work_pool to avoid blocking bthread
308
13
    auto st = _run_in_heavy_work_pool([this]() {
309
13
        _flush_token->wait();
310
13
        return Status::OK();
311
13
    });
312
13
    if (!st.ok()) {
313
        // If heavy_work_pool is unavailable, fall back to shutdown
314
        // which will cancel pending tasks and wait for running tasks
315
0
        _flush_token->shutdown();
316
0
        _status.update(st);
317
0
    }
318
13
}
319
320
14
void TabletStream::pre_close() {
321
14
    SCOPED_TIMER(_close_wait_timer);
322
14
    wait_for_flush_tasks();
323
324
14
    if (!_status.ok()) {
325
3
        return;
326
3
    }
327
328
11
    DBUG_EXECUTE_IF("TabletStream.close.segment_num_mismatch", { _num_segments++; });
329
11
    if (_check_num_segments && (_next_segid.load() != _num_segments)) {
330
2
        _status.update(Status::Corruption(
331
2
                "segment num mismatch in tablet {}, expected: {}, actual: {}, load_id: {}", _id,
332
2
                _num_segments, _next_segid.load(), print_id(_load_id)));
333
2
        return;
334
2
    }
335
336
9
    _status.update(_run_in_heavy_work_pool([this]() { return _load_stream_writer->pre_close(); }));
337
9
}
338
339
14
Status TabletStream::close() {
340
14
    if (!_status.ok()) {
341
6
        return _status.status();
342
6
    }
343
344
8
    SCOPED_TIMER(_close_wait_timer);
345
8
    _status.update(_run_in_heavy_work_pool([this]() { return _load_stream_writer->close(); }));
346
8
    return _status.status();
347
14
}
348
349
IndexStream::IndexStream(const PUniqueId& load_id, int64_t id, int64_t txn_id,
350
                         std::shared_ptr<OlapTableSchemaParam> schema,
351
                         LoadStreamMgr* load_stream_mgr, RuntimeProfile* profile)
352
28
        : _id(id),
353
28
          _load_id(load_id),
354
28
          _txn_id(txn_id),
355
28
          _schema(schema),
356
28
          _load_stream_mgr(load_stream_mgr) {
357
28
    _profile = profile->create_child(fmt::format("IndexStream {}", id), true, true);
358
28
    _append_data_timer = ADD_TIMER(_profile, "AppendDataTime");
359
28
    _close_wait_timer = ADD_TIMER(_profile, "CloseWaitTime");
360
28
}
361
362
28
IndexStream::~IndexStream() {
363
    // Ensure all TabletStreams have their flush tokens properly handled before destruction.
364
    // In normal flow, close() should have called pre_close() on all tablet streams.
365
    // But if IndexStream is destroyed without close() being called (e.g., on_idle_timeout),
366
    // we need to wait for flush tasks here to ensure flush tokens are properly shut down.
367
28
    for (auto& [_, tablet_stream] : _tablet_streams_map) {
368
14
        tablet_stream->wait_for_flush_tasks();
369
14
    }
370
28
}
371
372
27
Status IndexStream::append_data(const PStreamHeader& header, butil::IOBuf* data) {
373
27
    SCOPED_TIMER(_append_data_timer);
374
27
    int64_t tablet_id = header.tablet_id();
375
27
    TabletStreamSharedPtr tablet_stream;
376
27
    {
377
27
        std::lock_guard lock_guard(_lock);
378
27
        auto it = _tablet_streams_map.find(tablet_id);
379
27
        if (it == _tablet_streams_map.end()) {
380
13
            _init_tablet_stream(tablet_stream, tablet_id, header.partition_id());
381
14
        } else {
382
14
            tablet_stream = it->second;
383
14
        }
384
27
    }
385
386
27
    return tablet_stream->append_data(header, data);
387
27
}
388
389
void IndexStream::_init_tablet_stream(TabletStreamSharedPtr& tablet_stream, int64_t tablet_id,
390
14
                                      int64_t partition_id) {
391
14
    tablet_stream = std::make_shared<TabletStream>(_load_id, tablet_id, _txn_id, _load_stream_mgr,
392
14
                                                   _profile);
393
14
    _tablet_streams_map[tablet_id] = tablet_stream;
394
14
    auto st = tablet_stream->init(_schema, _id, partition_id);
395
14
    if (!st.ok()) {
396
1
        LOG(WARNING) << "tablet stream init failed " << *tablet_stream;
397
1
    }
398
14
}
399
400
0
void IndexStream::get_all_write_tablet_ids(std::vector<int64_t>* tablet_ids) {
401
0
    std::lock_guard lock_guard(_lock);
402
0
    for (const auto& [tablet_id, _] : _tablet_streams_map) {
403
0
        tablet_ids->push_back(tablet_id);
404
0
    }
405
0
}
406
407
void IndexStream::close(const std::vector<PTabletID>& tablets_to_commit,
408
28
                        std::vector<int64_t>* success_tablet_ids, FailedTablets* failed_tablets) {
409
28
    std::lock_guard lock_guard(_lock);
410
28
    SCOPED_TIMER(_close_wait_timer);
411
    // open all need commit tablets
412
34
    for (const auto& tablet : tablets_to_commit) {
413
34
        if (_id != tablet.index_id()) {
414
18
            continue;
415
18
        }
416
16
        TabletStreamSharedPtr tablet_stream;
417
16
        auto it = _tablet_streams_map.find(tablet.tablet_id());
418
16
        if (it == _tablet_streams_map.end()) {
419
1
            _init_tablet_stream(tablet_stream, tablet.tablet_id(), tablet.partition_id());
420
15
        } else {
421
15
            tablet_stream = it->second;
422
15
        }
423
16
        if (tablet.has_num_segments()) {
424
16
            tablet_stream->add_num_segments(tablet.num_segments());
425
16
        } else {
426
            // for compatibility reasons (sink from old version BE)
427
0
            tablet_stream->disable_num_segments_check();
428
0
        }
429
16
    }
430
431
28
    for (auto& [_, tablet_stream] : _tablet_streams_map) {
432
14
        tablet_stream->pre_close();
433
14
    }
434
435
28
    for (auto& [_, tablet_stream] : _tablet_streams_map) {
436
14
        auto st = tablet_stream->close();
437
14
        if (st.ok()) {
438
8
            success_tablet_ids->push_back(tablet_stream->id());
439
8
        } else {
440
6
            LOG(INFO) << "close tablet stream " << *tablet_stream << ", status=" << st;
441
6
            failed_tablets->emplace_back(tablet_stream->id(), st);
442
6
        }
443
14
    }
444
28
}
445
446
// TODO: Profile is temporary disabled, because:
447
// 1. It's not being processed by the upstream for now
448
// 2. There are some problems in _profile->to_thrift()
449
LoadStream::LoadStream(const PUniqueId& load_id, LoadStreamMgr* load_stream_mgr,
450
                       bool enable_profile)
451
14
        : _load_id(load_id), _enable_profile(false), _load_stream_mgr(load_stream_mgr) {
452
14
    g_load_stream_cnt << 1;
453
14
    _profile = std::make_unique<RuntimeProfile>("LoadStream");
454
14
    _append_data_timer = ADD_TIMER(_profile, "AppendDataTime");
455
14
    _close_wait_timer = ADD_TIMER(_profile, "CloseWaitTime");
456
14
    TUniqueId load_tid = ((UniqueId)load_id).to_thrift();
457
#ifndef BE_TEST
458
    std::shared_ptr<QueryContext> query_context =
459
            ExecEnv::GetInstance()->fragment_mgr()->get_query_ctx(load_tid);
460
    if (query_context != nullptr) {
461
        _resource_ctx = query_context->resource_ctx();
462
    } else {
463
        _resource_ctx = ResourceContext::create_shared();
464
        _resource_ctx->task_controller()->set_task_id(load_tid);
465
        std::shared_ptr<MemTrackerLimiter> mem_tracker = MemTrackerLimiter::create_shared(
466
                MemTrackerLimiter::Type::LOAD,
467
                fmt::format("(FromLoadStream)Load#Id={}", ((UniqueId)load_id).to_string()));
468
        _resource_ctx->memory_context()->set_mem_tracker(mem_tracker);
469
    }
470
#else
471
14
    _resource_ctx = ResourceContext::create_shared();
472
14
    _resource_ctx->task_controller()->set_task_id(load_tid);
473
14
    std::shared_ptr<MemTrackerLimiter> mem_tracker = MemTrackerLimiter::create_shared(
474
14
            MemTrackerLimiter::Type::LOAD,
475
14
            fmt::format("(FromLoadStream)Load#Id={}", ((UniqueId)load_id).to_string()));
476
14
    _resource_ctx->memory_context()->set_mem_tracker(mem_tracker);
477
14
#endif
478
14
}
479
480
14
LoadStream::~LoadStream() {
481
14
    g_load_stream_cnt << -1;
482
14
    LOG(INFO) << "load stream is deconstructed " << *this;
483
14
}
484
485
14
Status LoadStream::init(const POpenLoadStreamRequest* request) {
486
14
    _txn_id = request->txn_id();
487
14
    _total_streams = static_cast<int32_t>(request->total_streams());
488
14
    _is_incremental = (_total_streams == 0);
489
490
14
    _schema = std::make_shared<OlapTableSchemaParam>();
491
14
    RETURN_IF_ERROR(_schema->init(request->schema()));
492
28
    for (auto& index : request->schema().indexes()) {
493
28
        _index_streams_map[index.id()] = std::make_shared<IndexStream>(
494
28
                _load_id, index.id(), _txn_id, _schema, _load_stream_mgr, _profile.get());
495
28
    }
496
14
    LOG(INFO) << "succeed to init load stream " << *this;
497
14
    return Status::OK();
498
14
}
499
500
bool LoadStream::close(int64_t src_id, const std::vector<PTabletID>& tablets_to_commit,
501
16
                       std::vector<int64_t>* success_tablet_ids, FailedTablets* failed_tablets) {
502
16
    std::lock_guard<bthread::Mutex> lock_guard(_lock);
503
16
    SCOPED_TIMER(_close_wait_timer);
504
505
    // we do nothing until recv CLOSE_LOAD from all stream to ensure all data are handled before ack
506
16
    _open_streams[src_id]--;
507
16
    if (_open_streams[src_id] == 0) {
508
16
        _open_streams.erase(src_id);
509
16
    }
510
16
    _close_load_cnt++;
511
16
    LOG(INFO) << "received CLOSE_LOAD from sender " << src_id << ", remaining "
512
16
              << _total_streams - _close_load_cnt << " senders, " << *this;
513
514
16
    _tablets_to_commit.insert(_tablets_to_commit.end(), tablets_to_commit.begin(),
515
16
                              tablets_to_commit.end());
516
517
16
    if (_close_load_cnt < _total_streams) {
518
        // do not return commit info if there is remaining streams.
519
2
        return false;
520
2
    }
521
522
28
    for (auto& [_, index_stream] : _index_streams_map) {
523
28
        index_stream->close(_tablets_to_commit, success_tablet_ids, failed_tablets);
524
28
    }
525
14
    LOG(INFO) << "close load " << *this << ", success_tablet_num=" << success_tablet_ids->size()
526
14
              << ", failed_tablet_num=" << failed_tablets->size();
527
14
    return true;
528
16
}
529
530
void LoadStream::_report_result(StreamId stream, const Status& status,
531
                                const std::vector<int64_t>& success_tablet_ids,
532
36
                                const FailedTablets& failed_tablets, bool eos) {
533
36
    LOG(INFO) << "report result " << *this << ", success tablet num " << success_tablet_ids.size()
534
36
              << ", failed tablet num " << failed_tablets.size();
535
36
    butil::IOBuf buf;
536
36
    PLoadStreamResponse response;
537
36
    response.set_eos(eos);
538
36
    status.to_protobuf(response.mutable_status());
539
36
    for (auto& id : success_tablet_ids) {
540
8
        response.add_success_tablet_ids(id);
541
8
    }
542
36
    for (auto& [id, st] : failed_tablets) {
543
10
        auto pb = response.add_failed_tablets();
544
10
        pb->set_id(id);
545
10
        st.to_protobuf(pb->mutable_status());
546
10
    }
547
548
36
    if (_enable_profile && _close_load_cnt == _total_streams) {
549
0
        TRuntimeProfileTree tprofile;
550
0
        ThriftSerializer ser(false, 4096);
551
0
        uint8_t* profile_buf = nullptr;
552
0
        uint32_t len = 0;
553
0
        std::unique_lock<bthread::Mutex> l(_lock);
554
555
0
        _profile->to_thrift(&tprofile);
556
0
        auto st = ser.serialize(&tprofile, &len, &profile_buf);
557
0
        if (st.ok()) {
558
0
            response.set_load_stream_profile(profile_buf, len);
559
0
        } else {
560
0
            LOG(WARNING) << "TRuntimeProfileTree serialize failed, errmsg=" << st << ", " << *this;
561
0
        }
562
0
    }
563
564
36
    buf.append(response.SerializeAsString());
565
36
    auto wst = _write_stream(stream, buf);
566
36
    if (!wst.ok()) {
567
0
        LOG(WARNING) << " report result failed with " << wst << ", " << *this;
568
0
    }
569
36
}
570
571
0
void LoadStream::_report_schema(StreamId stream, const PStreamHeader& hdr) {
572
0
    butil::IOBuf buf;
573
0
    PLoadStreamResponse response;
574
0
    Status st = Status::OK();
575
0
    for (const auto& req : hdr.tablets()) {
576
0
        BaseTabletSPtr tablet;
577
0
        if (auto res = ExecEnv::get_tablet(req.tablet_id()); res.has_value()) {
578
0
            tablet = std::move(res).value();
579
0
        } else {
580
0
            st = std::move(res).error();
581
0
            break;
582
0
        }
583
0
        auto* resp = response.add_tablet_schemas();
584
0
        resp->set_index_id(req.index_id());
585
0
        resp->set_enable_unique_key_merge_on_write(tablet->enable_unique_key_merge_on_write());
586
0
        tablet->tablet_schema()->to_schema_pb(resp->mutable_tablet_schema());
587
0
    }
588
0
    st.to_protobuf(response.mutable_status());
589
590
0
    buf.append(response.SerializeAsString());
591
0
    auto wst = _write_stream(stream, buf);
592
0
    if (!wst.ok()) {
593
0
        LOG(WARNING) << " report result failed with " << wst << ", " << *this;
594
0
    }
595
0
}
596
597
0
void LoadStream::_report_tablet_load_info(StreamId stream, int64_t index_id) {
598
0
    std::vector<int64_t> write_tablet_ids;
599
0
    auto it = _index_streams_map.find(index_id);
600
0
    if (it != _index_streams_map.end()) {
601
0
        it->second->get_all_write_tablet_ids(&write_tablet_ids);
602
0
    }
603
604
0
    if (!write_tablet_ids.empty()) {
605
0
        butil::IOBuf buf;
606
0
        PLoadStreamResponse response;
607
0
        auto* tablet_load_infos = response.mutable_tablet_load_rowset_num_infos();
608
0
        _collect_tablet_load_info_from_tablets(write_tablet_ids, tablet_load_infos);
609
0
        buf.append(response.SerializeAsString());
610
0
        auto wst = _write_stream(stream, buf);
611
0
        if (!wst.ok()) {
612
0
            LOG(WARNING) << "report tablet load info failed with " << wst << ", " << *this;
613
0
        }
614
0
    }
615
0
}
616
617
void LoadStream::_collect_tablet_load_info_from_tablets(
618
        const std::vector<int64_t>& tablet_ids,
619
0
        google::protobuf::RepeatedPtrField<PTabletLoadRowsetInfo>* tablet_load_infos) {
620
0
    for (auto tablet_id : tablet_ids) {
621
0
        BaseTabletSPtr tablet;
622
0
        if (auto res = ExecEnv::get_tablet(tablet_id); res.has_value()) {
623
0
            tablet = std::move(res).value();
624
0
        } else {
625
0
            continue;
626
0
        }
627
0
        BaseDeltaWriter::collect_tablet_load_rowset_num_info(tablet.get(), tablet_load_infos);
628
0
    }
629
0
}
630
631
36
Status LoadStream::_write_stream(StreamId stream, butil::IOBuf& buf) {
632
36
    for (;;) {
633
36
        int ret = 0;
634
36
        DBUG_EXECUTE_IF("LoadStream._write_stream.EAGAIN", { ret = EAGAIN; });
635
36
        if (ret == 0) {
636
36
            ret = brpc::StreamWrite(stream, buf);
637
36
        }
638
36
        switch (ret) {
639
36
        case 0:
640
36
            return Status::OK();
641
0
        case EAGAIN: {
642
0
            const timespec time = butil::seconds_from_now(config::load_stream_eagain_wait_seconds);
643
0
            int wait_ret = brpc::StreamWait(stream, &time);
644
0
            if (wait_ret != 0) {
645
0
                return Status::InternalError("StreamWait failed, err={}", wait_ret);
646
0
            }
647
0
            break;
648
0
        }
649
0
        default:
650
0
            return Status::InternalError("StreamWrite failed, err={}", ret);
651
36
        }
652
36
    }
653
0
    return Status::OK();
654
36
}
655
656
62
void LoadStream::_parse_header(butil::IOBuf* const message, PStreamHeader& hdr) {
657
62
    butil::IOBufAsZeroCopyInputStream wrapper(*message);
658
62
    hdr.ParseFromZeroCopyStream(&wrapper);
659
62
    VLOG_DEBUG << "header parse result: " << hdr.DebugString();
660
62
}
661
662
28
Status LoadStream::_append_data(const PStreamHeader& header, butil::IOBuf* data) {
663
28
    SCOPED_TIMER(_append_data_timer);
664
28
    IndexStreamSharedPtr index_stream;
665
666
28
    int64_t index_id = header.index_id();
667
28
    DBUG_EXECUTE_IF("TabletStream._append_data.unknown_indexid",
668
28
                    { index_id = UNKNOWN_ID_FOR_TEST; });
669
28
    auto it = _index_streams_map.find(index_id);
670
28
    if (it == _index_streams_map.end()) {
671
1
        return Status::Error<ErrorCode::INVALID_ARGUMENT>("unknown index_id {}", index_id);
672
27
    } else {
673
27
        index_stream = it->second;
674
27
    }
675
676
27
    return index_stream->append_data(header, data);
677
28
}
678
679
50
int LoadStream::on_received_messages(StreamId id, butil::IOBuf* const messages[], size_t size) {
680
50
    VLOG_DEBUG << "on_received_messages " << id << " " << size;
681
112
    for (size_t i = 0; i < size; ++i) {
682
124
        while (messages[i]->size() > 0) {
683
            // step 1: parse header
684
62
            size_t hdr_len = 0;
685
62
            messages[i]->cutn((void*)&hdr_len, sizeof(size_t));
686
62
            butil::IOBuf hdr_buf;
687
62
            PStreamHeader hdr;
688
62
            messages[i]->cutn(&hdr_buf, hdr_len);
689
62
            _parse_header(&hdr_buf, hdr);
690
691
            // step 2: cut data
692
62
            size_t data_len = 0;
693
62
            messages[i]->cutn((void*)&data_len, sizeof(size_t));
694
62
            butil::IOBuf data_buf;
695
62
            PStreamHeader data;
696
62
            messages[i]->cutn(&data_buf, data_len);
697
698
            // step 3: dispatch
699
62
            _dispatch(id, hdr, &data_buf);
700
62
        }
701
62
    }
702
50
    return 0;
703
50
}
704
705
62
void LoadStream::_dispatch(StreamId id, const PStreamHeader& hdr, butil::IOBuf* data) {
706
62
    VLOG_DEBUG << PStreamHeader_Opcode_Name(hdr.opcode()) << " from " << hdr.src_id()
707
0
               << " with tablet " << hdr.tablet_id();
708
62
    SCOPED_ATTACH_TASK(_resource_ctx);
709
    // CLOSE_LOAD message should not be fault injected,
710
    // otherwise the message will be ignored and causing close wait timeout
711
62
    if (hdr.opcode() != PStreamHeader::CLOSE_LOAD) {
712
30
        DBUG_EXECUTE_IF("LoadStream._dispatch.unknown_loadid", {
713
30
            PStreamHeader& t_hdr = const_cast<PStreamHeader&>(hdr);
714
30
            PUniqueId* load_id = t_hdr.mutable_load_id();
715
30
            load_id->set_hi(UNKNOWN_ID_FOR_TEST);
716
30
            load_id->set_lo(UNKNOWN_ID_FOR_TEST);
717
30
        });
718
30
        DBUG_EXECUTE_IF("LoadStream._dispatch.unknown_srcid", {
719
30
            PStreamHeader& t_hdr = const_cast<PStreamHeader&>(hdr);
720
30
            t_hdr.set_src_id(UNKNOWN_ID_FOR_TEST);
721
30
        });
722
30
    }
723
62
    if (UniqueId(hdr.load_id()) != UniqueId(_load_id)) {
724
1
        Status st = Status::Error<ErrorCode::INVALID_ARGUMENT>(
725
1
                "invalid load id {}, expected {}", print_id(hdr.load_id()), print_id(_load_id));
726
1
        _report_failure(id, st, hdr);
727
1
        return;
728
1
    }
729
730
61
    {
731
61
        std::lock_guard lock_guard(_lock);
732
61
        if (!_open_streams.contains(hdr.src_id())) {
733
17
            Status st = Status::Error<ErrorCode::INVALID_ARGUMENT>("no open stream from source {}",
734
17
                                                                   hdr.src_id());
735
17
            _report_failure(id, st, hdr);
736
17
            return;
737
17
        }
738
61
    }
739
740
44
    switch (hdr.opcode()) {
741
0
    case PStreamHeader::ADD_SEGMENT: {
742
0
        auto st = _append_data(hdr, data);
743
0
        if (!st.ok()) {
744
0
            _report_failure(id, st, hdr);
745
0
        } else {
746
            // Report tablet load info only on ADD_SEGMENT to reduce frequency.
747
            // ADD_SEGMENT is sent once per segment, while APPEND_DATA is sent
748
            // for every data batch. This reduces unnecessary writes and avoids
749
            // potential stream write failures when the sender is closing.
750
0
            _report_tablet_load_info(id, hdr.index_id());
751
0
        }
752
0
    } break;
753
28
    case PStreamHeader::APPEND_DATA: {
754
28
        auto st = _append_data(hdr, data);
755
28
        if (!st.ok()) {
756
2
            _report_failure(id, st, hdr);
757
2
        }
758
28
    } break;
759
16
    case PStreamHeader::CLOSE_LOAD: {
760
16
        std::vector<int64_t> success_tablet_ids;
761
16
        FailedTablets failed_tablets;
762
16
        std::vector<PTabletID> tablets_to_commit(hdr.tablets().begin(), hdr.tablets().end());
763
16
        bool all_closed =
764
16
                close(hdr.src_id(), tablets_to_commit, &success_tablet_ids, &failed_tablets);
765
16
        _report_result(id, Status::OK(), success_tablet_ids, failed_tablets, true);
766
16
        std::lock_guard<bthread::Mutex> lock_guard(_lock);
767
        // if incremental stream, we need to wait for all non-incremental streams to be closed
768
        // before closing incremental streams. We need a fencing mechanism to avoid use after closing
769
        // across different be.
770
16
        if (hdr.has_num_incremental_streams() && hdr.num_incremental_streams() > 0) {
771
0
            _closing_stream_ids.push_back(id);
772
16
        } else {
773
16
            brpc::StreamClose(id);
774
16
        }
775
776
16
        if (all_closed) {
777
14
            for (auto& closing_id : _closing_stream_ids) {
778
0
                brpc::StreamClose(closing_id);
779
0
            }
780
14
            _closing_stream_ids.clear();
781
14
        }
782
16
    } break;
783
0
    case PStreamHeader::GET_SCHEMA: {
784
0
        _report_schema(id, hdr);
785
0
    } break;
786
0
    default:
787
0
        LOG(WARNING) << "unexpected stream message " << hdr.opcode() << ", " << *this;
788
0
        DCHECK(false);
789
44
    }
790
44
}
791
792
0
void LoadStream::on_idle_timeout(StreamId id) {
793
0
    LOG(WARNING) << "closing load stream on idle timeout, " << *this;
794
0
    brpc::StreamClose(id);
795
0
}
796
797
16
void LoadStream::on_closed(StreamId id) {
798
    // `this` may be freed by other threads after increasing `_close_rpc_cnt`,
799
    // format string first to prevent use-after-free
800
16
    std::stringstream ss;
801
16
    ss << *this;
802
16
    auto remaining_streams = _total_streams - _close_rpc_cnt.fetch_add(1) - 1;
803
16
    LOG(INFO) << "stream " << id << " on_closed, remaining streams = " << remaining_streams << ", "
804
16
              << ss.str();
805
16
    if (remaining_streams == 0) {
806
14
        _load_stream_mgr->clear_load(_load_id);
807
14
    }
808
16
}
809
810
110
inline std::ostream& operator<<(std::ostream& ostr, const LoadStream& load_stream) {
811
110
    ostr << "load_id=" << print_id(load_stream._load_id) << ", txn_id=" << load_stream._txn_id;
812
110
    return ostr;
813
110
}
814
815
#include "common/compile_check_end.h"
816
} // namespace doris