patch 4.0
This commit is contained in:
@ -18,20 +18,36 @@
|
||||
#include "sql/engine/px/ob_px_sqc_proxy.h"
|
||||
#include "sql/engine/px/ob_sqc_ctx.h"
|
||||
#include "sql/engine/px/datahub/ob_dh_msg_provider.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "storage/tx/ob_trans_service.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
using namespace oceanbase::sql::dtl;
|
||||
|
||||
ObPxSQCProxy::ObPxSQCProxy(ObSqcCtx& sqc_ctx, ObPxRpcInitSqcArgs& arg)
|
||||
: sqc_ctx_(sqc_ctx),
|
||||
sqc_arg_(arg),
|
||||
leader_token_lock_(common::ObLatchIds::PX_WORKER_LEADER_LOCK),
|
||||
first_buffer_cache_(nullptr)
|
||||
{}
|
||||
|
||||
ObPxSQCProxy::ObPxSQCProxy(ObSqcCtx &sqc_ctx,
|
||||
ObPxRpcInitSqcArgs &arg)
|
||||
: sqc_ctx_(sqc_ctx),
|
||||
sqc_arg_(arg),
|
||||
leader_token_lock_(common::ObLatchIds::PX_WORKER_LEADER_LOCK),
|
||||
first_buffer_cache_(nullptr),
|
||||
bloom_filter_ready_(false),
|
||||
bloom_filter_channels_(),
|
||||
bf_ch_set_(),
|
||||
filter_data_(NULL),
|
||||
filter_indexes_(),
|
||||
per_channel_bf_count_(0),
|
||||
filter_channel_idx_(0),
|
||||
bf_compressor_type_(common::ObCompressorType::NONE_COMPRESSOR),
|
||||
sample_msg_()
|
||||
{
|
||||
}
|
||||
|
||||
ObPxSQCProxy::~ObPxSQCProxy()
|
||||
{}
|
||||
{
|
||||
sample_msg_.reset();
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::init()
|
||||
{
|
||||
@ -44,38 +60,45 @@ int ObPxSQCProxy::init()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::link_sqc_qc_channel(ObPxRpcInitSqcArgs& sqc_arg)
|
||||
int ObPxSQCProxy::link_sqc_qc_channel(ObPxRpcInitSqcArgs &sqc_arg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPxSqcMeta& sqc = sqc_arg.sqc_;
|
||||
ObDtlChannel* ch = sqc.get_sqc_channel();
|
||||
// Note: ch has been linked in ObInitSqcP::process() in advance
|
||||
// This is an optimization,
|
||||
// in order to receive the data channel information issued by qc as soon as possible
|
||||
ObPxSqcMeta &sqc = sqc_arg.sqc_;
|
||||
ObDtlChannel *ch = sqc.get_sqc_channel();
|
||||
// 注意:ch 已经提前在 ObInitSqcP::process() 中 link 过了
|
||||
// 这是一个优化,为了能够尽早收到 qc 下发的 data channel 信息
|
||||
if (OB_ISNULL(ch)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail link sqc qc channel", K(sqc), K(ret));
|
||||
} else {
|
||||
(void)sqc_ctx_.msg_loop_.register_channel(*ch);
|
||||
const ObDtlBasicChannel* basic_channel = static_cast<ObDtlBasicChannel*>(sqc.get_sqc_channel());
|
||||
(void) sqc_ctx_.msg_loop_.register_channel(*ch);
|
||||
const ObDtlBasicChannel *basic_channel = static_cast<ObDtlBasicChannel*>(sqc.get_sqc_channel());
|
||||
sqc_ctx_.msg_loop_.set_tenant_id(basic_channel->get_tenant_id());
|
||||
sqc_ctx_.msg_loop_.set_process_query_time(get_process_query_time());
|
||||
LOG_TRACE("register sqc-qc channel", K(sqc));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::setup_loop_proc(ObSqcCtx& sqc_ctx) const
|
||||
int ObPxSQCProxy::setup_loop_proc(ObSqcCtx &sqc_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(sqc_ctx.receive_data_ch_provider_.init())) {
|
||||
LOG_WARN("fail init receive ch provider", K(ret));
|
||||
} else if (OB_FAIL(sqc_ctx.transmit_data_ch_provider_.init())) {
|
||||
LOG_WARN("fail init transmit ch provider", K(ret));
|
||||
} else if (OB_FAIL(sqc_ctx.bf_ch_provider_.init())) {
|
||||
LOG_WARN("fail init bool filter provider", K(ret));
|
||||
} else {
|
||||
(void)sqc_ctx.msg_loop_.register_processor(sqc_ctx.receive_data_ch_msg_proc_)
|
||||
(void)sqc_ctx.msg_loop_
|
||||
.register_processor(sqc_ctx.receive_data_ch_msg_proc_)
|
||||
.register_processor(sqc_ctx.transmit_data_ch_msg_proc_)
|
||||
.register_processor(sqc_ctx.px_bloom_filter_msg_proc_)
|
||||
.register_processor(sqc_ctx.barrier_whole_msg_proc_)
|
||||
.register_processor(sqc_ctx.winbuf_whole_msg_proc_)
|
||||
.register_processor(sqc_ctx.sample_whole_msg_proc_)
|
||||
.register_processor(sqc_ctx.rollup_key_whole_msg_proc_)
|
||||
.register_processor(sqc_ctx.rd_wf_whole_msg_proc_)
|
||||
.register_interrupt_processor(sqc_ctx.interrupt_proc_);
|
||||
}
|
||||
return ret;
|
||||
@ -88,15 +111,23 @@ void ObPxSQCProxy::destroy()
|
||||
// the following unlink actions is not safe is any unregister failure happened
|
||||
LOG_ERROR("fail unregister all channel from msg_loop", KR(ret_unreg));
|
||||
}
|
||||
if (bf_ch_set_.count() > 0) {
|
||||
if (OB_SUCCESS != (ret_unreg = ObPxChannelUtil::unlink_ch_set(bf_ch_set_,
|
||||
nullptr, false))) {
|
||||
LOG_TRACE("fail to unlink bloom filter channel", K(ret_unreg));
|
||||
}
|
||||
bf_ch_set_.reset();
|
||||
}
|
||||
sample_msg_.reset();
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::unlink_sqc_qc_channel(ObPxRpcInitSqcArgs& sqc_arg)
|
||||
int ObPxSQCProxy::unlink_sqc_qc_channel(ObPxRpcInitSqcArgs &sqc_arg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObDtlChannel* ch = NULL;
|
||||
ObPxSqcMeta& sqc = sqc_arg.sqc_;
|
||||
ObDtlChannel *ch = NULL;
|
||||
ObPxSqcMeta &sqc = sqc_arg.sqc_;
|
||||
ch = sqc.get_sqc_channel();
|
||||
ObDtlChannelInfo& ci = sqc.get_sqc_channel_info();
|
||||
ObDtlChannelInfo &ci = sqc.get_sqc_channel_info();
|
||||
|
||||
if (OB_NOT_NULL(ch)) {
|
||||
if (OB_FAIL(ch->flush())) {
|
||||
@ -118,8 +149,8 @@ int ObPxSQCProxy::process_dtl_msg(int64_t timeout_ts)
|
||||
// next loop
|
||||
}
|
||||
|
||||
// If do_process_dtl_msg does not get any messages,
|
||||
// EAGAIN is returned, otherwise SUCC is returned
|
||||
// 如果 do_process_dtl_msg 没有获取到任何消息,
|
||||
// 则返回 EAGAIN,否则返回 SUCC
|
||||
if (OB_EAGAIN == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
@ -133,7 +164,7 @@ int ObPxSQCProxy::do_process_dtl_msg(int64_t timeout_ts)
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(timeout_ts);
|
||||
while (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(sqc_ctx_.msg_loop_.process_one(1000))) {
|
||||
if (OB_FAIL(sqc_ctx_.msg_loop_.process_any(10))) {
|
||||
if (OB_EAGAIN == ret) {
|
||||
LOG_TRACE("no message for sqc, exit", K(ret), K(timeout_ts));
|
||||
} else {
|
||||
@ -144,8 +175,12 @@ int ObPxSQCProxy::do_process_dtl_msg(int64_t timeout_ts)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::get_transmit_data_ch(const int64_t sqc_id, const int64_t task_id, int64_t timeout_ts,
|
||||
ObPxTaskChSet& task_ch_set, ObDtlChTotalInfo** ch_info)
|
||||
int ObPxSQCProxy::get_transmit_data_ch(
|
||||
const int64_t sqc_id,
|
||||
const int64_t task_id,
|
||||
int64_t timeout_ts,
|
||||
ObPxTaskChSet &task_ch_set,
|
||||
ObDtlChTotalInfo **ch_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool need_process_dtl = need_transmit_channel_map_via_dtl();
|
||||
@ -156,15 +191,14 @@ int ObPxSQCProxy::get_transmit_data_ch(const int64_t sqc_id, const int64_t task_
|
||||
ret = process_dtl_msg(timeout_ts);
|
||||
}
|
||||
|
||||
// After receiving all the news, focus on doing your own task
|
||||
// See if the transmit channel map you expect has been received
|
||||
// 当收完所有消息后,再关注做自己的任务
|
||||
// 看看自己期望的 transmit channel map 是否已经收到
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(sqc_ctx_.transmit_data_ch_provider_.get_data_ch_nonblock(
|
||||
sqc_id, task_id, timeout_ts, task_ch_set, ch_info))) {
|
||||
sqc_id, task_id, timeout_ts, task_ch_set, ch_info,
|
||||
sqc_arg_.sqc_.get_qc_addr(), get_process_query_time()))) {
|
||||
if (OB_EAGAIN == ret) {
|
||||
// If there is no message in the provider,
|
||||
// and there is no need to fetch data through dtl,
|
||||
// it means that there is a logic error
|
||||
// 如果 provider 里没有任何消息,同时又判定不需要通过 dtl 取数据,说明存在逻辑错误
|
||||
if (!need_process_dtl) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expect peek data channel succ", K(ret));
|
||||
@ -182,8 +216,12 @@ int ObPxSQCProxy::get_transmit_data_ch(const int64_t sqc_id, const int64_t task_
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::get_receive_data_ch(int64_t child_dfo_id, const int64_t sqc_id, const int64_t task_id,
|
||||
int64_t timeout_ts, ObPxTaskChSet& task_ch_set, ObDtlChTotalInfo* ch_info)
|
||||
int ObPxSQCProxy::get_receive_data_ch(int64_t child_dfo_id,
|
||||
const int64_t sqc_id,
|
||||
const int64_t task_id,
|
||||
int64_t timeout_ts,
|
||||
ObPxTaskChSet &task_ch_set,
|
||||
ObDtlChTotalInfo *ch_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool need_process_dtl = need_receive_channel_map_via_dtl(child_dfo_id);
|
||||
@ -198,21 +236,20 @@ int ObPxSQCProxy::get_receive_data_ch(int64_t child_dfo_id, const int64_t sqc_id
|
||||
}
|
||||
|
||||
LOG_TRACE("process dtl msg done", K(ret));
|
||||
// After receiving all the news, focus on doing your own task
|
||||
// see if the receive channel map you expect has been received
|
||||
// 当收完所有消息后,再关注做自己的任务
|
||||
// 看看自己期望的 receive channel map 是否已经收到
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(sqc_ctx_.receive_data_ch_provider_.get_data_ch_nonblock(
|
||||
child_dfo_id, sqc_id, task_id, timeout_ts, task_ch_set, ch_info))) {
|
||||
child_dfo_id, sqc_id, task_id, timeout_ts, task_ch_set, ch_info,
|
||||
sqc_arg_.sqc_.get_qc_addr(), get_process_query_time()))) {
|
||||
if (OB_EAGAIN == ret) {
|
||||
// If there is no msg in the provider,
|
||||
// and there is no need to fetch data through dtl,
|
||||
// it means that there is a logic error
|
||||
// 如果 provider 里没有任何消息,同时又判定不需要通过 dtl 取数据,说明存在逻辑错误
|
||||
if (!need_process_dtl) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expect peek data channel succ", K(ret));
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("fail peek data channel from ch_provider", K(ret));
|
||||
LOG_WARN("fail peek data channel from ch_provider", K(ret));
|
||||
}
|
||||
} else {
|
||||
LOG_TRACE("SUCC got nonblock receive channel", K(task_ch_set), K(child_dfo_id));
|
||||
@ -222,14 +259,15 @@ int ObPxSQCProxy::get_receive_data_ch(int64_t child_dfo_id, const int64_t sqc_id
|
||||
} else {
|
||||
// follower
|
||||
LOG_TRACE("ready to block wait get_data_ch", K(child_dfo_id));
|
||||
ret =
|
||||
sqc_ctx_.receive_data_ch_provider_.get_data_ch(child_dfo_id, sqc_id, task_id, timeout_ts, task_ch_set, ch_info);
|
||||
ret = sqc_ctx_.receive_data_ch_provider_.get_data_ch(
|
||||
child_dfo_id, sqc_id, task_id, timeout_ts, task_ch_set, ch_info);
|
||||
LOG_TRACE("block wait get_data_ch done", K(child_dfo_id), K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::get_part_ch_map(ObPxPartChInfo& map, int64_t timeout_ts)
|
||||
|
||||
int ObPxSQCProxy::get_part_ch_map(ObPxPartChInfo &map, int64_t timeout_ts)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool need_process_dtl = need_transmit_channel_map_via_dtl();
|
||||
@ -239,20 +277,19 @@ int ObPxSQCProxy::get_part_ch_map(ObPxPartChInfo& map, int64_t timeout_ts)
|
||||
if (need_process_dtl) {
|
||||
ret = process_dtl_msg(timeout_ts);
|
||||
}
|
||||
// After receiving all the news, focus on doing your own task
|
||||
// see if the receive channel map you expect has been received
|
||||
// 当收完所有消息后,再关注做自己的任务
|
||||
// 看看自己期望的 transmit channel map 是否已经收到
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(sqc_ctx_.transmit_data_ch_provider_.get_part_ch_map_nonblock(map, timeout_ts))) {
|
||||
if (OB_FAIL(sqc_ctx_.transmit_data_ch_provider_.get_part_ch_map_nonblock(
|
||||
map, timeout_ts, sqc_arg_.sqc_.get_qc_addr(), get_process_query_time()))) {
|
||||
if (OB_EAGAIN == ret) {
|
||||
// If there is no msg in the provider,
|
||||
// and there is no need to fetch data through dtl,
|
||||
// it means that there is a logic error
|
||||
// 如果 provider 里没有任何消息,同时又判定不需要通过 dtl 取数据,说明存在逻辑错误
|
||||
if (!need_process_dtl) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expect peek data channel succ", K(ret));
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("fail peek data channel from ch_provider", K(ret));
|
||||
LOG_WARN("fail peek data channel from ch_provider", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -267,12 +304,12 @@ int ObPxSQCProxy::get_part_ch_map(ObPxPartChInfo& map, int64_t timeout_ts)
|
||||
int ObPxSQCProxy::report_task_finish_status(int64_t task_idx, int rc)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
auto& tasks = sqc_ctx_.get_tasks();
|
||||
auto &tasks = sqc_ctx_.get_tasks();
|
||||
if (task_idx < 0 || task_idx >= tasks.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid task idx", K(task_idx), K(tasks.count()), K(ret));
|
||||
} else {
|
||||
ObPxTask& task = tasks.at(task_idx);
|
||||
ObPxTask &task = tasks.at(task_idx);
|
||||
if (task.has_result()) {
|
||||
ret = OB_ENTRY_EXIST;
|
||||
LOG_WARN("task finish status already set", K(task), K(task_idx), K(rc), K(ret));
|
||||
@ -287,7 +324,7 @@ int ObPxSQCProxy::report_task_finish_status(int64_t task_idx, int rc)
|
||||
int ObPxSQCProxy::check_task_finish_status(int64_t timeout_ts)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
auto& tasks = sqc_ctx_.get_tasks();
|
||||
auto &tasks = sqc_ctx_.get_tasks();
|
||||
bool all_tasks_finish = true;
|
||||
do {
|
||||
if (timeout_ts < ObTimeUtility::current_time()) {
|
||||
@ -295,9 +332,8 @@ int ObPxSQCProxy::check_task_finish_status(int64_t timeout_ts)
|
||||
break;
|
||||
}
|
||||
all_tasks_finish = true;
|
||||
ARRAY_FOREACH(tasks, idx)
|
||||
{
|
||||
ObPxTask& task = tasks.at(idx);
|
||||
ARRAY_FOREACH(tasks, idx) {
|
||||
ObPxTask &task = tasks.at(idx);
|
||||
if (!task.has_result()) {
|
||||
all_tasks_finish = false;
|
||||
break;
|
||||
@ -305,13 +341,21 @@ int ObPxSQCProxy::check_task_finish_status(int64_t timeout_ts)
|
||||
}
|
||||
ObSqcLeaderTokenGuard guard(leader_token_lock_);
|
||||
if (guard.hold_token()) {
|
||||
// If there are still tasks that have not been completed, try to receive dtl messages
|
||||
// Pay special attention, there may not be any DTL messages to receive at this time,
|
||||
// and the channel information of the task has been collected.
|
||||
// The reason for trying process_dtl_msg here is because
|
||||
// when the root thread calls check_task_finish_status,
|
||||
// there may also be slave threads waiting in get_data_ch, which need help from someone.
|
||||
// 如果还有 task 没有完成,则尝试收取 dtl 消息
|
||||
// 要特别注意,此时可能并没有什么 DTL 消息要
|
||||
// 收了,task 的 channel 信息已经收取完毕。
|
||||
// 这里之所以要尝试一次 process_dtl_msg 是因为
|
||||
// 在 root 线程调用 check_task_finish_status
|
||||
// 时,还可能存在 slave 线程等在 get_data_ch 中,
|
||||
// 需要人帮助推进。
|
||||
//
|
||||
// FIXME: 这段代码可能引入额外代价,白等若干毫秒
|
||||
// 更精细的控制方式是,先判断是否所有预期的消息都已经
|
||||
// 收到并处理,如果是,则跳过 process_dtl_msg 步骤,
|
||||
// 如果不是,则的确需要执行 process_dtl_msg
|
||||
// 暂时总假设 all_ctrl_msg_received = false
|
||||
bool all_ctrl_msg_received = false;
|
||||
|
||||
if (!all_tasks_finish && !all_ctrl_msg_received) {
|
||||
if (OB_FAIL(process_dtl_msg(timeout_ts))) {
|
||||
if (OB_EAGAIN != ret) {
|
||||
@ -321,7 +365,7 @@ int ObPxSQCProxy::check_task_finish_status(int64_t timeout_ts)
|
||||
}
|
||||
} else {
|
||||
// TODO: wait 100us
|
||||
usleep(1000);
|
||||
ob_usleep(1000);
|
||||
}
|
||||
} while (!all_tasks_finish);
|
||||
|
||||
@ -334,72 +378,83 @@ int ObPxSQCProxy::check_task_finish_status(int64_t timeout_ts)
|
||||
int ObPxSQCProxy::report(int end_ret) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPxRpcInitSqcArgs& sqc_arg = sqc_arg_;
|
||||
ObSqcCtx& sqc_ctx = sqc_ctx_;
|
||||
ObPxSqcMeta& sqc = sqc_arg.sqc_;
|
||||
ObPxRpcInitSqcArgs &sqc_arg = sqc_arg_;
|
||||
ObSqcCtx &sqc_ctx = sqc_ctx_;
|
||||
ObPxSqcMeta &sqc = sqc_arg.sqc_;
|
||||
ObPxFinishSqcResultMsg finish_msg;
|
||||
int64_t affected_rows = 0;
|
||||
// If any task fails, it means all tasks have failed
|
||||
// The first version does not support retry
|
||||
// 任意一个 task 失败,则意味着全部 task 失败
|
||||
// 第一版暂不支持重试
|
||||
int sqc_ret = OB_SUCCESS;
|
||||
auto& tasks = sqc_ctx.get_tasks();
|
||||
auto &tasks = sqc_ctx.get_tasks();
|
||||
update_error_code(sqc_ret, end_ret);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tasks.count(); ++i) {
|
||||
ObPxTask& task = tasks.at(i);
|
||||
ObSQLSessionInfo *session = NULL;
|
||||
CK(OB_NOT_NULL(sqc_arg.exec_ctx_) &&
|
||||
OB_NOT_NULL(session = GET_MY_SESSION(*sqc_arg.exec_ctx_)));
|
||||
for (int64_t i = 0; i < tasks.count(); ++i) {
|
||||
// overwrite ret
|
||||
ObPxTask &task = tasks.at(i);
|
||||
update_error_code(sqc_ret, task.get_result());
|
||||
(void)finish_msg.task_monitor_info_array_.push_back(task.get_task_monitor_info());
|
||||
affected_rows += task.get_affected_rows();
|
||||
finish_msg.dml_row_info_.add_px_dml_row_info(task.dml_row_info_);
|
||||
finish_msg.temp_table_id_ = task.temp_table_id_;
|
||||
if (OB_NOT_NULL(session)) {
|
||||
transaction::ObTxDesc *&sqc_tx_desc = session->get_tx_desc();
|
||||
transaction::ObTxDesc *&task_tx_desc = tasks.at(i).get_tx_desc();
|
||||
if (OB_NOT_NULL(task_tx_desc)) {
|
||||
if (OB_NOT_NULL(sqc_tx_desc)) {
|
||||
(void)MTL(transaction::ObTransService*)->merge_tx_state(*sqc_tx_desc, *task_tx_desc);
|
||||
(void)MTL(transaction::ObTransService*)->release_tx(*task_tx_desc);
|
||||
} else {
|
||||
sqc_tx_desc = task_tx_desc;
|
||||
}
|
||||
task_tx_desc = NULL;
|
||||
}
|
||||
}
|
||||
OZ(append(finish_msg.interm_result_ids_, task.interm_result_ids_));
|
||||
}
|
||||
if (OB_SUCCESS != ret && OB_SUCCESS == sqc_ret) {
|
||||
sqc_ret = ret;
|
||||
}
|
||||
finish_msg.sqc_affected_rows_ = affected_rows;
|
||||
finish_msg.sqc_id_ = sqc.get_sqc_id();
|
||||
finish_msg.dfo_id_ = sqc.get_dfo_id();
|
||||
finish_msg.rc_ = sqc_ret;
|
||||
// Rewrite the error code so that the scheduler can wait for the remote schema to refresh and try again
|
||||
// 重写错误码,使得scheduler端能等待远端schema刷新并重试
|
||||
if (OB_SUCCESS != sqc_ret && is_schema_error(sqc_ret)) {
|
||||
finish_msg.rc_ = OB_ERR_WAIT_REMOTE_SCHEMA_REFRESH;
|
||||
} else if (common::OB_INVALID_ID != sqc_ctx.get_temp_table_id()) {
|
||||
finish_msg.temp_table_id_ = sqc_ctx.get_temp_table_id();
|
||||
if (OB_FAIL(finish_msg.interm_result_ids_.assign(sqc_ctx.interm_result_ids_))) {
|
||||
LOG_WARN("failed to assgin to interm result ids.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
if (OB_NOT_NULL(session)
|
||||
&& GSCHEMASERVICE.is_schema_error_need_retry(NULL, session->get_effective_tenant_id())) {
|
||||
finish_msg.rc_ = OB_ERR_REMOTE_SCHEMA_NOT_FULL;
|
||||
} else {
|
||||
finish_msg.rc_ = OB_ERR_WAIT_REMOTE_SCHEMA_REFRESH;
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
|
||||
// If session is null, rc will not be SUCCESS,
|
||||
// and it does not matter if trans_result is not set
|
||||
ObSQLSessionInfo* session = NULL;
|
||||
if (OB_ISNULL(sqc_arg.exec_ctx_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("null exec ctx", K(ret));
|
||||
} else if (OB_ISNULL(session = GET_MY_SESSION(*sqc_arg.exec_ctx_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("deserialized exec ctx without phy plan session set. Unexpected", K(ret));
|
||||
} else {
|
||||
// 如果 session 为 null,rc 不会为 SUCCESS,没有设置 trans_result 也无妨
|
||||
if (OB_NOT_NULL(session) && OB_NOT_NULL(session->get_tx_desc())) {
|
||||
// overwrite ret
|
||||
if (OB_FAIL(finish_msg.get_trans_result().merge_result(session->get_trans_result()))) {
|
||||
LOG_WARN("fail merge trans result",
|
||||
K(ret),
|
||||
"msg_trans_result",
|
||||
finish_msg.get_trans_result(),
|
||||
"session_trans_result",
|
||||
session->get_trans_result());
|
||||
if (OB_FAIL(MTL(transaction::ObTransService*)
|
||||
->get_tx_exec_result(*session->get_tx_desc(),
|
||||
finish_msg.get_trans_result()))) {
|
||||
LOG_WARN("fail get tx result", K(ret),
|
||||
"msg_trans_result", finish_msg.get_trans_result(),
|
||||
"tx_desc", *session->get_tx_desc());
|
||||
finish_msg.rc_ = (OB_SUCCESS != sqc_ret) ? sqc_ret : ret;
|
||||
} else {
|
||||
LOG_DEBUG("report trans_result",
|
||||
"msg_trans_result",
|
||||
finish_msg.get_trans_result(),
|
||||
"session_trans_result",
|
||||
session->get_trans_result());
|
||||
LOG_TRACE("report trans_result",
|
||||
"msg_trans_result", finish_msg.get_trans_result(),
|
||||
"tx_desc", *session->get_tx_desc());
|
||||
}
|
||||
}
|
||||
|
||||
ObDtlChannel* ch = sqc.get_sqc_channel();
|
||||
ObDtlChannel *ch = sqc.get_sqc_channel();
|
||||
// overwrite ret
|
||||
if (OB_ISNULL(ch)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("empty channel", K(sqc), K(ret));
|
||||
} else if (OB_FAIL(ch->send(finish_msg, sqc_arg.exec_ctx_->get_physical_plan_ctx()->get_timeout_timestamp()))) {
|
||||
} else if (OB_FAIL(ch->send(finish_msg,
|
||||
sqc_arg.exec_ctx_->get_physical_plan_ctx()->get_timeout_timestamp()))) {
|
||||
// 尽力而为,如果 push 失败就由其它机制处理
|
||||
LOG_WARN("fail push data to channel", K(ret));
|
||||
} else if (OB_FAIL(ch->flush())) {
|
||||
LOG_WARN("fail flush dtl data", K(ret));
|
||||
@ -408,36 +463,73 @@ int ObPxSQCProxy::report(int end_ret) const
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObPxSQCProxy::get_self_dfo_key(ObDtlDfoKey& key)
|
||||
void ObPxSQCProxy::get_self_sqc_info(ObDtlSqcInfo &sqc_info)
|
||||
{
|
||||
ObPxSqcMeta& sqc = sqc_arg_.sqc_;
|
||||
ObPxSqcMeta &sqc = sqc_arg_.sqc_;
|
||||
sqc_info.set(sqc.get_qc_id(), sqc.get_dfo_id(), sqc.get_sqc_id());
|
||||
}
|
||||
|
||||
void ObPxSQCProxy::get_self_dfo_key(ObDtlDfoKey &key)
|
||||
{
|
||||
ObPxSqcMeta &sqc = sqc_arg_.sqc_;
|
||||
key.set(sqc.get_qc_server_id(), sqc.get_px_sequence_id(), sqc.get_qc_id(), sqc.get_dfo_id());
|
||||
}
|
||||
|
||||
void ObPxSQCProxy::get_parent_dfo_key(ObDtlDfoKey& key)
|
||||
void ObPxSQCProxy::get_parent_dfo_key(ObDtlDfoKey &key)
|
||||
{
|
||||
ObPxSqcMeta& sqc = sqc_arg_.sqc_;
|
||||
key.set(sqc.get_qc_server_id(), sqc.get_px_sequence_id(), sqc.get_qc_id(), sqc.get_parent_dfo_id());
|
||||
ObPxSqcMeta &sqc = sqc_arg_.sqc_;
|
||||
key.set(sqc.get_qc_server_id(), sqc.get_px_sequence_id(),
|
||||
sqc.get_qc_id(), sqc.get_parent_dfo_id());
|
||||
}
|
||||
|
||||
bool ObPxSQCProxy::need_transmit_channel_map_via_dtl()
|
||||
{
|
||||
ObPxSqcMeta& sqc = sqc_arg_.sqc_;
|
||||
ObPxSqcMeta &sqc = sqc_arg_.sqc_;
|
||||
return !sqc.is_prealloc_transmit_channel();
|
||||
}
|
||||
|
||||
bool ObPxSQCProxy::need_receive_channel_map_via_dtl(int64_t child_dfo_id)
|
||||
{
|
||||
|
||||
ObPxSqcMeta& sqc = sqc_arg_.sqc_;
|
||||
ObPxSqcMeta &sqc = sqc_arg_.sqc_;
|
||||
bool via_sqc = false;
|
||||
if (sqc.is_prealloc_transmit_channel()) {
|
||||
via_sqc = (sqc.get_receive_channel_msg().get_child_dfo_id() == child_dfo_id);
|
||||
via_sqc = (sqc.get_receive_channel_msg().get_child_dfo_id() == child_dfo_id);
|
||||
}
|
||||
return !via_sqc;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::get_whole_msg_provider(uint64_t op_id, ObPxDatahubDataProvider*& provider)
|
||||
int ObPxSQCProxy::get_bloom_filter_ch(
|
||||
ObPxBloomFilterChSet &ch_set,
|
||||
int64_t &sqc_count,
|
||||
int64_t timeout_ts,
|
||||
bool is_transmit)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t wait_count = 0;
|
||||
do {
|
||||
ObSqcLeaderTokenGuard guard(leader_token_lock_);
|
||||
if (guard.hold_token()) {
|
||||
ret = process_dtl_msg(timeout_ts);
|
||||
LOG_DEBUG("process dtl bf msg done", K(ret));
|
||||
}
|
||||
if (OB_SUCCESS == ret || OB_EAGAIN == ret) {
|
||||
ret = sqc_ctx_.bf_ch_provider_.get_data_ch_nonblock(
|
||||
ch_set, sqc_count, timeout_ts, is_transmit, sqc_arg_.sqc_.get_qc_addr(),
|
||||
get_process_query_time());
|
||||
}
|
||||
if (OB_EAGAIN == ret) {
|
||||
if(0 == (++wait_count) % 100) {
|
||||
LOG_TRACE("try to get bf data channel repeatly", K(wait_count), K(ret));
|
||||
}
|
||||
// wait 1000us
|
||||
ob_usleep(1000);
|
||||
}
|
||||
} while (OB_EAGAIN == ret);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::get_whole_msg_provider(uint64_t op_id, ObPxDatahubDataProvider *&provider)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(sqc_ctx_.get_whole_msg_provider(op_id, provider))) {
|
||||
@ -445,3 +537,85 @@ int ObPxSQCProxy::get_whole_msg_provider(uint64_t op_id, ObPxDatahubDataProvider
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::generate_filter_indexes(
|
||||
int64_t each_group_size,
|
||||
int64_t channel_count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(filter_data_) || channel_count <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("filter data is null", K(ret));
|
||||
} else {
|
||||
int64_t send_size = GCONF._send_bloom_filter_size * 125;
|
||||
int64_t filter_len = filter_data_->filter_.get_bits_array_length();
|
||||
int64_t count = ceil(filter_len / (double)send_size);
|
||||
int64_t start_idx = 0, end_idx = 0;
|
||||
int64_t group_channel_count = each_group_size > channel_count ?
|
||||
channel_count : each_group_size;
|
||||
BloomFilterIndex filter_index;
|
||||
for (int i = 0; OB_SUCC(ret) && i < count; ++i) {
|
||||
start_idx = i * send_size;
|
||||
end_idx = (i + 1) * send_size;
|
||||
if (start_idx >= filter_len) {
|
||||
start_idx = filter_len - 1;
|
||||
}
|
||||
if (end_idx >= filter_len) {
|
||||
end_idx = filter_len - 1;
|
||||
}
|
||||
filter_index.begin_idx_ = start_idx;
|
||||
filter_index.end_idx_ = end_idx;
|
||||
int64_t group_count = ceil((double)channel_count / group_channel_count);
|
||||
int64_t start_channel = ObRandom::rand(0, group_count - 1);
|
||||
start_channel *= group_channel_count;
|
||||
int pos = 0;
|
||||
for (int j = start_channel; OB_SUCC(ret) &&
|
||||
j < start_channel + channel_count;
|
||||
j += group_channel_count) {
|
||||
pos = (j >= channel_count ? j - channel_count : j);
|
||||
pos = (pos / group_channel_count) * group_channel_count;
|
||||
filter_index.channel_ids_.reset();
|
||||
if (pos + group_channel_count > channel_count) {
|
||||
filter_index.channel_id_ = (i % (channel_count - pos)) + pos;
|
||||
} else {
|
||||
filter_index.channel_id_ = (i % group_channel_count) + pos;
|
||||
}
|
||||
for (int k = pos; OB_SUCC(ret) && k < channel_count && k < pos + group_channel_count; ++k) {
|
||||
OZ(filter_index.channel_ids_.push_back(k));
|
||||
}
|
||||
OZ(filter_indexes_.push_back(filter_index));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPxSQCProxy::make_sqc_sample_piece_msg(ObDynamicSamplePieceMsg &msg, bool &finish)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (msg.sample_type_ == HEADER_INPUT_SAMPLE && sample_msg_.row_stores_.empty()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected row sotres", K(ret));
|
||||
} else if (OB_FAIL(sample_msg_.merge_piece_msg(
|
||||
sqc_ctx_.get_task_count(),
|
||||
msg,
|
||||
finish))) {
|
||||
LOG_WARN("fail to merge piece msg", K(ret));
|
||||
} else if (finish) {
|
||||
sample_msg_.expect_range_count_ = msg.expect_range_count_;
|
||||
sample_msg_.dfo_id_ = msg.dfo_id_;
|
||||
sample_msg_.op_id_ = msg.op_id_;
|
||||
sample_msg_.sample_type_ = msg.sample_type_;
|
||||
OZ(sample_msg_.tablet_ids_.assign(msg.tablet_ids_));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObPxSQCProxy::get_process_query_time()
|
||||
{
|
||||
int64_t res = 0;
|
||||
if (OB_NOT_NULL(sqc_arg_.exec_ctx_) && OB_NOT_NULL(sqc_arg_.exec_ctx_->get_my_session())) {
|
||||
res = sqc_arg_.exec_ctx_->get_my_session()->get_process_query_time();
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user