Skip to content

Commit 327cbf8

Browse files
authored
[fix](move-memtable) fix move memtable core when use multi table load (apache#35458)
## Proposed changes Move memtable core when use multi table load: ``` 0x51f000c73860 is located 3040 bytes inside of 3456-byte region [0x51f000c72c80,0x51f000c73a00) freed by thread T4867 (FragmentMgrThre) here: #0 0x558f6ad7f43d in operator delete(void*) (/mnt/hdd01/STRESS_ENV/be/lib/doris_be+0x22eec43d) (BuildId: b46f73d1f76dfcd6) #1 0x558f6e6cea2c in std::__new_allocator<doris::PTabletID>::deallocate(doris::PTabletID*, unsigned long) /mnt/disk2/xujianxu/ldb_toolchain/bin/../lib/gcc/x86_64-linux-gnu/13/../../../../include/c++/13/bits/new_allocator.h:168:2 #2 0x558f6e6ce9e7 in std::allocator<doris::PTabletID>::deallocate(doris::PTabletID*, unsigned long) /mnt/disk2/xujianxu/ldb_toolchain/bin/../lib/gcc/x86_64-linux-gnu/13/../../../../include/c++/13/bits/allocator.h:210:25 #3 0x558f6e6ce9e7 in std::allocator_traits<std::allocator<doris::PTabletID>>::deallocate(std::allocator<doris::PTabletID>&, doris::PTabletID*, unsigned long) /mnt/disk2/xujianxu/ldb_toolchain/bin/../lib/gcc/x86_64-linux-gnu/13/../../../../include/c++/13/bits/alloc_traits.h:516:13 #4 0x558f6e6ce9e7 in std::_Vector_base<doris::PTabletID, std::allocator<doris::PTabletID>>::_M_deallocate(doris::PTabletID*, unsigned long) /mnt/disk2/xujianxu/ldb_toolchain/bin/../lib/gcc/x86_64-linux-gnu/13/../../../../include/c++/13/bits/stl_vector.h:387:4 #5 0x558f6e6d0780 in void std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>::_M_range_insert<__gnu_cxx::__normal_iterator<doris::PTabletID const*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>>(__gnu_cxx::__normal_iterator<doris::PTabletID*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>, __gnu_cxx::__normal_iterator<doris::PTabletID const*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>, __gnu_cxx::__normal_iterator<doris::PTabletID const*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>, std::forward_iterator_tag) /mnt/disk2/xujianxu/ldb_toolchain/bin/../lib/gcc/x86_64-linux-gnu/13/../../../../include/c++/13/bits/vector.tcc:832:3 #6 0x558f6e6c54c5 in __gnu_cxx::__normal_iterator<doris::PTabletID*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>> std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>::insert<__gnu_cxx::__normal_iterator<doris::PTabletID const*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>, void>(__gnu_cxx::__normal_iterator<doris::PTabletID const*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>, __gnu_cxx::__normal_iterator<doris::PTabletID const*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>, __gnu_cxx::__normal_iterator<doris::PTabletID const*, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>>>) /mnt/disk2/xujianxu/ldb_toolchain/bin/../lib/gcc/x86_64-linux-gnu/13/../../../../include/c++/13/bits/stl_vector.h:1483:4 #7 0x558f9b4b214f in doris::LoadStreamMap::save_tablets_to_commit(long, std::vector<doris::PTabletID, std::allocator<doris::PTabletID>> const&) /mnt/disk2/xujianxu/doris/be/src/vec/sink/load_stream_map_pool.cpp:90:13 apache#8 0x558f9b7258dd in doris::vectorized::VTabletWriterV2::_calc_tablets_to_commit() /mnt/disk2/xujianxu/doris/be/src/vec/sink/writer/vtablet_writer_v2.cpp:650:27 apache#9 0x558f9b7229f1 in doris::vectorized::VTabletWriterV2::close(doris::Status) /mnt/disk2/xujianxu/doris/be/src/vec/sink/writer/vtablet_writer_v2.cpp:547:9 ``` Multiple sinks with different table loads use the load id, causing confusion in the use of shared data structures between sinks.
1 parent 6a005b6 commit 327cbf8

File tree

3 files changed

+99
-151
lines changed

3 files changed

+99
-151
lines changed

be/src/io/file_factory.cpp

-20
Original file line numberDiff line numberDiff line change
@@ -214,26 +214,6 @@ Status FileFactory::create_pipe_reader(const TUniqueId& load_id, io::FileReaderS
214214
*file_reader = stream_load_ctx->pipe;
215215
}
216216

217-
if (file_reader->get() == nullptr) {
218-
return Status::OK();
219-
}
220-
221-
auto multi_table_pipe = std::dynamic_pointer_cast<io::MultiTablePipe>(*file_reader);
222-
if (multi_table_pipe == nullptr || runtime_state == nullptr) {
223-
return Status::OK();
224-
}
225-
226-
TUniqueId pipe_id;
227-
if (runtime_state->enable_pipeline_x_exec()) {
228-
pipe_id = io::StreamLoadPipe::calculate_pipe_id(runtime_state->query_id(),
229-
runtime_state->fragment_id());
230-
} else {
231-
pipe_id = runtime_state->fragment_instance_id();
232-
}
233-
*file_reader = multi_table_pipe->get_pipe(pipe_id);
234-
LOG(INFO) << "create pipe reader for fragment instance: " << pipe_id
235-
<< " pipe: " << (*file_reader).get();
236-
237217
return Status::OK();
238218
}
239219

