-
Notifications
You must be signed in to change notification settings - Fork 411
/
MPPTask.cpp
571 lines (522 loc) · 21.8 KB
/
MPPTask.cpp
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
// Copyright 2022 PingCAP, Ltd.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <Common/CPUAffinityManager.h>
#include <Common/FailPoint.h>
#include <Common/ThreadFactory.h>
#include <Common/ThreadManager.h>
#include <Common/TiFlashMetrics.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataStreams/SquashingBlockOutputStream.h>
#include <Flash/Coprocessor/DAGCodec.h>
#include <Flash/Coprocessor/DAGUtils.h>
#include <Flash/CoprocessorHandler.h>
#include <Flash/Mpp/ExchangeReceiver.h>
#include <Flash/Mpp/GRPCReceiverContext.h>
#include <Flash/Mpp/MPPTask.h>
#include <Flash/Mpp/MPPTaskManager.h>
#include <Flash/Mpp/MPPTunnelSet.h>
#include <Flash/Mpp/MinTSOScheduler.h>
#include <Flash/Mpp/Utils.h>
#include <Flash/Statistics/traverseExecutors.h>
#include <Flash/executeQuery.h>
#include <Interpreters/ProcessList.h>
#include <Storages/Transaction/KVStore.h>
#include <Storages/Transaction/TMTContext.h>
#include <fmt/core.h>
#include <chrono>
#include <ext/scope_guard.h>
#include <map>
namespace DB
{
namespace FailPoints
{
extern const char exception_before_mpp_register_non_root_mpp_task[];
extern const char exception_before_mpp_register_root_mpp_task[];
extern const char exception_before_mpp_register_tunnel_for_non_root_mpp_task[];
extern const char exception_before_mpp_register_tunnel_for_root_mpp_task[];
extern const char exception_during_mpp_register_tunnel_for_non_root_mpp_task[];
extern const char exception_during_mpp_write_err_to_tunnel[];
extern const char force_no_local_region_for_mpp_task[];
extern const char random_task_lifecycle_failpoint[];
} // namespace FailPoints
MPPTask::MPPTask(const mpp::TaskMeta & meta_, const ContextPtr & context_)
: context(context_)
, meta(meta_)
, id(meta.start_ts(), meta.task_id())
, log(Logger::get("MPPTask", id.toString()))
, mpp_task_statistics(id, meta.address())
, needed_threads(0)
, schedule_state(ScheduleState::WAITING)
{}
MPPTask::~MPPTask()
{
/// MPPTask maybe destructed by different thread, set the query memory_tracker
/// to current_memory_tracker in the destructor
if (current_memory_tracker != memory_tracker)
current_memory_tracker = memory_tracker;
closeAllTunnels("");
if (schedule_state == ScheduleState::SCHEDULED)
{
/// the threads of this task are not fully freed now, since the BlockIO and DAGContext are not destructed
/// TODO: finish all threads before here, except the current one.
manager.load()->releaseThreadsFromScheduler(needed_threads);
schedule_state = ScheduleState::COMPLETED;
}
LOG_FMT_DEBUG(log, "finish MPPTask: {}", id.toString());
}
void MPPTask::abortTunnels(const String & message, AbortType abort_type)
{
if (abort_type == AbortType::ONCANCELLATION)
{
closeAllTunnels(message);
}
else
{
RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set");
tunnel_set->writeError(message);
}
}
void MPPTask::abortReceivers()
{
if (likely(receiver_set != nullptr))
{
receiver_set->cancel();
}
}
void MPPTask::abortDataStreams(AbortType abort_type)
{
/// When abort type is ONERROR, it means MPPTask already known it meet error, so let the remaining task stop silently to avoid too many useless error message
bool is_kill = abort_type == AbortType::ONCANCELLATION;
context->getProcessList().sendCancelToQuery(context->getCurrentQueryId(), context->getClientInfo().current_user, is_kill);
}
void MPPTask::closeAllTunnels(const String & reason)
{
if (likely(tunnel_set))
tunnel_set->close(reason);
}
void MPPTask::finishWrite()
{
RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set");
tunnel_set->finishWrite();
}
void MPPTask::run()
{
newThreadManager()->scheduleThenDetach(true, "MPPTask", [self = shared_from_this()] { self->runImpl(); });
}
void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request)
{
tunnel_set = std::make_shared<MPPTunnelSet>(log->identifier());
std::chrono::seconds timeout(task_request.timeout());
const auto & exchange_sender = dag_req.root_executor().exchange_sender();
for (int i = 0; i < exchange_sender.encoded_task_meta_size(); ++i)
{
// exchange sender will register the tunnels and wait receiver to found a connection.
mpp::TaskMeta task_meta;
if (unlikely(!task_meta.ParseFromString(exchange_sender.encoded_task_meta(i))))
throw TiFlashException("Failed to decode task meta info in ExchangeSender", Errors::Coprocessor::BadRequest);
bool is_local = context->getSettingsRef().enable_local_tunnel && meta.address() == task_meta.address();
bool is_async = !is_local && context->getSettingsRef().enable_async_server;
MPPTunnelPtr tunnel = std::make_shared<MPPTunnel>(task_meta, task_request.meta(), timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier());
LOG_FMT_DEBUG(log, "begin to register the tunnel {}", tunnel->id());
if (status != INITIALIZING)
throw Exception(fmt::format("The tunnel {} can not be registered, because the task is not in initializing state", tunnel->id()));
tunnel_set->registerTunnel(MPPTaskId{task_meta.start_ts(), task_meta.task_id()}, tunnel);
if (!dag_context->isRootMPPTask())
{
FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_register_tunnel_for_non_root_mpp_task);
}
}
}
void MPPTask::initExchangeReceivers()
{
receiver_set = std::make_shared<MPPReceiverSet>(log->identifier());
traverseExecutors(&dag_req, [&](const tipb::Executor & executor) {
if (executor.tp() == tipb::ExecType::TypeExchangeReceiver)
{
assert(executor.has_executor_id());
const auto & executor_id = executor.executor_id();
// In order to distinguish different exchange receivers.
auto exchange_receiver = std::make_shared<ExchangeReceiver>(
std::make_shared<GRPCReceiverContext>(
executor.exchange_receiver(),
dag_context->getMPPTaskMeta(),
context->getTMTContext().getKVCluster(),
context->getTMTContext().getMPPTaskManager(),
context->getSettingsRef().enable_local_tunnel,
context->getSettingsRef().enable_async_grpc_client),
executor.exchange_receiver().encoded_task_meta_size(),
context->getMaxStreams(),
log->identifier(),
executor_id,
executor.fine_grained_shuffle_stream_count());
if (status != RUNNING)
throw Exception("exchange receiver map can not be initialized, because the task is not in running state");
receiver_set->addExchangeReceiver(executor_id, exchange_receiver);
new_thread_count_of_exchange_receiver += exchange_receiver->computeNewThreadCount();
}
return true;
});
dag_context->setMPPReceiverSet(receiver_set);
}
std::pair<MPPTunnelPtr, String> MPPTask::getTunnel(const ::mpp::EstablishMPPConnectionRequest * request)
{
if (status == CANCELLED)
{
auto err_msg = fmt::format(
"can't find tunnel ({} + {}) because the task is cancelled",
request->sender_meta().task_id(),
request->receiver_meta().task_id());
return {nullptr, err_msg};
}
MPPTaskId receiver_id{request->receiver_meta().start_ts(), request->receiver_meta().task_id()};
RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set");
auto tunnel_ptr = tunnel_set->getTunnelByReceiverTaskId(receiver_id);
if (tunnel_ptr == nullptr)
{
auto err_msg = fmt::format(
"can't find tunnel ({} + {})",
request->sender_meta().task_id(),
request->receiver_meta().task_id());
return {nullptr, err_msg};
}
return {tunnel_ptr, ""};
}
void MPPTask::unregisterTask()
{
auto * manager_ptr = manager.load();
if (manager_ptr != nullptr)
{
LOG_DEBUG(log, "task unregistered");
manager_ptr->unregisterTask(this);
}
else
{
LOG_ERROR(log, "task manager is unset");
}
}
void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request)
{
dag_req = getDAGRequestFromStringWithRetry(task_request.encoded_plan());
TMTContext & tmt_context = context->getTMTContext();
/// MPP task will only use key ranges in mpp::DispatchTaskRequest::regions/mpp::DispatchTaskRequest::table_regions.
/// The ones defined in tipb::TableScan will never be used and can be removed later.
TablesRegionsInfo tables_regions_info = TablesRegionsInfo::create(task_request.regions(), task_request.table_regions(), tmt_context);
LOG_FMT_DEBUG(
log,
"Handling {} regions from {} physical tables in MPP task",
tables_regions_info.regionCount(),
tables_regions_info.tableCount());
// set schema ver and start ts.
auto schema_ver = task_request.schema_ver();
auto start_ts = task_request.meta().start_ts();
context->setSetting("read_tso", start_ts);
context->setSetting("schema_version", schema_ver);
if (unlikely(task_request.timeout() < 0))
{
/// this is only for test
context->setSetting("mpp_task_timeout", static_cast<Int64>(5));
context->setSetting("mpp_task_running_timeout", static_cast<Int64>(10));
}
else
{
context->setSetting("mpp_task_timeout", task_request.timeout());
if (task_request.timeout() > 0)
{
/// in the implementation, mpp_task_timeout is actually the task writing tunnel timeout
/// so make the mpp_task_running_timeout a little bigger than mpp_task_timeout
context->setSetting("mpp_task_running_timeout", task_request.timeout() + 30);
}
}
context->getTimezoneInfo().resetByDAGRequest(dag_req);
bool is_root_mpp_task = false;
const auto & exchange_sender = dag_req.root_executor().exchange_sender();
if (exchange_sender.encoded_task_meta_size() == 1)
{
/// root mpp task always has 1 task_meta because there is only one TiDB
/// node for each mpp query
mpp::TaskMeta task_meta;
if (!task_meta.ParseFromString(exchange_sender.encoded_task_meta(0)))
{
throw TiFlashException("Failed to decode task meta info in ExchangeSender", Errors::Coprocessor::BadRequest);
}
is_root_mpp_task = task_meta.task_id() == -1;
}
dag_context = std::make_unique<DAGContext>(dag_req, task_request.meta(), is_root_mpp_task);
dag_context->log = log;
dag_context->tables_regions_info = std::move(tables_regions_info);
dag_context->tidb_host = context->getClientInfo().current_address.toString();
context->setDAGContext(dag_context.get());
if (dag_context->isRootMPPTask())
{
FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_before_mpp_register_tunnel_for_root_mpp_task);
}
else
{
FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_before_mpp_register_tunnel_for_non_root_mpp_task);
}
// register tunnels
registerTunnels(task_request);
dag_context->tunnel_set = tunnel_set;
// register task.
auto task_manager = tmt_context.getMPPTaskManager();
LOG_FMT_DEBUG(log, "begin to register the task {}", id.toString());
if (dag_context->isRootMPPTask())
{
FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_before_mpp_register_root_mpp_task);
}
else
{
FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_before_mpp_register_non_root_mpp_task);
}
if (!task_manager->registerTask(shared_from_this()))
{
throw TiFlashException(std::string(__PRETTY_FUNCTION__) + ": Failed to register MPP Task", Errors::Coprocessor::BadRequest);
}
mpp_task_statistics.initializeExecutorDAG(dag_context.get());
mpp_task_statistics.logTracingJson();
}
void MPPTask::preprocess()
{
auto start_time = Clock::now();
initExchangeReceivers();
executeQuery(*context);
auto end_time = Clock::now();
dag_context->compile_time_ns = std::chrono::duration_cast<std::chrono::nanoseconds>(end_time - start_time).count();
mpp_task_statistics.setCompileTimestamp(start_time, end_time);
mpp_task_statistics.recordReadWaitIndex(*dag_context);
}
void MPPTask::runImpl()
{
CPUAffinityManager::getInstance().bindSelfQueryThread();
if (!switchStatus(INITIALIZING, RUNNING))
{
LOG_WARNING(log, "task not in initializing state, skip running");
return;
}
Stopwatch stopwatch;
GET_METRIC(tiflash_coprocessor_request_count, type_run_mpp_task).Increment();
GET_METRIC(tiflash_coprocessor_handling_request_count, type_run_mpp_task).Increment();
SCOPE_EXIT({
GET_METRIC(tiflash_coprocessor_handling_request_count, type_run_mpp_task).Decrement();
GET_METRIC(tiflash_coprocessor_request_duration_seconds, type_run_mpp_task).Observe(stopwatch.elapsedSeconds());
});
String err_msg;
try
{
LOG_FMT_INFO(log, "task starts preprocessing");
preprocess();
needed_threads = estimateCountOfNewThreads();
LOG_FMT_DEBUG(log, "Estimate new thread count of query: {} including tunnel_threads: {}, receiver_threads: {}", needed_threads, dag_context->tunnel_set->getRemoteTunnelCnt(), new_thread_count_of_exchange_receiver);
scheduleOrWait();
LOG_FMT_INFO(log, "task starts running");
memory_tracker = current_memory_tracker;
if (status.load() != RUNNING)
{
/// when task is in running state, canceling the task will call sendCancelToQuery to do the cancellation, however
/// if the task is cancelled during preprocess, sendCancelToQuery may just be ignored because the processlist of
/// current task is not registered yet, so need to check the task status explicitly
throw Exception("task not in running state, may be cancelled");
}
mpp_task_statistics.start();
auto from = dag_context->getBlockIO().in;
from->readPrefix();
LOG_DEBUG(log, "begin read ");
while (from->read())
continue;
from->readSuffix();
finishWrite();
const auto & return_statistics = mpp_task_statistics.collectRuntimeStatistics();
LOG_FMT_DEBUG(
log,
"finish write with {} rows, {} blocks, {} bytes",
return_statistics.rows,
return_statistics.blocks,
return_statistics.bytes);
}
catch (...)
{
err_msg = getCurrentExceptionMessage(true, true);
}
if (err_msg.empty())
{
if (switchStatus(RUNNING, FINISHED))
LOG_INFO(log, "finish task");
else
LOG_FMT_WARNING(log, "finish task which is in {} state", taskStatusToString(status));
if (status == FINISHED)
{
// todo when error happens, should try to update the metrics if it is available
if (auto throughput = dag_context->getTableScanThroughput(); throughput.first)
GET_METRIC(tiflash_storage_logical_throughput_bytes).Observe(throughput.second);
auto process_info = context->getProcessListElement()->getInfo();
auto peak_memory = process_info.peak_memory_usage > 0 ? process_info.peak_memory_usage : 0;
GET_METRIC(tiflash_coprocessor_request_memory_usage, type_run_mpp_task).Observe(peak_memory);
mpp_task_statistics.setMemoryPeak(peak_memory);
}
}
else
{
if (status == RUNNING)
{
LOG_FMT_ERROR(log, "task running meets error: {}", err_msg);
/// trim the stack trace to avoid too many useless information in log
trimStackTrace(err_msg);
try
{
handleError(err_msg);
}
catch (...)
{
tryLogCurrentException(log, "Meet error while try to handle error in MPPTask");
}
}
}
LOG_FMT_INFO(log, "task ends, time cost is {} ms.", stopwatch.elapsedMilliseconds());
// unregister flag is only for FailPoint usage, to produce the situation that MPPTask is destructed
// by grpc CancelMPPTask thread;
bool unregister = true;
fiu_do_on(FailPoints::random_task_lifecycle_failpoint, {
if (!err_msg.empty())
unregister = false;
});
if (unregister)
unregisterTask();
mpp_task_statistics.end(status.load(), err_string);
mpp_task_statistics.logTracingJson();
}
void MPPTask::handleError(const String & error_msg)
{
auto * manager_ptr = manager.load();
/// if manager_ptr is not nullptr, it means the task has already been registered,
/// MPPTaskManager::cancelMPPQuery will handle it properly if the query is to be cancelled.
if (manager_ptr == nullptr || !manager_ptr->isQueryToBeCancelled(id.start_ts))
abort(error_msg, AbortType::ONERROR);
}
void MPPTask::abort(const String & message, AbortType abort_type)
{
String abort_type_string;
TaskStatus next_task_status;
switch (abort_type)
{
case AbortType::ONCANCELLATION:
abort_type_string = "ONCANCELLATION";
next_task_status = CANCELLED;
break;
case AbortType::ONERROR:
abort_type_string = "ONERROR";
next_task_status = FAILED;
break;
}
LOG_FMT_WARNING(log, "Begin abort task: {}, abort type: {}", id.toString(), abort_type_string);
while (true)
{
auto previous_status = status.load();
if (previous_status == FINISHED || previous_status == CANCELLED || previous_status == FAILED)
{
LOG_FMT_WARNING(log, "task already in {} state", taskStatusToString(previous_status));
return;
}
else if (previous_status == INITIALIZING && switchStatus(INITIALIZING, next_task_status))
{
err_string = message;
/// if the task is in initializing state, mpp task can return error to TiDB directly,
/// so just close all tunnels here
closeAllTunnels(message);
unregisterTask();
LOG_WARNING(log, "Finish abort task from uninitialized");
return;
}
else if (previous_status == RUNNING && switchStatus(RUNNING, next_task_status))
{
/// abort the components from top to bottom because if bottom components are aborted
/// first, the top components may see an error caused by the abort, which is not
/// the original error
err_string = message;
abortTunnels(message, abort_type);
abortDataStreams(abort_type);
abortReceivers();
scheduleThisTask(ScheduleState::FAILED);
/// runImpl is running, leave remaining work to runImpl
LOG_WARNING(log, "Finish abort task from running");
return;
}
}
}
void MPPTask::cancel(const String & reason)
{
CPUAffinityManager::getInstance().bindSelfQueryThread();
abort(reason, AbortType::ONCANCELLATION);
}
bool MPPTask::switchStatus(TaskStatus from, TaskStatus to)
{
return status.compare_exchange_strong(from, to);
}
void MPPTask::scheduleOrWait()
{
if (!manager.load()->tryToScheduleTask(shared_from_this()))
{
LOG_FMT_INFO(log, "task waits for schedule");
Stopwatch stopwatch;
double time_cost = 0;
{
std::unique_lock lock(schedule_mu);
schedule_cv.wait(lock, [&] { return schedule_state != ScheduleState::WAITING; });
time_cost = stopwatch.elapsedSeconds();
GET_METRIC(tiflash_task_scheduler_waiting_duration_seconds).Observe(time_cost);
if (schedule_state == ScheduleState::EXCEEDED)
{
throw Exception(fmt::format("{} is failed to schedule because of exceeding the thread hard limit in min-tso scheduler after waiting for {}s.", id.toString(), time_cost));
}
else if (schedule_state == ScheduleState::FAILED)
{
throw Exception(fmt::format("{} is failed to schedule because of being cancelled in min-tso scheduler after waiting for {}s.", id.toString(), time_cost));
}
}
LOG_FMT_INFO(log, "task waits for {} s to schedule and starts to run in parallel.", time_cost);
}
}
bool MPPTask::scheduleThisTask(ScheduleState state)
{
std::unique_lock lock(schedule_mu);
if (schedule_state == ScheduleState::WAITING)
{
LOG_FMT_INFO(log, "task is {}.", state == ScheduleState::SCHEDULED ? "scheduled" : " failed to schedule");
schedule_state = state;
schedule_cv.notify_one();
return true;
}
return false;
}
int MPPTask::estimateCountOfNewThreads()
{
if (dag_context == nullptr || dag_context->getBlockIO().in == nullptr || dag_context->tunnel_set == nullptr)
throw Exception("It should not estimate the threads for the uninitialized task" + id.toString());
// Estimated count of new threads from InputStreams(including ExchangeReceiver), remote MppTunnels s.
return dag_context->getBlockIO().in->estimateNewThreadCount() + 1
+ dag_context->tunnel_set->getRemoteTunnelCnt();
}
int MPPTask::getNeededThreads()
{
if (needed_threads == 0)
{
throw Exception(" the needed_threads of task " + id.toString() + " is not initialized!");
}
return needed_threads;
}
bool MPPTask::isScheduled()
{
std::unique_lock lock(schedule_mu);
return schedule_state == ScheduleState::SCHEDULED;
}
} // namespace DB