be/src/io/fs/multi_table_pipe.cpp

+97-122
Original file line numberDiff line numberDiff line change
@@ -61,9 +61,9 @@ Status MultiTablePipe::append_json(const char* data, size_t size) {
6161
}
6262

6363
KafkaConsumerPipePtr MultiTablePipe::get_pipe_by_table(const std::string& table) {
64-
auto pipe = _planned_pipes.find(table);
65-
DCHECK(pipe != _planned_pipes.end());
66-
return pipe->second;
64+
auto pair = _planned_tables.find(table);
65+
DCHECK(pair != _planned_tables.end());
66+
return std::static_pointer_cast<io::KafkaConsumerPipe>(pair->second->pipe);
6767
}
6868

6969
static std::string_view get_first_part(const char* dat, char delimiter) {
@@ -78,15 +78,15 @@ static std::string_view get_first_part(const char* dat, char delimiter) {
7878
}
7979

8080
Status MultiTablePipe::finish() {
81-
for (auto& pair : _planned_pipes) {
82-
RETURN_IF_ERROR(pair.second->finish());
81+
for (auto& pair : _planned_tables) {
82+
RETURN_IF_ERROR(pair.second->pipe->finish());
8383
}
8484
return Status::OK();
8585
}
8686

8787
void MultiTablePipe::cancel(const std::string& reason) {
88-
for (auto& pair : _planned_pipes) {
89-
pair.second->cancel(reason);
88+
for (auto& pair : _planned_tables) {
89+
pair.second->pipe->cancel(reason);
9090
}
9191
}
9292

@@ -101,19 +101,29 @@ Status MultiTablePipe::dispatch(const std::string& table, const char* data, size
101101
return Status::InternalError("empty data");
102102
}
103103
KafkaConsumerPipePtr pipe = nullptr;
104-
auto iter = _planned_pipes.find(table);
105-
if (iter != _planned_pipes.end()) {
106-
pipe = iter->second;
104+
auto iter = _planned_tables.find(table);
105+
if (iter != _planned_tables.end()) {
106+
pipe = std::static_pointer_cast<io::KafkaConsumerPipe>(iter->second->pipe);
107107
RETURN_NOT_OK_STATUS_WITH_WARN((pipe.get()->*cb)(data, size),
108108
"append failed in planned kafka pipe");
109109
} else {
110-
iter = _unplanned_pipes.find(table);
111-
if (iter == _unplanned_pipes.end()) {
110+
iter = _unplanned_tables.find(table);
111+
if (iter == _unplanned_tables.end()) {
112+
std::shared_ptr<StreamLoadContext> ctx =
113+
std::make_shared<StreamLoadContext>(doris::ExecEnv::GetInstance());
114+
ctx->id = UniqueId::gen_uid();
112115
pipe = std::make_shared<io::KafkaConsumerPipe>();
113-
LOG(INFO) << "create new unplanned pipe: " << pipe.get() << ", ctx: " << _ctx->brief();
114-
_unplanned_pipes.emplace(table, pipe);
116+
ctx->pipe = pipe;
117+
#ifndef BE_TEST
118+
RETURN_NOT_OK_STATUS_WITH_WARN(
119+
doris::ExecEnv::GetInstance()->new_load_stream_mgr()->put(ctx->id, ctx),
120+
"put stream load ctx error");
121+
#endif
122+
_unplanned_tables.emplace(table, ctx);
123+
LOG(INFO) << "create new unplanned table ctx, table: " << table
124+
<< "load id: " << ctx->id << ", txn id: " << _ctx->txn_id;
115125
} else {
116-
pipe = iter->second;
126+
pipe = std::static_pointer_cast<io::KafkaConsumerPipe>(iter->second->pipe);
117127
}
118128

119129
// It is necessary to determine whether the sum of pipe_current_capacity and size is greater than pipe_max_capacity,
@@ -124,7 +134,7 @@ Status MultiTablePipe::dispatch(const std::string& table, const char* data, size
124134
auto pipe_current_capacity = pipe->current_capacity();
125135
auto pipe_max_capacity = pipe->max_capacity();
126136
if (_unplanned_row_cnt >= _row_threshold ||
127-
_unplanned_pipes.size() >= _wait_tables_threshold ||
137+
_unplanned_tables.size() >= _wait_tables_threshold ||
128138
pipe_current_capacity + size > pipe_max_capacity) {
129139
LOG(INFO) << fmt::format(
130140
"unplanned row cnt={} reach row_threshold={} or "
@@ -151,112 +161,106 @@ Status MultiTablePipe::dispatch(const std::string& table, const char* data, size
151161

152162
#ifndef BE_TEST
153163
Status MultiTablePipe::request_and_exec_plans() {
154-
if (_unplanned_pipes.empty()) {
164+
if (_unplanned_tables.empty()) {
155165
return Status::OK();
156166
}
157167

158-
// get list of table names in unplanned pipes
159-
std::vector<std::string> tables;
160168
fmt::memory_buffer log_buffer;
161169
log_buffer.clear();
162-
fmt::format_to(log_buffer, "request plans for {} tables: [ ", _unplanned_pipes.size());
163-
for (auto& pair : _unplanned_pipes) {
164-
tables.push_back(pair.first);
170+
fmt::format_to(log_buffer, "request plans for {} tables: [ ", _unplanned_tables.size());
171+
for (auto& pair : _unplanned_tables) {
165172
fmt::format_to(log_buffer, "{} ", pair.first);
166173
}
167174
fmt::format_to(log_buffer, "]");
168175
LOG(INFO) << fmt::to_string(log_buffer);
169176

170-
TStreamLoadPutRequest request;
171-
set_request_auth(&request, _ctx->auth);
172-
request.db = _ctx->db;
173-
request.table_names = tables;
174-
request.__isset.table_names = true;
175-
request.txnId = _ctx->txn_id;
176-
request.formatType = _ctx->format;
177-
request.__set_compress_type(_ctx->compress_type);
178-
request.__set_header_type(_ctx->header_type);
179-
request.__set_loadId(_ctx->id.to_thrift());
180-
request.fileType = TFileType::FILE_STREAM;
181-
request.__set_thrift_rpc_timeout_ms(config::thrift_rpc_timeout_ms);
182-
request.__set_memtable_on_sink_node(_ctx->memtable_on_sink_node);
183-
request.__set_user(_ctx->qualified_user);
184-
request.__set_cloud_cluster(_ctx->cloud_cluster);
185-
// no need to register new_load_stream_mgr coz it is already done in routineload submit task
186-
187-
// plan this load
188-
ExecEnv* exec_env = doris::ExecEnv::GetInstance();
189-
TNetworkAddress master_addr = exec_env->master_info()->network_address;
190-
int64_t stream_load_put_start_time = MonotonicNanos();
191-
RETURN_IF_ERROR(ThriftRpcHelper::rpc<FrontendServiceClient>(
192-
master_addr.hostname, master_addr.port,
193-
[&request, this](FrontendServiceConnection& client) {
194-
client->streamLoadMultiTablePut(_ctx->multi_table_put_result, request);
195-
}));
196-
_ctx->stream_load_put_cost_nanos = MonotonicNanos() - stream_load_put_start_time;
197-
198-
Status plan_status(Status::create(_ctx->multi_table_put_result.status));
199-
if (!plan_status.ok()) {
200-
LOG(WARNING) << "plan streaming load failed. errmsg=" << plan_status << _ctx->brief();
201-
return plan_status;
202-
}
203-
204177
Status st;
205-
if (_ctx->multi_table_put_result.__isset.params &&
206-
!_ctx->multi_table_put_result.__isset.pipeline_params) {
207-
st = exec_plans(exec_env, _ctx->multi_table_put_result.params);
208-
} else if (!_ctx->multi_table_put_result.__isset.params &&
209-
_ctx->multi_table_put_result.__isset.pipeline_params) {
210-
st = exec_plans(exec_env, _ctx->multi_table_put_result.pipeline_params);
211-
} else {
212-
return Status::Aborted("too many or too few params are set in multi_table_put_result.");
213-
}
178+
for (auto& pair : _unplanned_tables) {
179+
TStreamLoadPutRequest request;
180+
set_request_auth(&request, _ctx->auth);
181+
std::vector<std::string> tables;
182+
tables.push_back(pair.first);
183+
request.db = _ctx->db;
184+
request.table_names = tables;
185+
request.__isset.table_names = true;
186+
request.txnId = _ctx->txn_id;
187+
request.formatType = _ctx->format;
188+
request.__set_compress_type(_ctx->compress_type);
189+
request.__set_header_type(_ctx->header_type);
190+
request.__set_loadId((pair.second->id).to_thrift());
191+
request.fileType = TFileType::FILE_STREAM;
192+
request.__set_thrift_rpc_timeout_ms(config::thrift_rpc_timeout_ms);
193+
request.__set_memtable_on_sink_node(_ctx->memtable_on_sink_node);
194+
request.__set_user(_ctx->qualified_user);
195+
request.__set_cloud_cluster(_ctx->cloud_cluster);
196+
// no need to register new_load_stream_mgr coz it is already done in routineload submit task
197+
198+
// plan this load
199+
ExecEnv* exec_env = doris::ExecEnv::GetInstance();
200+
TNetworkAddress master_addr = exec_env->master_info()->network_address;
201+
int64_t stream_load_put_start_time = MonotonicNanos();
202+
RETURN_IF_ERROR(ThriftRpcHelper::rpc<FrontendServiceClient>(
203+
master_addr.hostname, master_addr.port,
204+
[&request, this](FrontendServiceConnection& client) {
205+
client->streamLoadMultiTablePut(_ctx->multi_table_put_result, request);
206+
}));
207+
_ctx->stream_load_put_cost_nanos = MonotonicNanos() - stream_load_put_start_time;
208+
209+
Status plan_status(Status::create(_ctx->multi_table_put_result.status));
210+
if (!plan_status.ok()) {
211+
LOG(WARNING) << "plan streaming load failed. errmsg=" << plan_status << _ctx->brief();
212+
return plan_status;
213+
}
214214

215+
if (_ctx->multi_table_put_result.__isset.params &&
216+
!_ctx->multi_table_put_result.__isset.pipeline_params) {
217+
st = exec_plans(exec_env, _ctx->multi_table_put_result.params);
218+
} else if (!_ctx->multi_table_put_result.__isset.params &&
219+
_ctx->multi_table_put_result.__isset.pipeline_params) {
220+
st = exec_plans(exec_env, _ctx->multi_table_put_result.pipeline_params);
221+
} else {
222+
return Status::Aborted("too many or too few params are set in multi_table_put_result.");
223+
}
224+
if (!st.ok()) {
225+
return st;
226+
}
227+
}
228+
_unplanned_tables.clear();
215229
return st;
216230
}
217231

218232
template <typename ExecParam>
219233
Status MultiTablePipe::exec_plans(ExecEnv* exec_env, std::vector<ExecParam> params) {
220234
// put unplanned pipes into planned pipes and clear unplanned pipes
221-
for (auto& pipe : _unplanned_pipes) {
222-
_ctx->table_list.push_back(pipe.first);
223-
_planned_pipes.emplace(pipe.first, pipe.second);
235+
for (auto& pair : _unplanned_tables) {
236+
_ctx->table_list.push_back(pair.first);
237+
_planned_tables.emplace(pair.first, pair.second);
224238
}
225239
LOG(INFO) << fmt::format("{} tables plan complete, planned table cnt={}, returned plan cnt={}",
226-
_unplanned_pipes.size(), _planned_pipes.size(), params.size())
240+
_unplanned_tables.size(), _planned_tables.size(), params.size())
227241
<< ", ctx: " << _ctx->brief();
228-
_unplanned_pipes.clear();
229242

230243
for (auto& plan : params) {
231244
DBUG_EXECUTE_IF("MultiTablePipe.exec_plans.failed",
232245
{ return Status::Aborted("MultiTablePipe.exec_plans.failed"); });
233246
if (!plan.__isset.table_name ||
234-
_planned_pipes.find(plan.table_name) == _planned_pipes.end()) {
247+
_unplanned_tables.find(plan.table_name) == _unplanned_tables.end()) {
235248
return Status::Aborted("Missing vital param: table_name");
236249
}
237250

238-
if constexpr (std::is_same_v<ExecParam, TExecPlanFragmentParams>) {
239-
RETURN_IF_ERROR(
240-
put_pipe(plan.params.fragment_instance_id, _planned_pipes[plan.table_name]));
241-
LOG(INFO) << "fragment_instance_id=" << print_id(plan.params.fragment_instance_id)
242-
<< " table=" << plan.table_name << ", ctx: " << _ctx->brief();
243-
} else if constexpr (std::is_same_v<ExecParam, TPipelineFragmentParams>) {
244-
auto pipe_id = calculate_pipe_id(plan.query_id, plan.fragment_id);
245-
RETURN_IF_ERROR(put_pipe(pipe_id, _planned_pipes[plan.table_name]));
246-
LOG(INFO) << "pipe_id=" << pipe_id << ", table=" << plan.table_name
247-
<< ", ctx: " << _ctx->brief();
248-
} else {
249-
LOG(WARNING) << "illegal exec param type, need `TExecPlanFragmentParams` or "
250-
"`TPipelineFragmentParams`, will crash"
251-
<< ", ctx: " << _ctx->brief();
252-
CHECK(false);
253-
}
254-
255251
_inflight_cnt++;
256252

257253
RETURN_IF_ERROR(exec_env->fragment_mgr()->exec_plan_fragment(
258-
plan, [this](RuntimeState* state, Status* status) {
254+
plan, [this, plan](RuntimeState* state, Status* status) {
259255
DCHECK(state);
256+
auto pair = _planned_tables.find(plan.table_name);
257+
if (pair == _planned_tables.end()) {
258+
LOG(WARNING) << "failed to get ctx, table: " << plan.table_name;
259+
} else {
260+
doris::ExecEnv::GetInstance()->new_load_stream_mgr()->remove(
261+
pair->second->id);
262+
}
263+
260264
{
261265
std::lock_guard<std::mutex> l(_tablet_commit_infos_lock);
262266
_tablet_commit_infos.insert(_tablet_commit_infos.end(),
@@ -300,12 +304,12 @@ Status MultiTablePipe::exec_plans(ExecEnv* exec_env, std::vector<ExecParam> para
300304
#else
301305
Status MultiTablePipe::request_and_exec_plans() {
302306
// put unplanned pipes into planned pipes
303-
for (auto& pipe : _unplanned_pipes) {
304-
_planned_pipes.emplace(pipe.first, pipe.second);
307+
for (auto& pipe : _unplanned_tables) {
308+
_planned_tables.emplace(pipe.first, pipe.second);
305309
}
306310
LOG(INFO) << fmt::format("{} tables plan complete, planned table cnt={}",
307-
_unplanned_pipes.size(), _planned_pipes.size());
308-
_unplanned_pipes.clear();
311+
_unplanned_tables.size(), _planned_tables.size());
312+
_unplanned_tables.clear();
309313
return Status::OK();
310314
}
311315

@@ -330,35 +334,6 @@ void MultiTablePipe::_handle_consumer_finished() {
330334
_ctx->promise.set_value(_status); // when all done, finish the routine load task
331335
}
332336

333-
Status MultiTablePipe::put_pipe(const TUniqueId& pipe_id,
334-
std::shared_ptr<io::StreamLoadPipe> pipe) {
335-
std::lock_guard<std::mutex> l(_pipe_map_lock);
336-
auto it = _pipe_map.find(pipe_id);
337-
if (it != std::end(_pipe_map)) {
338-
return Status::InternalError("id already exist");
339-
}
340-
_pipe_map.emplace(pipe_id, pipe);
341-
return Status::OK();
342-
}
343-
344-
std::shared_ptr<io::StreamLoadPipe> MultiTablePipe::get_pipe(const TUniqueId& pipe_id) {
345-
std::lock_guard<std::mutex> l(_pipe_map_lock);
346-
auto it = _pipe_map.find(pipe_id);
347-
if (it == std::end(_pipe_map)) {
348-
return {};
349-
}
350-
return it->second;
351-
}
352-
353-
void MultiTablePipe::remove_pipe(const TUniqueId& pipe_id) {
354-
std::lock_guard<std::mutex> l(_pipe_map_lock);
355-
auto it = _pipe_map.find(pipe_id);
356-
if (it != std::end(_pipe_map)) {
357-
_pipe_map.erase(it);
358-
VLOG_NOTICE << "remove stream load pipe: " << pipe_id;
359-
}
360-
}
361-
362337
template Status MultiTablePipe::exec_plans(ExecEnv* exec_env,
363338
std::vector<TExecPlanFragmentParams> params);
364339
template Status MultiTablePipe::exec_plans(ExecEnv* exec_env,

be/src/io/fs/multi_table_pipe.h

+2-9
Original file line numberDiff line numberDiff line change
@@ -60,13 +60,6 @@ class MultiTablePipe : public KafkaConsumerPipe {
6060

6161
void cancel(const std::string& reason) override;
6262

63-
// register <instance id, pipe> pair
64-
Status put_pipe(const TUniqueId& pipe_id, std::shared_ptr<io::StreamLoadPipe> pipe);
65-
66-
std::shared_ptr<io::StreamLoadPipe> get_pipe(const TUniqueId& pipe_id);
67-
68-
void remove_pipe(const TUniqueId& pipe_id);
69-
7063
private:
7164
// parse table name from data
7265
std::string parse_dst_table(const char* data, size_t size);
@@ -82,8 +75,8 @@ class MultiTablePipe : public KafkaConsumerPipe {
8275
void _handle_consumer_finished();
8376

8477
private:
85-
std::unordered_map<std::string /*table*/, KafkaConsumerPipePtr> _planned_pipes;
86-
std::unordered_map<std::string /*table*/, KafkaConsumerPipePtr> _unplanned_pipes;
78+
std::unordered_map<std::string /*table*/, std::shared_ptr<StreamLoadContext>> _planned_tables;
79+
std::unordered_map<std::string /*table*/, std::shared_ptr<StreamLoadContext>> _unplanned_tables;
8780
std::atomic<uint64_t> _unplanned_row_cnt {0}; // trigger plan request when exceed threshold
8881
// inflight count, when it is zero, means consume and all plans is finished
8982
std::atomic<uint64_t> _inflight_cnt {1};

0 commit comments

Comments
 (0)