init push

This commit is contained in:
oceanbase-admin
2021-05-31 22:56:52 +08:00
commit cea7de1475
7020 changed files with 5689869 additions and 0 deletions

View File

@ -0,0 +1,40 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_ADDRS_PROVIDER_
#define OCEANBASE_SQL_EXECUTOR_OB_ADDRS_PROVIDER_
#include "lib/container/ob_iarray.h"
#include "lib/net/ob_addr.h"
namespace oceanbase {
namespace sql {
class ObAddrsProvider {
public:
enum {
INVALID_PROVIDER = 0,
RANDOM_PROVIDER = 1,
};
public:
ObAddrsProvider()
{}
virtual ~ObAddrsProvider()
{}
public:
virtual int select_servers(int64_t select_count, common::ObIArray<common::ObAddr>& servers) = 0;
virtual int64_t to_string(char* buf, const int64_t buf_len) const = 0;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_ADDRS_PROVIDER_ */

View File

@ -0,0 +1,78 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_addrs_provider_factory.h"
#include "sql/executor/ob_random_addrs_provider.h"
#include "sql/engine/ob_exec_context.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObAddrsProviderFactory::ObAddrsProviderFactory() : store_()
{}
ObAddrsProviderFactory::~ObAddrsProviderFactory()
{
for (int64_t i = 0; i < store_.count(); ++i) {
ObAddrsProvider* ap = store_.at(i);
if (OB_LIKELY(NULL != ap)) {
ap->~ObAddrsProvider();
}
}
}
void ObAddrsProviderFactory::reset()
{
for (int64_t i = 0; i < store_.count(); ++i) {
ObAddrsProvider* ap = store_.at(i);
if (OB_LIKELY(NULL != ap)) {
ap->~ObAddrsProvider();
}
}
store_.reset();
}
int ObAddrsProviderFactory::create(ObExecContext& exec_ctx, int provider_type, ObAddrsProvider*& servers_provider)
{
int ret = OB_SUCCESS;
ObIAllocator& allocator = exec_ctx.get_allocator();
void* ptr = NULL;
switch (provider_type) {
case ObAddrsProvider::RANDOM_PROVIDER: {
ObAddrsProvider* ap = NULL;
ptr = allocator.alloc(sizeof(ObRandomAddrsProvider));
if (OB_ISNULL(ptr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc ObRandomAddrsProvider", K(ret));
} else if (OB_ISNULL(ap = new (ptr) ObRandomAddrsProvider())) {
LOG_WARN("fail to new ObRandomAddrsProvider", K(ret));
} else if (OB_FAIL(store_.push_back(ap))) {
LOG_WARN("fail to push back ObAddrsProvider ptr to store", K(ret));
ap->~ObAddrsProvider();
} else {
servers_provider = ap;
}
break;
}
default: {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("unexpected provider type", K(ret), K(provider_type));
break;
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,38 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_ADDRS_PROVIDER_FACTORY_
#define OCEANBASE_SQL_EXECUTOR_OB_ADDRS_PROVIDER_FACTORY_
#include "lib/allocator/ob_allocator.h"
#include "lib/container/ob_se_array.h"
namespace oceanbase {
namespace sql {
class ObAddrsProvider;
class ObExecContext;
class ObAddrsProviderFactory {
public:
ObAddrsProviderFactory();
virtual ~ObAddrsProviderFactory();
void reset();
int create(ObExecContext& exec_ctx, int provider_type, ObAddrsProvider*& servers_provider);
private:
common::ObSEArray<ObAddrsProvider*, 4> store_;
private:
DISALLOW_COPY_AND_ASSIGN(ObAddrsProviderFactory);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_ADDRS_PROVIDER_FACTORY_ */

View File

@ -0,0 +1,448 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "ob_bkgd_dist_task.h"
#include "sql/executor/ob_executor_rpc_processor.h"
#include "lib/time/ob_time_utility.h"
#include "lib/stat/ob_session_stat.h"
#include "sql/executor/ob_determinate_task_transmit.h"
#include "sql/engine/dml/ob_table_append_local_sort_data.h"
#include "share/schema/ob_multi_version_schema_service.h"
#include "observer/ob_server_struct.h"
#include "lib/utility/ob_tracepoint.h"
namespace oceanbase {
namespace sql {
using namespace common;
using namespace share;
ObBKGDDistTaskDag::ObBKGDDistTaskDag()
: ObIDag(DAG_TYPE_SQL_BUILD_INDEX, DAG_PRIO_CREATE_INDEX), tenant_id_(OB_INVALID_ID), scheduler_id_(0)
{}
ObBKGDDistTaskDag::~ObBKGDDistTaskDag()
{}
int ObBKGDDistTaskDag::init(const uint64_t tenant_id, const ObTaskID& task_id, const uint64_t scheduler_id)
{
int ret = OB_SUCCESS;
if (OB_INVALID_ARGUMENT == tenant_id || !task_id.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(task_id));
} else {
tenant_id_ = tenant_id;
task_id_ = task_id;
scheduler_id_ = scheduler_id;
}
return ret;
}
bool ObBKGDDistTaskDag::operator==(const ObIDag& other) const
{
bool equal = false;
if (this == &other) {
equal = true;
} else {
if (get_type() == other.get_type()) {
const ObBKGDDistTaskDag& o = static_cast<const ObBKGDDistTaskDag&>(other);
equal = (tenant_id_ == o.tenant_id_ && task_id_ == o.task_id_);
}
}
return equal;
}
int64_t ObBKGDDistTaskDag::hash() const
{
// task_id_ is unique, %tenant_id_ is not used
return task_id_.hash();
}
int ObBKGDDistTaskDag::fill_comment(char* buf, const int64_t len) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(
databuff_printf(buf, len, "build index task, tenant_id=%lu, task_id=%s", tenant_id_, to_cstring(task_id_)))) {
LOG_WARN("data buffer print failed", K(ret));
}
return ret;
}
int64_t ObBKGDDistTaskDag::get_compat_mode() const
{
int ret = OB_SUCCESS;
ObWorker::CompatMode compat_mode = ObWorker::CompatMode::MYSQL;
FETCH_ENTITY(TENANT_SPACE, tenant_id_)
{
compat_mode = THIS_WORKER.get_compatibility_mode();
}
return static_cast<int64_t>(compat_mode);
}
ObBKGDDistTask::ObBKGDDistTask()
: ObITask(TASK_TYPE_SQL_BUILD_INDEX),
abs_timeout_us_(0),
create_time_us_(0),
task_allocator_(ObModIds::OB_CS_BUILD_INDEX)
{}
ObBKGDDistTask::~ObBKGDDistTask()
{
if (!serialized_task_.empty()) {
task_allocator_.free(serialized_task_.ptr());
}
}
class ObBKGDDistTask::ObDistTaskProcessor : public ObDistExecuteBaseP {
public:
ObDistTaskProcessor(ObBKGDDistTask& task) : ObDistExecuteBaseP(GCTX, false /* do not send result */), task_(task)
{}
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp)
{
UNUSED(is_first);
exec_timestamp.rpc_send_ts_ = task_.create_time_us_;
exec_timestamp.receive_ts_ = task_.create_time_us_;
exec_timestamp.enter_queue_ts_ = task_.create_time_us_;
exec_timestamp.run_ts_ = get_process_timestamp();
exec_timestamp.before_process_ts_ = get_process_timestamp();
exec_timestamp.single_process_ts_ = get_single_process_timestamp();
exec_timestamp.process_executor_ts_ = get_exec_start_timestamp();
exec_timestamp.executor_end_ts_ = get_exec_end_timestamp();
}
int process_task(ObTask& task, ObTaskCompleteEvent& event)
{
int ret = OB_SUCCESS;
if (OB_FAIL(param_preprocess(task))) {
LOG_WARN("param pre-process failed", K(ret));
} else if (OB_FAIL(execute_dist_plan(task, event))) {
LOG_WARN("execute distribute plan failed", K(ret), K(task));
}
return ret;
}
private:
ObBKGDDistTask& task_;
};
int ObBKGDDistTask::init(const common::ObAddr& addr, const common::ObString& task, const int64_t abs_timeout_us)
{
int ret = OB_SUCCESS;
ObBKGDDistTaskDag* dag = static_cast<ObBKGDDistTaskDag*>(get_dag());
if (!addr.is_valid() || task.empty()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret));
} else if (abs_timeout_us < ObTimeUtility::current_time()) {
ret = OB_TIMEOUT;
LOG_WARN("task already timeout", K(ret));
} else if (OB_ISNULL(dag)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("dag is NULL", K(ret));
} else if (OB_FAIL(ob_write_string(task_allocator_, task, serialized_task_))) {
LOG_WARN("copy string failed", K(ret));
} else {
return_addr_ = addr;
abs_timeout_us_ = abs_timeout_us;
create_time_us_ = ObTimeUtility::current_time();
if (NULL != ObCurTraceId::get_trace_id()) {
trace_id_ = *ObCurTraceId::get_trace_id();
}
}
return ret;
}
int ObBKGDDistTask::get_index_tid(const ObTask& task, uint64_t& tid) const
{
int ret = OB_SUCCESS;
// Only too build task execute in background:
//
// 1. scan index data, get index table id from transmit
//
// PHY_DETERMINATE_TASK_TRANSMIT
// PHY_UK_ROW_TRANSFORM (for uniq index)
// PHY_TABLE_SCAN_WITH_CHECKSUM
//
// 2. build index table macro, get index table id from PHY_APPEND_LOCAL_SORT_DATA
//
// PHY_DETERMINATE_TASK_TRANSMIT
// PHY_APPEND_LOCAL_SORT_DATA
// PHY_SORT
// PHY_TASK_ORDER_RECEIVE
//
ObPhyOperator* root = task.get_des_phy_plan().get_main_query();
ObPhyOperator* child = NULL;
if (NULL == root || PHY_DETERMINATE_TASK_TRANSMIT != root->get_type()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("main query is NULL or unexpected type", K(ret), KP(root));
} else {
const ObDeterminateTaskTransmit* transmit = static_cast<ObDeterminateTaskTransmit*>(root);
if (NULL == (child = transmit->get_child(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("transmit has no child", K(ret));
} else if (PHY_APPEND_LOCAL_SORT_DATA == child->get_type() || PHY_UK_ROW_TRANSFORM == child->get_type() ||
PHY_TABLE_SCAN_WITH_CHECKSUM == child->get_type()) {
if (PHY_APPEND_LOCAL_SORT_DATA == child->get_type()) {
tid = static_cast<const ObTableAppendLocalSortData*>(child)->get_table_id();
} else {
tid = transmit->get_repartition_table_id();
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected operator type", K(ret), K(child->get_type()));
}
}
return ret;
}
class ObExtraIndexBuildCheck : public ObIExtraStatusCheck {
public:
ObExtraIndexBuildCheck(const uint64_t index_tid) : index_tid_(index_tid), last_check_time_(0)
{}
const char* name() const override
{
return "index build check";
}
int check() const override
{
int ret = OB_SUCCESS;
const int64_t CHECK_INTERVAL = 1000000; // 1 second
int64_t cur_time = ObTimeUtil::fast_current_time();
if (cur_time - last_check_time_ > CHECK_INTERVAL) {
ret = do_check();
if (OB_SUCC(ret)) {
last_check_time_ = cur_time;
}
}
return ret;
}
int do_check() const
{
int ret = OB_SUCCESS;
schema::ObSchemaGetterGuard schema_guard;
const schema::ObTableSchema* table_schema = NULL;
if (NULL == GCTX.schema_service_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL schema service", K(ret));
} else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(extract_tenant_id(index_tid_), schema_guard))) {
if (OB_TENANT_NOT_EXIST == ret) {
ret = OB_CANCELED;
LOG_INFO("tenant not exist", K(ret), K(index_tid_));
} else {
LOG_WARN("get tenant schema guard failed", K(ret));
}
} else if (OB_FAIL(schema_guard.get_table_schema(index_tid_, table_schema))) {
LOG_WARN("get table schema failed", K(ret));
} else if (NULL == table_schema) {
ret = OB_CANCELED;
LOG_INFO("index table not exist", K(ret), K(index_tid_));
} else {
if (table_schema->is_final_invalid_index()) {
ret = OB_CANCELED;
LOG_INFO("index table is in final status or droped",
K(ret),
K(index_tid_),
"index_status",
table_schema->get_index_status(),
"is_drop_index",
table_schema->is_drop_index());
}
}
return ret;
}
private:
uint64_t index_tid_;
mutable int64_t last_check_time_;
};
int ObBKGDDistTask::process()
{
int ret = OB_SUCCESS;
if (NULL != ObCurTraceId::get_trace_id()) {
*ObCurTraceId::get_trace_id() = trace_id_;
}
const int64_t worker_abs_timeout_bak = THIS_WORKER.get_timeout_ts();
ObBKGDDistTaskDag* dag = static_cast<ObBKGDDistTaskDag*>(get_dag());
if (OB_ISNULL(dag)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("dag is NULL", K(ret));
} else if (OB_ISNULL(GCTX.executor_rpc_) || OB_ISNULL(GCTX.executor_rpc_->get_proxy())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("rpc proxy is NULL", K(ret));
} else {
// setup memory context for plan deserialize
lib::ContextParam param;
param.set_mem_attr(dag->get_tenant_id(), ObModIds::OB_REQ_DESERIALIZATION, ObCtxIds::DEFAULT_CTX_ID);
FETCH_ENTITY(TENANT_SPACE, dag->get_tenant_id())
{
CREATE_WITH_TEMP_CONTEXT(param)
{
ObTask task;
ObDistTaskProcessor processor(*this);
int64_t pos = 0;
ObPhysicalPlanCtx* plan_ctx = NULL;
ObSQLSessionInfo* session = NULL;
LOG_INFO("begin process background build index task", K(ret), K(dag->get_task_id()));
ObBKGDTaskCompleteArg res;
res.task_id_ = dag->get_task_id();
res.scheduler_id_ = dag->get_scheduler_id();
uint64_t index_tid = OB_INVALID_ID;
if (OB_FAIL(processor.init(task))) {
LOG_WARN("processor init failed", K(ret));
} else if (OB_FAIL(task.deserialize(serialized_task_.ptr(), serialized_task_.length(), pos))) {
LOG_WARN("task deserialize failed", K(ret));
} else if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(processor.get_exec_ctx()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("plan execute context is NULL", K(ret));
} else if (OB_ISNULL(session = processor.get_exec_ctx().get_my_session())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is NULL", K(ret));
} else if (OB_FAIL(get_index_tid(task, index_tid))) {
LOG_WARN("get index table tid failed", K(ret));
} else {
ObExtraIndexBuildCheck index_status_checker(index_tid);
ObExtraIndexBuildCheck::Guard index_check_guard(*session, index_status_checker);
share::CompatModeGuard compat_mode_guard(ORACLE_MODE == session->get_compatibility_mode()
? share::ObWorker::CompatMode::ORACLE
: share::ObWorker::CompatMode::MYSQL);
THIS_WORKER.set_timeout_ts(plan_ctx->get_timeout_timestamp());
if (OB_FAIL(index_status_checker.check())) {
LOG_WARN("check index status failed", K(ret));
} else if (OB_FAIL(session->store_query_string(
ObString::make_string("SQL DISTRIBUTE BACKGROUND PLAN EXECUTING")))) {
LOG_WARN("store session query string failed", K(ret));
} else if (OB_FAIL(processor.process_task(task, res.event_))) {
LOG_WARN("process task failed", K(ret));
}
}
if (!res.event_.is_valid()) {
// init task event if not valid.
ObTaskLocation task_loc;
task_loc.set_ob_task_id(res.task_id_);
task_loc.set_server(GCONF.self_addr_);
int tmp_ret = res.event_.init(task_loc, ret);
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("init task event failed", K(ret));
ret = OB_SUCCESS == ret ? tmp_ret : ret;
}
}
res.return_code_ = ret;
ret = OB_SUCCESS;
ret = E(EventTable::EN_BKGD_TASK_REPORT_COMPLETE) ret;
const int64_t timeout_us = abs_timeout_us_ - ObTimeUtility::current_time();
if (OB_FAIL(ret)) {
LOG_INFO("do not report background task complete", K(res.task_id_));
} else if (OB_FAIL(GCTX.executor_rpc_->get_proxy()
->to(return_addr_)
.by(OB_SYS_TENANT_ID) // always send as system tenant for background task
.as(OB_SYS_TENANT_ID)
.timeout(timeout_us)
.bkgd_task_complete(res))) {
LOG_WARN("send task complete message failed", K(ret));
} else {
LOG_INFO("send background task complete message success", K(ret), K(res.task_id_), K(res.return_code_));
}
}
}
else
{
ret = OB_ERR_UNEXPECTED;
LOG_WARN("change tenant context fail when execute background task", K(ret), K(dag->get_tenant_id()));
}
}
THIS_WORKER.set_timeout_ts(worker_abs_timeout_bak);
ObCurTraceId::init(GCTX.self_addr_);
return ret;
}
int ObSchedBKGDDistTask::init(const uint64_t tenant_id, const int64_t abs_timeout_us, const ObTaskID& task_id,
uint64_t scheduler_id, const common::ObPartitionKey& pkey, const common::ObAddr& dest,
const common::ObString& serialized_task)
{
int ret = OB_SUCCESS;
ObSchedBKGDDistTask t;
t.tenant_id_ = tenant_id;
t.abs_timeout_us_ = abs_timeout_us;
t.task_id_ = task_id;
t.scheduler_id_ = scheduler_id;
t.pkey_ = pkey;
t.dest_ = dest;
t.serialized_task_ = serialized_task;
if (OB_FAIL(assign(t))) {
LOG_WARN("assign failed", K(ret));
}
t.serialized_task_.reset();
return ret;
}
int ObSchedBKGDDistTask::init_execute_over_task(const ObTaskID& task_id)
{
int ret = OB_SUCCESS;
task_id_ = task_id;
return ret;
}
int ObSchedBKGDDistTask::assign(const ObSchedBKGDDistTask& o)
{
int ret = OB_SUCCESS;
if (this != &o) {
tenant_id_ = o.tenant_id_;
abs_timeout_us_ = o.abs_timeout_us_;
task_id_ = o.task_id_;
scheduler_id_ = o.scheduler_id_;
pkey_ = o.pkey_;
dest_ = o.dest_;
if (NULL != serialized_task_.ptr()) {
ob_free(serialized_task_.ptr());
serialized_task_.reset();
}
if (o.serialized_task_.length() > 0) {
ObMemAttr attr;
attr.label_ = ObModIds::OB_SQL_EXECUTOR_BKGD_TASK;
void* buf = ob_malloc(o.serialized_task_.length(), attr);
if (NULL == buf) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
MEMCPY(buf, o.serialized_task_.ptr(), o.serialized_task_.length());
serialized_task_.assign(static_cast<char*>(buf), o.serialized_task_.length());
}
}
}
return ret;
}
void ObSchedBKGDDistTask::destroy()
{
if (NULL != serialized_task_.ptr()) {
ob_free(serialized_task_.ptr());
serialized_task_.reset();
}
}
void ObSchedBKGDDistTask::to_schedule_pkey(common::ObPartitionKey& pkey) const
{
pkey.init(task_id_.get_execution_id(), task_id_.get_job_id(), task_id_.get_task_id());
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,159 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_EXECUTOR_OB_BKGD_DIST_TASK_H_
#define OCEANBASE_EXECUTOR_OB_BKGD_DIST_TASK_H_
#include "share/scheduler/ob_dag_scheduler.h"
#include "sql/executor/ob_task_id.h"
#include "sql/executor/ob_task.h"
namespace oceanbase {
namespace sql {
// dag information for background executing distributed task
class ObBKGDDistTaskDag : public share::ObIDag {
public:
ObBKGDDistTaskDag();
int init(const uint64_t tenant_id, const ObTaskID& task_id, const uint64_t scheduler_id);
virtual ~ObBKGDDistTaskDag();
virtual bool operator==(const ObIDag& other) const override;
virtual int64_t hash() const override;
virtual int64_t get_tenant_id() const override
{
return tenant_id_;
};
virtual int fill_comment(char* buf, const int64_t len) const override;
const ObTaskID& get_task_id() const
{
return task_id_;
}
uint64_t get_scheduler_id() const
{
return scheduler_id_;
}
virtual int64_t get_compat_mode() const override;
private:
uint64_t tenant_id_;
ObTaskID task_id_;
uint64_t scheduler_id_;
DISALLOW_COPY_AND_ASSIGN(ObBKGDDistTaskDag);
};
// background executing distributed task
class ObBKGDDistTask : public share::ObITask {
public:
ObBKGDDistTask();
virtual ~ObBKGDDistTask();
int init(const common::ObAddr& addr, const common::ObString& task, const int64_t abs_timeout_us);
virtual int process() override;
int get_index_tid(const ObTask& task, uint64_t& tid) const;
private:
class ObDistTaskProcessor;
common::ObAddr return_addr_;
common::ObString serialized_task_;
int64_t abs_timeout_us_;
int64_t create_time_us_; // task create time
common::ObCurTraceId::TraceId trace_id_;
common::ObMalloc task_allocator_;
DISALLOW_COPY_AND_ASSIGN(ObBKGDDistTask);
};
// background executing distributed task global schedule info (scheduled by RS).
// Memory are self managed, so we disable default copy constructor and assign function.
class ObSchedBKGDDistTask {
public:
ObSchedBKGDDistTask() : tenant_id_(common::OB_INVALID_ID), abs_timeout_us_(0), scheduler_id_(0)
{}
virtual ~ObSchedBKGDDistTask()
{
destroy();
}
ObSchedBKGDDistTask(const ObSchedBKGDDistTask&) = delete;
ObSchedBKGDDistTask& operator=(const ObSchedBKGDDistTask&) = delete;
int init(const uint64_t tenant_id, const int64_t abs_timeout_us, const ObTaskID& task_id, uint64_t scheduler_id,
const common::ObPartitionKey& pkey, const common::ObAddr& des, const common::ObString& serialized_task);
void destroy();
int init_execute_over_task(const ObTaskID& task_id);
int assign(const ObSchedBKGDDistTask& o);
// RS use partition key to identify task, partition key must be unique in schedule queue.
// We use task id instead, so we wrap task id to partition key for RS scheduler.
// (can not use %pkey_ directly)
void to_schedule_pkey(common::ObPartitionKey& pkey) const;
bool is_valid() const
{
return common::OB_INVALID_ID != tenant_id_ && abs_timeout_us_ > 0 && task_id_.is_valid() && scheduler_id_ > 0 &&
pkey_.is_valid() && dest_.is_valid() && !serialized_task_.empty();
}
TO_STRING_KV(K_(tenant_id), K_(abs_timeout_us), K_(pkey), K_(dest), K(serialized_task_.length()));
uint64_t get_tenant_id() const
{
return tenant_id_;
}
int64_t get_abs_timeout_us() const
{
return abs_timeout_us_;
}
const ObTaskID& get_task_id() const
{
return task_id_;
}
uint64_t get_scheduler_id() const
{
return scheduler_id_;
}
const common::ObPartitionKey& get_partition_key() const
{
return pkey_;
}
const common::ObAddr& get_dest() const
{
return dest_;
}
const common::ObString& get_serialized_task() const
{
return serialized_task_;
}
private:
uint64_t tenant_id_;
int64_t abs_timeout_us_;
ObTaskID task_id_;
uint64_t scheduler_id_;
common::ObPartitionKey pkey_;
common::ObAddr dest_;
common::ObString serialized_task_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_EXECUTOR_OB_BKGD_DIST_TASK_H_

View File

@ -0,0 +1,790 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/resolver/ob_cmd.h"
#include "sql/executor/ob_cmd_executor.h"
#include "lib/ob_name_def.h"
#include "share/ob_common_rpc_proxy.h"
#include "share/system_variable/ob_sys_var_class_type.h"
#include "sql/resolver/ddl/ob_create_tenant_stmt.h"
#include "sql/resolver/ddl/ob_drop_tenant_stmt.h"
#include "sql/resolver/ddl/ob_modify_tenant_stmt.h"
#include "sql/resolver/ddl/ob_lock_tenant_stmt.h"
#include "sql/resolver/ddl/ob_create_table_stmt.h"
#include "sql/resolver/ddl/ob_create_index_stmt.h"
#include "sql/resolver/ddl/ob_drop_index_stmt.h"
#include "sql/resolver/ddl/ob_alter_table_stmt.h"
#include "sql/resolver/ddl/ob_drop_table_stmt.h"
#include "sql/resolver/ddl/ob_drop_index_stmt.h"
#include "sql/resolver/ddl/ob_create_index_stmt.h"
#include "sql/resolver/ddl/ob_alter_database_stmt.h"
#include "sql/resolver/ddl/ob_drop_database_stmt.h"
#include "sql/resolver/ddl/ob_create_database_stmt.h"
#include "sql/resolver/ddl/ob_use_database_stmt.h"
#include "sql/resolver/ddl/ob_create_tablegroup_stmt.h"
#include "sql/resolver/ddl/ob_alter_tablegroup_stmt.h"
#include "sql/resolver/ddl/ob_drop_tablegroup_stmt.h"
#include "sql/resolver/ddl/ob_create_outline_stmt.h"
#include "sql/resolver/ddl/ob_alter_outline_stmt.h"
#include "sql/resolver/ddl/ob_drop_outline_stmt.h"
#include "sql/resolver/ddl/ob_rename_table_stmt.h"
#include "sql/resolver/ddl/ob_truncate_table_stmt.h"
#include "sql/resolver/ddl/ob_create_table_like_stmt.h"
#include "sql/resolver/ddl/ob_purge_stmt.h"
#include "sql/resolver/ddl/ob_alter_baseline_stmt.h"
#include "sql/resolver/dcl/ob_create_user_stmt.h"
#include "sql/resolver/dcl/ob_drop_user_stmt.h"
#include "sql/resolver/dcl/ob_rename_user_stmt.h"
#include "sql/resolver/dcl/ob_lock_user_stmt.h"
#include "sql/resolver/dcl/ob_set_password_stmt.h"
#include "sql/resolver/dcl/ob_grant_stmt.h"
#include "sql/resolver/dcl/ob_revoke_stmt.h"
#include "sql/resolver/dcl/ob_create_role_stmt.h"
#include "sql/resolver/dcl/ob_drop_role_stmt.h"
#include "sql/resolver/dcl/ob_alter_user_profile_stmt.h"
#include "sql/resolver/dcl/ob_alter_user_primary_zone_stmt.h"
#include "sql/resolver/tcl/ob_start_trans_stmt.h"
#include "sql/resolver/tcl/ob_end_trans_stmt.h"
#include "sql/resolver/tcl/ob_savepoint_stmt.h"
#include "sql/resolver/cmd/ob_bootstrap_stmt.h"
#include "sql/resolver/cmd/ob_kill_stmt.h"
#include "sql/resolver/cmd/ob_empty_query_stmt.h"
#include "sql/resolver/cmd/ob_resource_stmt.h"
#include "sql/resolver/cmd/ob_alter_system_stmt.h"
#include "sql/resolver/cmd/ob_variable_set_stmt.h"
#include "sql/resolver/cmd/ob_clear_balance_task_stmt.h"
#include "sql/resolver/prepare/ob_prepare_stmt.h"
#include "sql/resolver/prepare/ob_execute_stmt.h"
#include "sql/resolver/prepare/ob_deallocate_stmt.h"
#include "sql/resolver/ddl/ob_rename_table_stmt.h"
#include "sql/resolver/ddl/ob_truncate_table_stmt.h"
#include "sql/resolver/ddl/ob_create_table_like_stmt.h"
#include "sql/resolver/ddl/ob_purge_stmt.h"
#include "sql/resolver/ddl/ob_create_synonym_stmt.h"
#include "sql/resolver/ddl/ob_drop_synonym_stmt.h"
#include "sql/resolver/ddl/ob_create_func_stmt.h"
#include "sql/resolver/ddl/ob_drop_func_stmt.h"
#include "sql/resolver/ddl/ob_sequence_stmt.h"
#include "sql/resolver/xa/ob_xa_stmt.h"
#include "sql/resolver/ddl/ob_optimize_stmt.h"
#include "sql/resolver/ddl/ob_create_profile_stmt.h"
#include "sql/resolver/ddl/ob_create_dblink_stmt.h"
#include "sql/resolver/ddl/ob_drop_dblink_stmt.h"
#include "sql/resolver/cmd/ob_create_restore_point_stmt.h"
#include "sql/resolver/cmd/ob_drop_restore_point_stmt.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/cmd/ob_empty_query_executor.h"
#include "sql/engine/cmd/ob_dcl_executor.h"
#include "sql/engine/cmd/ob_tcl_executor.h"
#include "sql/engine/cmd/ob_tenant_executor.h"
#include "sql/engine/cmd/ob_set_names_executor.h"
#include "sql/engine/cmd/ob_alter_system_executor.h"
#include "sql/engine/cmd/ob_set_password_executor.h"
#include "sql/engine/cmd/ob_tablegroup_executor.h"
#include "sql/engine/cmd/ob_database_executor.h"
#include "sql/engine/cmd/ob_variable_set_executor.h"
#include "sql/engine/cmd/ob_table_executor.h"
#include "sql/engine/cmd/ob_index_executor.h"
#include "sql/engine/cmd/ob_resource_executor.h"
#include "sql/engine/cmd/ob_kill_executor.h"
#include "sql/engine/cmd/ob_user_cmd_executor.h"
#include "sql/engine/cmd/ob_outline_executor.h"
#include "sql/engine/cmd/ob_restore_executor.h"
#include "sql/engine/cmd/ob_baseline_executor.h"
#include "sql/engine/cmd/ob_synonym_executor.h"
#include "sql/engine/cmd/ob_udf_executor.h"
#include "sql/engine/cmd/ob_dblink_executor.h"
#include "sql/engine/cmd/ob_load_data_executor.h"
#include "sql/engine/cmd/ob_sequence_executor.h"
#include "sql/engine/cmd/ob_role_cmd_executor.h"
#include "sql/engine/cmd/ob_xa_executor.h"
#include "sql/engine/cmd/ob_profile_cmd_executor.h"
#include "sql/engine/prepare/ob_prepare_executor.h"
#include "sql/engine/prepare/ob_execute_executor.h"
#include "sql/engine/prepare/ob_deallocate_executor.h"
#include "observer/ob_server_event_history_table_operator.h"
namespace oceanbase {
using namespace common;
namespace sql {
#define DEFINE_EXECUTE_CMD(Statement, Executor) \
Statement& stmt = *(static_cast<Statement*>(&cmd)); \
Executor executor; \
sql_text = stmt.get_sql_stmt(); \
ret = executor.execute(ctx, stmt);
int ObCmdExecutor::execute(ObExecContext& ctx, ObICmd& cmd)
{
int ret = OB_SUCCESS;
ObString sql_text;
ObSQLSessionInfo* my_session = ctx.get_my_session();
bool is_ddl_or_dcl_stmt = false;
int64_t ori_query_timeout;
int64_t ori_trx_timeout;
my_session->get_query_timeout(ori_query_timeout);
my_session->get_tx_timeout(ori_trx_timeout);
if (ObStmt::is_ddl_stmt(static_cast<stmt::StmtType>(cmd.get_cmd_type()), true) ||
ObStmt::is_dcl_stmt(static_cast<stmt::StmtType>(cmd.get_cmd_type()))) {
if (stmt::T_VARIABLE_SET == static_cast<stmt::StmtType>(cmd.get_cmd_type()) &&
!static_cast<ObVariableSetStmt*>(&cmd)->has_global_variable()) {
// only set global variable belong to DDL OP,session level is not...
// do nothing
} else {
ObObj val;
val.set_int(GCONF._ob_ddl_timeout);
is_ddl_or_dcl_stmt = true;
if (OB_ISNULL(my_session)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is null", K(ret));
} else if (OB_FAIL(my_session->update_sys_variable(share::SYS_VAR_OB_QUERY_TIMEOUT, val))) {
LOG_WARN("set sys variable failed", K(ret), K(val.get_int()));
} else if (OB_FAIL(my_session->update_sys_variable(share::SYS_VAR_OB_TRX_TIMEOUT, val))) {
LOG_WARN("set sys variable failed", K(ret), K(val.get_int()));
} else {
ctx.get_physical_plan_ctx()->set_timeout_timestamp(my_session->get_query_start_time() + GCONF._ob_ddl_timeout);
THIS_WORKER.set_timeout_ts(my_session->get_query_start_time() + GCONF._ob_ddl_timeout);
if (stmt::T_CREATE_OUTLINE == static_cast<stmt::StmtType>(cmd.get_cmd_type()) ||
stmt::T_ALTER_OUTLINE == static_cast<stmt::StmtType>(cmd.get_cmd_type())) {
} else if (OB_FAIL(ctx.get_sql_ctx()->schema_guard_->reset())) {
LOG_WARN("schema_guard reset failed", K(ret));
}
}
}
}
switch (cmd.get_cmd_type()) {
case stmt::T_CREATE_RESOURCE_POOL: {
DEFINE_EXECUTE_CMD(ObCreateResourcePoolStmt, ObCreateResourcePoolExecutor);
break;
}
case stmt::T_DROP_RESOURCE_POOL: {
DEFINE_EXECUTE_CMD(ObDropResourcePoolStmt, ObDropResourcePoolExecutor);
break;
}
case stmt::T_SPLIT_RESOURCE_POOL: {
DEFINE_EXECUTE_CMD(ObSplitResourcePoolStmt, ObSplitResourcePoolExecutor);
break;
}
case stmt::T_MERGE_RESOURCE_POOL: {
DEFINE_EXECUTE_CMD(ObMergeResourcePoolStmt, ObMergeResourcePoolExecutor);
break;
}
case stmt::T_ALTER_RESOURCE_POOL: {
DEFINE_EXECUTE_CMD(ObAlterResourcePoolStmt, ObAlterResourcePoolExecutor);
break;
}
case stmt::T_CREATE_RESOURCE_UNIT: {
DEFINE_EXECUTE_CMD(ObCreateResourceUnitStmt, ObCreateResourceUnitExecutor);
break;
}
case stmt::T_ALTER_RESOURCE_UNIT: {
DEFINE_EXECUTE_CMD(ObAlterResourceUnitStmt, ObAlterResourceUnitExecutor);
break;
}
case stmt::T_DROP_RESOURCE_UNIT: {
DEFINE_EXECUTE_CMD(ObDropResourceUnitStmt, ObDropResourceUnitExecutor);
break;
}
case stmt::T_CREATE_TENANT: {
DEFINE_EXECUTE_CMD(ObCreateTenantStmt, ObCreateTenantExecutor);
break;
}
case stmt::T_DROP_TENANT: {
DEFINE_EXECUTE_CMD(ObDropTenantStmt, ObDropTenantExecutor);
break;
}
case stmt::T_MODIFY_TENANT: {
DEFINE_EXECUTE_CMD(ObModifyTenantStmt, ObModifyTenantExecutor);
break;
}
case stmt::T_LOCK_TENANT: {
DEFINE_EXECUTE_CMD(ObLockTenantStmt, ObLockTenantExecutor);
break;
}
case stmt::T_CREATE_VIEW: // fall through
case stmt::T_CREATE_TABLE: {
DEFINE_EXECUTE_CMD(ObCreateTableStmt, ObCreateTableExecutor);
break;
}
case stmt::T_ALTER_TABLE: {
DEFINE_EXECUTE_CMD(ObAlterTableStmt, ObAlterTableExecutor);
break;
}
case stmt::T_START_TRANS: {
DEFINE_EXECUTE_CMD(ObStartTransStmt, ObStartTransExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_END_TRANS: {
DEFINE_EXECUTE_CMD(ObEndTransStmt, ObEndTransExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_CREATE_SAVEPOINT: {
DEFINE_EXECUTE_CMD(ObCreateSavePointStmt, ObCreateSavePointExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_ROLLBACK_SAVEPOINT: {
DEFINE_EXECUTE_CMD(ObRollbackSavePointStmt, ObRollbackSavePointExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_RELEASE_SAVEPOINT: {
DEFINE_EXECUTE_CMD(ObReleaseSavePointStmt, ObReleaseSavePointExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_DROP_VIEW: // fall through
case stmt::T_DROP_TABLE: {
DEFINE_EXECUTE_CMD(ObDropTableStmt, ObDropTableExecutor);
break;
}
case stmt::T_RENAME_TABLE: {
DEFINE_EXECUTE_CMD(ObRenameTableStmt, ObRenameTableExecutor);
break;
}
case stmt::T_TRUNCATE_TABLE: {
DEFINE_EXECUTE_CMD(ObTruncateTableStmt, ObTruncateTableExecutor);
break;
}
case stmt::T_VARIABLE_SET: {
DEFINE_EXECUTE_CMD(ObVariableSetStmt, ObVariableSetExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_CREATE_DATABASE: {
DEFINE_EXECUTE_CMD(ObCreateDatabaseStmt, ObCreateDatabaseExecutor);
break;
}
case stmt::T_USE_DATABASE: {
DEFINE_EXECUTE_CMD(ObUseDatabaseStmt, ObUseDatabaseExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_ALTER_DATABASE: {
DEFINE_EXECUTE_CMD(ObAlterDatabaseStmt, ObAlterDatabaseExecutor);
break;
}
case stmt::T_DROP_DATABASE: {
DEFINE_EXECUTE_CMD(ObDropDatabaseStmt, ObDropDatabaseExecutor);
break;
}
case stmt::T_CREATE_TABLEGROUP: {
DEFINE_EXECUTE_CMD(ObCreateTablegroupStmt, ObCreateTablegroupExecutor);
break;
}
case stmt::T_ALTER_TABLEGROUP: {
DEFINE_EXECUTE_CMD(ObAlterTablegroupStmt, ObAlterTablegroupExecutor);
break;
}
case stmt::T_DROP_TABLEGROUP: {
DEFINE_EXECUTE_CMD(ObDropTablegroupStmt, ObDropTablegroupExecutor);
break;
}
case stmt::T_CREATE_INDEX: {
DEFINE_EXECUTE_CMD(ObCreateIndexStmt, ObCreateIndexExecutor);
break;
}
case stmt::T_DROP_INDEX: {
DEFINE_EXECUTE_CMD(ObDropIndexStmt, ObDropIndexExecutor);
break;
}
case stmt::T_ALTER_VIEW: {
break;
}
case stmt::T_CREATE_TABLE_LIKE: {
DEFINE_EXECUTE_CMD(ObCreateTableLikeStmt, ObCreateTableLikeExecutor);
break;
}
case stmt::T_PURGE_TABLE: {
DEFINE_EXECUTE_CMD(ObPurgeTableStmt, ObPurgeTableExecutor);
break;
}
case stmt::T_PURGE_INDEX: {
DEFINE_EXECUTE_CMD(ObPurgeIndexStmt, ObPurgeIndexExecutor);
break;
}
case stmt::T_PURGE_DATABASE: {
DEFINE_EXECUTE_CMD(ObPurgeDatabaseStmt, ObPurgeDatabaseExecutor);
break;
}
case stmt::T_PURGE_TENANT: {
DEFINE_EXECUTE_CMD(ObPurgeTenantStmt, ObPurgeTenantExecutor);
break;
}
case stmt::T_PURGE_RECYCLEBIN: {
DEFINE_EXECUTE_CMD(ObPurgeRecycleBinStmt, ObPurgeRecycleBinExecutor);
break;
}
case stmt::T_OPTIMIZE_TABLE: {
DEFINE_EXECUTE_CMD(ObOptimizeTableStmt, ObOptimizeTableExecutor);
break;
}
case stmt::T_OPTIMIZE_TENANT: {
DEFINE_EXECUTE_CMD(ObOptimizeTenantStmt, ObOptimizeTenantExecutor);
break;
}
case stmt::T_OPTIMIZE_ALL: {
DEFINE_EXECUTE_CMD(ObOptimizeAllStmt, ObOptimizeAllExecutor);
break;
}
case stmt::T_CREATE_USER: {
DEFINE_EXECUTE_CMD(ObCreateUserStmt, ObCreateUserExecutor);
break;
}
case stmt::T_ALTER_USER_PROFILE:
case stmt::T_ALTER_USER: {
DEFINE_EXECUTE_CMD(ObAlterUserProfileStmt, ObAlterUserProfileExecutor);
break;
}
case stmt::T_ALTER_USER_PRIMARY_ZONE: {
DEFINE_EXECUTE_CMD(ObAlterUserPrimaryZoneStmt, ObAlterUserPrimaryZoneExecutor);
break;
}
case stmt::T_DROP_USER: {
DEFINE_EXECUTE_CMD(ObDropUserStmt, ObDropUserExecutor);
break;
}
case stmt::T_RENAME_USER: {
DEFINE_EXECUTE_CMD(ObRenameUserStmt, ObRenameUserExecutor);
break;
}
case stmt::T_SET_PASSWORD: {
DEFINE_EXECUTE_CMD(ObSetPasswordStmt, ObSetPasswordExecutor);
break;
}
case stmt::T_LOCK_USER: {
DEFINE_EXECUTE_CMD(ObLockUserStmt, ObLockUserExecutor);
break;
}
case stmt::T_SYSTEM_GRANT:
case stmt::T_GRANT_ROLE:
case stmt::T_GRANT: {
DEFINE_EXECUTE_CMD(ObGrantStmt, ObGrantExecutor);
break;
}
case stmt::T_SYSTEM_REVOKE:
case stmt::T_REVOKE_ROLE:
case stmt::T_REVOKE: {
DEFINE_EXECUTE_CMD(ObRevokeStmt, ObRevokeExecutor);
break;
}
case stmt::T_PREPARE: {
DEFINE_EXECUTE_CMD(ObPrepareStmt, ObPrepareExecutor);
break;
}
case stmt::T_EXECUTE: {
DEFINE_EXECUTE_CMD(ObExecuteStmt, ObExecuteExecutor);
break;
}
case stmt::T_DEALLOCATE: {
DEFINE_EXECUTE_CMD(ObDeallocateStmt, ObDeallocateExecutor);
break;
}
case stmt::T_CHANGE_OBI:
case stmt::T_SWITCH_MASTER:
case stmt::T_SERVER_ACTION: {
break;
}
case stmt::T_BOOTSTRAP: {
DEFINE_EXECUTE_CMD(ObBootstrapStmt, ObBootstrapExecutor);
break;
}
case stmt::T_ADMIN_SERVER: {
DEFINE_EXECUTE_CMD(ObAdminServerStmt, ObAdminServerExecutor);
break;
}
case stmt::T_ADMIN_ZONE: {
DEFINE_EXECUTE_CMD(ObAdminZoneStmt, ObAdminZoneExecutor);
break;
}
case stmt::T_FREEZE: {
DEFINE_EXECUTE_CMD(ObFreezeStmt, ObFreezeExecutor);
break;
}
case stmt::T_FLUSH_CACHE: {
DEFINE_EXECUTE_CMD(ObFlushCacheStmt, ObFlushCacheExecutor);
break;
}
case stmt::T_FLUSH_KVCACHE: {
DEFINE_EXECUTE_CMD(ObFlushKVCacheStmt, ObFlushKVCacheExecutor);
break;
}
case stmt::T_FLUSH_ILOGCACHE: {
DEFINE_EXECUTE_CMD(ObFlushIlogCacheStmt, ObFlushIlogCacheExecutor);
break;
}
case stmt::T_FLUSH_DAG_WARNINGS: {
DEFINE_EXECUTE_CMD(ObFlushDagWarningsStmt, ObFlushDagWarningsExecutor);
break;
}
case stmt::T_SWITCH_REPLICA_ROLE: {
DEFINE_EXECUTE_CMD(ObSwitchReplicaRoleStmt, ObSwitchReplicaRoleExecutor);
break;
}
case stmt::T_SWITCH_RS_ROLE: {
DEFINE_EXECUTE_CMD(ObSwitchRSRoleStmt, ObSwitchRSRoleExecutor);
break;
}
case stmt::T_CHANGE_REPLICA: {
DEFINE_EXECUTE_CMD(ObChangeReplicaStmt, ObChangeReplicaExecutor);
break;
}
case stmt::T_DROP_REPLICA: {
DEFINE_EXECUTE_CMD(ObDropReplicaStmt, ObDropReplicaExecutor);
break;
}
case stmt::T_MIGRATE_REPLICA: {
DEFINE_EXECUTE_CMD(ObMigrateReplicaStmt, ObMigrateReplicaExecutor);
break;
}
case stmt::T_REPORT_REPLICA: {
DEFINE_EXECUTE_CMD(ObReportReplicaStmt, ObReportReplicaExecutor);
break;
}
case stmt::T_RECYCLE_REPLICA: {
DEFINE_EXECUTE_CMD(ObRecycleReplicaStmt, ObRecycleReplicaExecutor);
break;
}
case stmt::T_ADMIN_MERGE: {
DEFINE_EXECUTE_CMD(ObAdminMergeStmt, ObAdminMergeExecutor);
break;
}
case stmt::T_CLEAR_ROOT_TABLE: {
DEFINE_EXECUTE_CMD(ObClearRoottableStmt, ObClearRoottableExecutor);
break;
}
case stmt::T_REFRESH_SCHEMA: {
DEFINE_EXECUTE_CMD(ObRefreshSchemaStmt, ObRefreshSchemaExecutor);
break;
}
case stmt::T_REFRESH_MEMORY_STAT: {
DEFINE_EXECUTE_CMD(ObRefreshMemStatStmt, ObRefreshMemStatExecutor);
break;
}
case stmt::T_ALTER_SYSTEM_SET_PARAMETER: {
DEFINE_EXECUTE_CMD(ObSetConfigStmt, ObSetConfigExecutor);
break;
}
case stmt::T_ALTER_SYSTEM_SETTP: {
DEFINE_EXECUTE_CMD(ObSetTPStmt, ObSetTPExecutor);
break;
}
case stmt::T_CLEAR_LOCATION_CACHE: {
DEFINE_EXECUTE_CMD(ObClearLocationCacheStmt, ObClearLocationCacheExecutor);
break;
}
case stmt::T_RELOAD_GTS: {
DEFINE_EXECUTE_CMD(ObReloadGtsStmt, ObReloadGtsExecutor);
break;
}
case stmt::T_RELOAD_UNIT: {
DEFINE_EXECUTE_CMD(ObReloadUnitStmt, ObReloadUnitExecutor);
break;
}
case stmt::T_RELOAD_SERVER: {
DEFINE_EXECUTE_CMD(ObReloadServerStmt, ObReloadServerExecutor);
break;
}
case stmt::T_RELOAD_ZONE: {
DEFINE_EXECUTE_CMD(ObReloadZoneStmt, ObReloadZoneExecutor);
break;
}
case stmt::T_CLEAR_MERGE_ERROR: {
DEFINE_EXECUTE_CMD(ObClearMergeErrorStmt, ObClearMergeErrorExecutor);
break;
}
case stmt::T_MIGRATE_UNIT: {
DEFINE_EXECUTE_CMD(ObMigrateUnitStmt, ObMigrateUnitExecutor);
break;
}
case stmt::T_UPGRADE_VIRTUAL_SCHEMA: {
DEFINE_EXECUTE_CMD(ObUpgradeVirtualSchemaStmt, ObUpgradeVirtualSchemaExecutor);
break;
}
case stmt::T_ADMIN_UPGRADE_CMD: {
DEFINE_EXECUTE_CMD(ObAdminUpgradeCmdStmt, ObAdminUpgradeCmdExecutor);
break;
}
case stmt::T_ADMIN_ROLLING_UPGRADE_CMD: {
DEFINE_EXECUTE_CMD(ObAdminRollingUpgradeCmdStmt, ObAdminRollingUpgradeCmdExecutor);
break;
}
case stmt::T_RUN_JOB: {
DEFINE_EXECUTE_CMD(ObRunJobStmt, ObRunJobExecutor);
break;
}
case stmt::T_ADMIN_RUN_UPGRADE_JOB: {
DEFINE_EXECUTE_CMD(ObRunUpgradeJobStmt, ObRunUpgradeJobExecutor);
break;
}
case stmt::T_ADMIN_STOP_UPGRADE_JOB: {
DEFINE_EXECUTE_CMD(ObStopUpgradeJobStmt, ObStopUpgradeJobExecutor);
break;
}
case stmt::T_CANCEL_TASK: {
DEFINE_EXECUTE_CMD(ObCancelTaskStmt, ObCancelTaskExecutor);
break;
}
case stmt::T_SET_NAMES: {
DEFINE_EXECUTE_CMD(ObSetNamesStmt, ObSetNamesExecutor);
sql_text = ObString::make_empty_string(); // do not record
break;
}
case stmt::T_LOAD_DATA: {
DEFINE_EXECUTE_CMD(ObLoadDataStmt, ObLoadDataExecutor);
break;
}
case stmt::T_KILL: {
DEFINE_EXECUTE_CMD(ObKillStmt, ObKillExecutor);
break;
}
case stmt::T_EMPTY_QUERY: {
DEFINE_EXECUTE_CMD(ObEmptyQueryStmt, ObEmptyQueryExecutor);
break;
}
case stmt::T_CREATE_OUTLINE: {
DEFINE_EXECUTE_CMD(ObCreateOutlineStmt, ObCreateOutlineExecutor);
break;
}
case stmt::T_ALTER_OUTLINE: {
DEFINE_EXECUTE_CMD(ObAlterOutlineStmt, ObAlterOutlineExecutor);
break;
}
case stmt::T_DROP_OUTLINE: {
DEFINE_EXECUTE_CMD(ObDropOutlineStmt, ObDropOutlineExecutor);
break;
}
case stmt::T_LOAD_BASELINE: {
DEFINE_EXECUTE_CMD(ObLoadBaselineStmt, ObLoadBaselineExecutor);
break;
}
case stmt::T_ALTER_BASELINE: {
DEFINE_EXECUTE_CMD(ObAlterBaselineStmt, ObAlterBaselineExecutor);
break;
}
case stmt::T_REFRESH_TIME_ZONE_INFO: {
DEFINE_EXECUTE_CMD(ObRefreshTimeZoneInfoStmt, ObRefreshTimeZoneInfoExecutor);
break;
}
case stmt::T_SET_DISK_VALID: {
DEFINE_EXECUTE_CMD(ObSetDiskValidStmt, ObSetDiskValidExecutor);
break;
}
case stmt::T_CREATE_SYNONYM: {
DEFINE_EXECUTE_CMD(ObCreateSynonymStmt, ObCreateSynonymExecutor);
break;
}
case stmt::T_DROP_SYNONYM: {
DEFINE_EXECUTE_CMD(ObDropSynonymStmt, ObDropSynonymExecutor);
break;
}
case stmt::T_CLEAR_BALANCE_TASK: {
DEFINE_EXECUTE_CMD(ObClearBalanceTaskStmt, ObClearBalanceTaskExecutor);
break;
}
case stmt::T_RESTORE_TENANT: {
DEFINE_EXECUTE_CMD(ObRestoreTenantStmt, ObRestoreTenantExecutor);
break;
}
case stmt::T_PHYSICAL_RESTORE_TENANT: {
DEFINE_EXECUTE_CMD(ObPhysicalRestoreTenantStmt, ObPhysicalRestoreTenantExecutor);
break;
}
case stmt::T_CHANGE_TENANT: {
DEFINE_EXECUTE_CMD(ObChangeTenantStmt, ObChangeTenantExecutor);
break;
}
case stmt::T_CREATE_FUNC: {
DEFINE_EXECUTE_CMD(ObCreateFuncStmt, ObCreateFuncExecutor);
break;
}
case stmt::T_DROP_FUNC: {
DEFINE_EXECUTE_CMD(ObDropFuncStmt, ObDropFuncExecutor);
break;
}
case stmt::T_CREATE_SEQUENCE: {
DEFINE_EXECUTE_CMD(ObCreateSequenceStmt, ObCreateSequenceExecutor);
break;
}
case stmt::T_DROP_SEQUENCE: {
DEFINE_EXECUTE_CMD(ObDropSequenceStmt, ObDropSequenceExecutor);
break;
}
case stmt::T_ALTER_SEQUENCE: {
DEFINE_EXECUTE_CMD(ObAlterSequenceStmt, ObAlterSequenceExecutor);
break;
}
case stmt::T_SET_TABLE_COMMENT:
case stmt::T_SET_COLUMN_COMMENT: {
DEFINE_EXECUTE_CMD(ObAlterTableStmt, ObAlterTableExecutor);
break;
}
case stmt::T_XA_START: {
DEFINE_EXECUTE_CMD(ObXaStartStmt, ObXaStartExecutor);
break;
}
case stmt::T_XA_END: {
DEFINE_EXECUTE_CMD(ObXaEndStmt, ObXaEndExecutor);
break;
}
case stmt::T_XA_PREPARE: {
DEFINE_EXECUTE_CMD(ObXaPrepareStmt, ObXaPrepareExecutor);
break;
}
case stmt::T_XA_COMMIT: {
DEFINE_EXECUTE_CMD(ObXaCommitStmt, ObXaEndTransExecutor);
break;
}
case stmt::T_XA_ROLLBACK: {
DEFINE_EXECUTE_CMD(ObXaRollBackStmt, ObXaEndTransExecutor);
break;
}
case stmt::T_ALTER_DISKGROUP_ADD_DISK: {
DEFINE_EXECUTE_CMD(ObAddDiskStmt, ObAddDiskExecutor);
break;
}
case stmt::T_ALTER_DISKGROUP_DROP_DISK: {
DEFINE_EXECUTE_CMD(ObDropDiskStmt, ObDropDiskExecutor);
break;
}
case stmt::T_CREATE_ROLE: {
DEFINE_EXECUTE_CMD(ObCreateRoleStmt, ObCreateRoleExecutor);
break;
}
case stmt::T_DROP_ROLE: {
DEFINE_EXECUTE_CMD(ObDropRoleStmt, ObDropRoleExecutor);
break;
}
/*case stmt::T_ALTER_ROLE: {
DEFINE_EXECUTE_CMD(ObAlterRoutineStmt, ObAlterRoleExecutor);
break;
}
case stmt::T_SET_ROLE: {
DEFINE_EXECUTE_CMD(ObSetRoutineStmt, ObSetRoleExecutor);
break;
}*/
case stmt::T_USER_PROFILE: {
DEFINE_EXECUTE_CMD(ObUserProfileStmt, ObProfileDDLExecutor);
break;
}
case stmt::T_ARCHIVE_LOG: {
DEFINE_EXECUTE_CMD(ObArchiveLogStmt, ObArchiveLogExecutor);
break;
}
case stmt::T_BACKUP_DATABASE: {
DEFINE_EXECUTE_CMD(ObBackupDatabaseStmt, ObBackupDatabaseExecutor);
break;
}
case stmt::T_BACKUP_MANAGE: {
DEFINE_EXECUTE_CMD(ObBackupManageStmt, ObBackupManageExecutor);
break;
}
case stmt::T_CREATE_DBLINK: {
DEFINE_EXECUTE_CMD(ObCreateDbLinkStmt, ObCreateDbLinkExecutor);
break;
}
case stmt::T_DROP_DBLINK: {
DEFINE_EXECUTE_CMD(ObDropDbLinkStmt, ObDropDbLinkExecutor);
break;
}
case stmt::T_BACKUP_SET_ENCRYPTION: {
DEFINE_EXECUTE_CMD(ObBackupSetEncryptionStmt, ObBackupSetEncryptionExecutor);
break;
}
case stmt::T_BACKUP_SET_DECRYPTION: {
DEFINE_EXECUTE_CMD(ObBackupSetDecryptionStmt, ObBackupSetDecryptionExecutor);
break;
}
case stmt::T_ENABLE_SQL_THROTTLE: {
DEFINE_EXECUTE_CMD(ObEnableSqlThrottleStmt, ObEnableSqlThrottleExecutor);
break;
}
case stmt::T_DISABLE_SQL_THROTTLE: {
DEFINE_EXECUTE_CMD(ObDisableSqlThrottleStmt, ObDisableSqlThrottleExecutor);
break;
}
case stmt::T_CREATE_RESTORE_POINT: {
DEFINE_EXECUTE_CMD(ObCreateRestorePointStmt, ObCreateRestorePointExecutor);
break;
}
case stmt::T_DROP_RESTORE_POINT: {
DEFINE_EXECUTE_CMD(ObDropRestorePointStmt, ObDropRestorePointExecutor);
break;
}
case stmt::T_CS_DISKMAINTAIN:
case stmt::T_TABLET_CMD:
case stmt::T_SWITCH_ROOTSERVER:
case stmt::T_SWITCH_UPDATESERVER:
case stmt::T_CLUSTER_MANAGER:
case stmt::T_DROP_MEMTABLE:
case stmt::T_CLEAR_MEMTABLE:
case stmt::T_ADD_UPDATESERVER:
case stmt::T_DELETE_UPDATESERVER:
case stmt::T_CHECK_ROOT_TABLE:
default: {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unknow cmd type", "cmd_type", cmd.get_cmd_type(), "T_MAX", T_MAX);
break;
}
}
if (!sql_text.empty()) {
SERVER_EVENT_ADD("sql",
"execute_cmd",
"cmd_type",
cmd.get_cmd_type(),
"sql_text",
ObHexEscapeSqlStr(sql_text),
"return_code",
ret);
}
if (is_ddl_or_dcl_stmt) {
// ddl/dcl changes the query_timeout,trx_timeout of session in processs
int tmp_ret = ret;
ObObj ori_query_timeout_obj;
ObObj ori_trx_timeout_obj;
ori_query_timeout_obj.set_int(ori_query_timeout);
ori_trx_timeout_obj.set_int(ori_trx_timeout);
if (OB_ISNULL(my_session)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is null", K(ret));
} else if (OB_ISNULL(ctx.get_task_exec_ctx().schema_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema_service_ is null", K(ret));
} else if (OB_FAIL(my_session->update_sys_variable(share::SYS_VAR_OB_QUERY_TIMEOUT, ori_query_timeout_obj))) {
LOG_WARN("set sys variable failed", K(ret), K(ori_query_timeout_obj.get_int()));
} else if (OB_FAIL(my_session->update_sys_variable(share::SYS_VAR_OB_TRX_TIMEOUT, ori_trx_timeout_obj))) {
LOG_WARN("set sys variable failed", K(ret), K(ori_trx_timeout_obj.get_int()));
} else if (OB_FAIL(ctx.get_task_exec_ctx().schema_service_->get_tenant_schema_guard(
my_session->get_effective_tenant_id(), *(ctx.get_sql_ctx()->schema_guard_)))) {
LOG_WARN("failed to get schema guard", K(ret));
}
if (OB_FAIL(tmp_ret)) {
ret = tmp_ret;
}
}
return ret;
}
#undef DEFINE_EXECUTE_CMD
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,34 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_CMD_EXECUTOR_
#define OCEANBASE_SQL_EXECUTOR_OB_CMD_EXECUTOR_
#include "share/ob_define.h"
namespace oceanbase {
namespace sql {
class ObICmd;
class ObExecContext;
class ObCmdExecutor {
public:
static int execute(ObExecContext& ctx, ObICmd& cmd);
private:
/* functions */
/* variables */
DISALLOW_COPY_AND_ASSIGN(ObCmdExecutor);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_CMD_EXECUTOR_ */
//// end of header file

View File

@ -0,0 +1,383 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "ob_determinate_task_spliter.h"
#include "ob_determinate_task_transmit.h"
#include "lib/string/ob_sql_string.h"
#include "lib/mysqlclient/ob_mysql_result.h"
#include "sql/engine/ob_exec_context.h"
#include "share/ob_dml_sql_splicer.h"
namespace oceanbase {
using namespace common;
using namespace share;
namespace sql {
template <typename T>
bool lexical_less(const T& l, const T& r)
{
return l < r;
}
template <typename T, typename... Args>
bool lexical_less(const T& l, const T& r, Args... rest)
{
bool less = false;
if (l == r) {
less = lexical_less(rest...);
} else {
less = l < r;
}
return less;
}
struct ObDeterminateTaskSpliter::SliceIDCompare {
SliceIDCompare(int& ret) : ret_(ret)
{}
bool operator()(const ObSliceEvent* l, const ObSliceEvent* r)
{
bool less = false;
if (OB_SUCCESS != ret_) {
} else if (OB_ISNULL(l) || OB_ISNULL(r)) {
ret_ = OB_ERR_UNEXPECTED;
LOG_WARN("NULL slice id", K(ret_));
} else {
less = lexical_less(l->get_ob_slice_id().get_job_id(),
r->get_ob_slice_id().get_job_id(),
l->get_ob_slice_id().get_task_id(),
r->get_ob_slice_id().get_task_id(),
l->get_ob_slice_id().get_slice_id(),
r->get_ob_slice_id().get_slice_id());
}
return less;
}
int& ret_;
};
ObDeterminateTaskSpliter::ObDeterminateTaskSpliter() : task_idx_(0), child_slices_fetched_(false)
{}
ObDeterminateTaskSpliter::~ObDeterminateTaskSpliter()
{}
int ObDeterminateTaskSpliter::get_next_task(ObTaskInfo*& task)
{
int ret = OB_SUCCESS;
ObPhyOperator* op = NULL;
if (OB_ISNULL(job_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_ISNULL(op = job_->get_root_op())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root operator of job is NULL", K(ret));
} else if (PHY_DETERMINATE_TASK_TRANSMIT != op->get_type()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("not determinate transmit", K(ret));
} else {
ObDeterminateTaskTransmit* transmit = static_cast<ObDeterminateTaskTransmit*>(op);
ObTaskID task_id;
task_id.set_ob_job_id(job_->get_ob_job_id());
task_id.set_task_id(task_idx_);
task_id.set_task_cnt(transmit->get_tasks().count());
if (task_idx_ >= transmit->get_tasks().count()) {
ret = OB_ITER_END;
} else if (OB_FAIL(create_task_info(task))) {
LOG_WARN("create task info failed", K(ret));
} else if (OB_ISNULL(task)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL task", K(ret));
} else {
task->set_background(transmit->is_background());
const auto& idx = transmit->get_tasks().at(task_idx_);
const auto& range_loc = transmit->get_range_locations().at(idx.loc_idx_);
const auto& part_loc = range_loc.part_locs_.at(idx.part_loc_idx_);
if (OB_FAIL(task->get_range_location().part_locs_.init(1))) {
LOG_WARN("fixed array init failed", K(ret));
} else if (OB_FAIL(task->get_range_location().part_locs_.push_back(part_loc))) {
LOG_WARN("fixed array push back failed", K(ret));
} else if (OB_FAIL(fetch_child_result(task_id, *task))) {
LOG_WARN("fetch child result failed", K(ret));
} else {
// send to same server if tasks in same range location
if (task_idx_ > 0 && idx.loc_idx_ == transmit->get_tasks().at(task_idx_ - 1).loc_idx_) {
task->get_range_location().server_ = pre_addr_;
} else {
if (OB_FAIL(set_task_destination(*transmit, task_id, *task))) {
LOG_WARN("set task destination failed", K(ret));
} else {
pre_addr_ = task->get_range_location().server_;
}
}
}
}
if (OB_SUCC(ret)) {
ObTaskLocation loc;
loc.set_ob_task_id(task_id);
loc.set_server(task->get_range_location().server_);
task->set_task_location(loc);
const auto& pkey = task->get_range_location().part_locs_.at(0).partition_key_;
if (pkey.is_valid()) {
const ObPhyTableLocation* phy_tlb_loc = NULL;
if (OB_FAIL(ObTaskExecutorCtxUtil::get_phy_table_location(
*exec_ctx_, pkey.get_table_id(), pkey.get_table_id(), phy_tlb_loc))) {
LOG_WARN("get table location failed", K(ret));
} else {
const auto& loc_list = phy_tlb_loc->get_partition_location_list();
int64_t idx = 0;
for (; idx < loc_list.count(); idx++) {
if (loc_list.at(idx).get_partition_id() == pkey.get_partition_id()) {
break;
}
}
if (idx == loc_list.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("partition not found in location", K(pkey), K(ret));
} else {
task->set_location_idx(idx);
}
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(record_task(
exec_ctx_->get_task_exec_ctx().get_sys_job_id(), task_id, *task, transmit->get_split_task_count()))) {
LOG_WARN("record task failed", K(ret));
}
}
if (OB_SUCC(ret)) {
task_idx_++;
}
}
return ret;
}
int ObDeterminateTaskSpliter::fetch_child_result(const ObTaskID& task_id, ObTaskInfo& task)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(job_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (!job_->has_child_job()) {
// do nothing
} else {
if (!child_slices_fetched_) {
const bool skip_empty = false;
for (int64_t i = 0; OB_SUCC(ret) && i < job_->get_child_count(); i++) {
ObJob* child = NULL;
if (OB_FAIL(job_->get_child_job(i, child))) {
LOG_WARN("get child job failed", K(ret));
} else if (OB_ISNULL(child)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child is NULL", K(ret));
} else if (OB_FAIL(child->append_finished_slice_events(child_slices_, skip_empty))) {
LOG_WARN("append finished slice events failed", K(ret));
} else {
std::sort(child_slices_.begin(), child_slices_.end(), SliceIDCompare(ret));
if (OB_FAIL(ret)) {
LOG_WARN("sort child slices failed", K(ret));
}
}
}
if (OB_SUCC(ret)) {
child_slices_fetched_ = true;
}
}
}
if (OB_SUCC(ret)) {
auto& child_results = task.get_child_task_results();
for (int64_t i = 0; OB_SUCC(ret) && i < job_->get_child_count(); i++) {
ObJob* child = NULL;
ObPhyOperator* op = NULL;
ObTaskResultBuf res;
if (OB_FAIL(job_->get_child_job(i, child))) {
LOG_WARN("get child job failed", K(ret));
} else if (OB_ISNULL(child)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child is NULL", K(ret));
} else if (OB_ISNULL(op = child->get_root_op())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get root operator failed", K(ret));
} else if (OB_FAIL(PHY_DETERMINATE_TASK_TRANSMIT != op->get_type())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("not determinate task transmit", K(ret), KP(op));
} else {
const auto& result_mapping = static_cast<ObDeterminateTaskTransmit*>(op)->get_result_mapping();
if (result_mapping.count() <= task_id.get_task_id()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task index exceed result mapping", K(ret));
} else {
const auto& mapping = result_mapping.at(task_id.get_task_id());
FOREACH_X(slice, child_slices_, OB_SUCC(ret))
{
const auto& slice_id = (*slice)->get_ob_slice_id();
if (child->get_job_id() == slice_id.get_job_id() && slice_id.get_task_id() >= mapping.task_range_.begin_ &&
slice_id.get_task_id() < mapping.task_range_.end_ &&
slice_id.get_slice_id() >= mapping.slice_range_.begin_ &&
slice_id.get_slice_id() < mapping.slice_range_.end_) {
res.reset();
ObTaskControl* tc = NULL;
if (OB_FAIL(child->get_task_control(*exec_ctx_, tc))) {
LOG_WARN("fail get task ctrl", K(ret));
} else if (tc->get_task_location(slice_id.get_task_id(), res.get_task_location())) {
LOG_WARN("get task location failed", K(ret), K(slice_id));
} else if (OB_FAIL(res.add_slice_event(**slice))) {
LOG_WARN("add slice event failed", K(ret));
} else if (OB_FAIL(child_results.push_back(res))) {
LOG_WARN("array push back failed", K(ret));
}
}
} // end FOREACH_X
}
}
} // end for
}
return ret;
}
int ObDeterminateTaskSpliter::set_task_destination(
const ObDeterminateTaskTransmit& transmit, const ObTaskID& task_id, ObTaskInfo& task)
{
int ret = OB_SUCCESS;
ObAddr dest;
ObArray<ObAddr> previous;
auto routing = transmit.get_task_routing();
auto policy = transmit.get_task_route_policy();
if (OB_ISNULL(job_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_ISNULL(routing)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL task routing", K(ret));
} else if (OB_FAIL(task_executed_servers(exec_ctx_->get_task_exec_ctx().get_sys_job_id(), task_id, previous))) {
LOG_WARN("get task executed servers failed",
K(ret),
"sys_job_id",
exec_ctx_->get_task_exec_ctx().get_sys_job_id(),
K(task_id));
} else if (OB_FAIL(routing->route(policy, task, previous, dest))) {
LOG_WARN("choose task destination failed", K(ret), K(policy), K(previous));
} else {
task.get_range_location().server_ = dest;
}
return ret;
}
int ObDeterminateTaskSpliter::task_executed_servers(
const int64_t sys_job_id, const ObTaskID& task_id, common::ObIArray<common::ObAddr>& servers)
{
int ret = OB_SUCCESS;
ObSqlString sql;
if (OB_ISNULL(exec_ctx_) || OB_ISNULL(exec_ctx_->get_sql_proxy())) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret), KP(exec_ctx_));
} else if (OB_FAIL(sql.assign_fmt("SELECT svr_ip, svr_port from %s WHERE job_id = %ld AND execution_id = %ld "
"AND sql_job_id = %lu AND task_id = %ld",
OB_ALL_SQL_EXECUTE_TASK_TNAME,
sys_job_id,
task_id.get_execution_id(),
task_id.get_job_id(),
task_id.get_task_id()))) {
LOG_WARN("assign sql failed", K(ret));
} else {
SMART_VAR(ObMySQLProxy::MySQLResult, res)
{
sqlclient::ObMySQLResult* result = NULL;
if (OB_FAIL(exec_ctx_->get_sql_proxy()->read(res, sql.ptr()))) {
LOG_WARN("execute sql failed", K(ret));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL result", K(ret));
} else {
char ip[OB_IP_STR_BUFF] = "";
int32_t port = 0;
ObAddr server;
while (OB_SUCC(ret)) {
if (OB_FAIL(result->next())) {
if (OB_ITER_END != ret) {
LOG_WARN("get next result failed", K(ret));
} else {
ret = OB_SUCCESS;
}
break;
}
int64_t tmp = 0;
EXTRACT_STRBUF_FIELD_MYSQL(*result, "svr_ip", ip, OB_IP_STR_BUFF, tmp);
UNUSED(tmp); // make compiler happy
EXTRACT_INT_FIELD_MYSQL(*result, "svr_port", port, int32_t);
if (OB_FAIL(ret)) {
} else if (!server.set_ip_addr(ip, port)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("set server failed", K(ret), K(ip), K(port));
} else if (OB_FAIL(servers.push_back(server))) {
LOG_WARN("array push back failed", K(ret));
}
}
}
}
}
return ret;
}
int ObDeterminateTaskSpliter::record_task(
const int64_t sys_job_id, const ObTaskID& task_id, const ObTaskInfo& task, const int64_t slice_count)
{
int ret = OB_SUCCESS;
const int64_t max_task_info_len = 1 << 12; // 4KB
char ip[OB_IP_STR_BUFF] = "";
ObDMLSqlSplicer dml;
ObSqlString buf;
if (OB_ISNULL(exec_ctx_) || OB_ISNULL(exec_ctx_->get_sql_proxy())) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret), KP(exec_ctx_));
} else if (!task.get_range_location().server_.ip_to_string(ip, sizeof(ip))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("convert address to string failed", K(ret));
} else if (OB_FAIL(buf.reserve(max_task_info_len))) {
LOG_WARN("string reserve failed", K(ret));
} else {
int64_t len = task.to_string(buf.ptr(), max_task_info_len);
if (OB_FAIL(buf.set_length(len))) {
LOG_WARN("set string length failed", K(ret), K(len));
} else if (OB_FAIL(dml.add_pk_column("job_id", sys_job_id)) ||
OB_FAIL(dml.add_pk_column("execution_id", task_id.get_execution_id())) ||
OB_FAIL(dml.add_pk_column("sql_job_id", task_id.get_job_id())) ||
OB_FAIL(dml.add_pk_column("task_id", task_id.get_task_id())) ||
OB_FAIL(dml.add_pk_column("svr_ip", ip)) ||
OB_FAIL(dml.add_pk_column("svr_port", task.get_range_location().server_.get_port())) ||
OB_FAIL(dml.add_column("slice_count", slice_count)) || OB_FAIL(dml.add_column("task_stat", "INIT")) ||
OB_FAIL(dml.add_column("task_result", 0)) ||
OB_FAIL(dml.add_column(
"task_info", ObHexEscapeSqlStr(ObString(static_cast<int32_t>(buf.length()), buf.ptr()))))) {
LOG_WARN("add column failed", K(ret));
} else {
ObDMLExecHelper exec(*exec_ctx_->get_sql_proxy(), OB_SYS_TENANT_ID);
int64_t affected_rows = 0;
if (OB_FAIL(exec.exec_insert_update(OB_ALL_SQL_EXECUTE_TASK_TNAME, dml, affected_rows))) {
LOG_WARN("execute insert on duplicate update failed", K(ret));
}
}
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,54 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_EXECUTOR_OB_DETERMINATE_TASK_SPLITER_H_
#define OCEANBASE_EXECUTOR_OB_DETERMINATE_TASK_SPLITER_H_
#include "sql/executor/ob_task_spliter.h"
namespace oceanbase {
namespace sql {
class ObDeterminateTaskTransmit;
class ObDeterminateTaskSpliter : public ObTaskSpliter {
public:
ObDeterminateTaskSpliter();
virtual ~ObDeterminateTaskSpliter();
virtual int get_next_task(ObTaskInfo*& task) override;
virtual TaskSplitType get_type() const override
{
return ObTaskSpliter::DETERMINATE_TASK_SPLIT;
}
private:
struct SliceIDCompare;
private:
int fetch_child_result(const ObTaskID& task_id, ObTaskInfo& task);
int set_task_destination(const ObDeterminateTaskTransmit& transmit, const ObTaskID& task_id, ObTaskInfo& task);
int record_task(const int64_t sys_job_id, const ObTaskID& task_id, const ObTaskInfo& task, const int64_t slice_count);
int task_executed_servers(
const int64_t sys_job_id, const ObTaskID& task_id, common::ObIArray<common::ObAddr>& servers);
private:
int64_t task_idx_;
common::ObAddr pre_addr_;
common::ObArray<const ObSliceEvent*> child_slices_;
bool child_slices_fetched_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_EXECUTOR_OB_DETERMINATE_TASK_SPLITER_H_

View File

@ -0,0 +1,530 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "ob_determinate_task_transmit.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/session/ob_sql_session_info.h"
#include "share/schema/ob_part_mgr_util.h"
#include "sql/optimizer/ob_table_location.h"
#include "lib/utility/ob_tracepoint.h"
namespace oceanbase {
namespace sql {
using namespace common;
using namespace share;
OB_SERIALIZE_MEMBER(ObDeterminateTaskTransmit::TaskIndex, loc_idx_, part_loc_idx_);
OB_SERIALIZE_MEMBER(ObDeterminateTaskTransmit::IdRange, begin_, end_);
OB_SERIALIZE_MEMBER(ObDeterminateTaskTransmit::ResultRange, task_range_, slice_range_);
struct ObDeterminateTaskTransmit::RangeStartCompare {
RangeStartCompare(int& ret) : ret_(ret)
{}
bool operator()(const ObNewRange& range, const ObNewRow& row)
{
start_row_.assign(const_cast<ObObj*>(range.start_key_.get_obj_ptr()), range.start_key_.get_obj_cnt());
int cmp = 0;
if (OB_SUCCESS == ret_) {
ret_ = ObRowUtil::compare_row(start_row_, row, cmp);
if (OB_SUCCESS != ret_) {
LOG_WARN("compare row failed", K(ret_));
} else {
if (0 == cmp && !range.border_flag_.inclusive_start()) {
cmp = 1;
}
}
}
return cmp < 0;
}
private:
ObNewRow start_row_;
int& ret_;
};
OB_DEF_SERIALIZE(ObDeterminateTaskTransmit)
{
int ret = OB_SUCCESS;
BASE_SER((ObDeterminateTaskTransmit, ObDistributedTransmit));
LST_DO_CODE(OB_UNIS_ENCODE, result_reusable_, shuffle_by_part_, shuffle_by_range_, start_slice_ids_, result_mapping_);
int64_t cnt = shuffle_ranges_.count();
LST_DO_CODE(OB_UNIS_ENCODE, cnt);
FOREACH_CNT_X(ranges, shuffle_ranges_, OB_SUCC(ret))
{
LST_DO_CODE(OB_UNIS_ENCODE, *ranges);
}
return ret;
}
OB_DEF_DESERIALIZE(ObDeterminateTaskTransmit)
{
int ret = OB_SUCCESS;
BASE_DESER((ObDeterminateTaskTransmit, ObDistributedTransmit));
LST_DO_CODE(OB_UNIS_DECODE, result_reusable_, shuffle_by_part_, shuffle_by_range_, start_slice_ids_, result_mapping_);
int64_t cnt = 0;
LST_DO_CODE(OB_UNIS_DECODE, cnt);
if (OB_ISNULL(get_phy_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL physical plan", K(ret));
}
if (OB_SUCC(ret) && cnt > 0) {
if (OB_FAIL(shuffle_ranges_.prepare_allocate(cnt))) {
LOG_WARN("fix array prepare allocate failed", K(ret), K(cnt));
} else {
FOREACH_CNT_X(ranges, shuffle_ranges_, OB_SUCC(ret))
{
ranges->set_allocator(&const_cast<ObPhysicalPlan*>(get_phy_plan())->get_allocator());
LST_DO_CODE(OB_UNIS_DECODE, *ranges);
}
}
}
return ret;
}
OB_DEF_SERIALIZE_SIZE(ObDeterminateTaskTransmit)
{
int64_t len = 0;
int ret = OB_SUCCESS;
BASE_ADD_LEN((ObDeterminateTaskTransmit, ObDistributedTransmit));
LST_DO_CODE(
OB_UNIS_ADD_LEN, result_reusable_, shuffle_by_part_, shuffle_by_range_, start_slice_ids_, result_mapping_);
int64_t cnt = shuffle_ranges_.count();
LST_DO_CODE(OB_UNIS_ADD_LEN, cnt);
FOREACH_CNT_X(ranges, shuffle_ranges_, OB_SUCC(ret))
{
LST_DO_CODE(OB_UNIS_ADD_LEN, *ranges);
}
return len;
}
ObDeterminateTaskTransmit::ObDeterminateTaskTransmit(ObIAllocator& alloc)
: ObDistributedTransmit(alloc),
result_reusable_(false),
range_locations_(alloc),
tasks_(alloc),
shuffle_by_part_(false),
shuffle_by_range_(false),
shuffle_ranges_(alloc),
start_slice_ids_(alloc),
result_mapping_(alloc),
task_route_policy_(ITaskRouting::DATA_REPLICA_PICKER),
task_routing_(NULL),
background_(false)
{}
int ObDeterminateTaskTransmit::init_op_ctx(ObExecContext& exec_ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObDeterminateTaskTransmitCtx, exec_ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("create physical operator context failed", K(ret));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL operator context", K(ret));
} else if (OB_FAIL(init_cur_row(*op_ctx, false))) {
LOG_WARN("init current row failed", K(ret));
}
return ret;
}
OperatorOpenOrder ObDeterminateTaskTransmit::get_operator_open_order(ObExecContext& ctx) const
{
UNUSED(ctx);
OperatorOpenOrder open_order = OPEN_CHILDREN_FIRST;
if (result_reusable_) {
open_order = OPEN_SELF_ONLY;
}
return open_order;
}
int ObDeterminateTaskTransmit::inner_open(ObExecContext& exec_ctx) const
{
int ret = OB_SUCCESS;
ObSQLSessionInfo* session = GET_MY_SESSION(exec_ctx);
ObDistributedTransmitInput* input = GET_PHY_OP_INPUT(ObDistributedTransmitInput, exec_ctx, get_id());
ObIntermResultManager* result_mgr = ObIntermResultManager::get_instance();
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(exec_ctx);
ObDeterminateTaskTransmitCtx* op_ctx = NULL;
ObIntermResult** results = NULL;
TaskIDSet* tasks = executing_tasks();
bool task_added = false;
ObIntermResultInfo res_key;
ObSliceID slice_id;
if (NULL != input) {
slice_id.set_ob_task_id(input->get_ob_task_id());
}
// check whether interm result already exist
bool reuse_result = result_reusable_;
if (OB_ISNULL(child_op_) || get_split_task_count() <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("not inited", K(ret), KP(child_op_), "task_cnt", get_split_task_count());
} else if (OB_ISNULL(session) || OB_ISNULL(input) || OB_ISNULL(result_mgr) || OB_ISNULL(plan_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("got NULL context", KP(session), KP(input), KP(result_mgr), KP(plan_ctx));
} else if (NULL == tasks) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("NULL tasks", K(ret));
} else if (OB_FAIL(ObTransmit::inner_open(exec_ctx))) {
LOG_WARN("init operator context failed", K(ret));
} else if (OB_ISNULL(op_ctx = GET_PHY_OPERATOR_CTX(ObDeterminateTaskTransmitCtx, exec_ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get operator ctx failed", K(ret));
} else if (OB_FAIL(tasks->set_refactored(input->get_ob_task_id(), 0 /* no overwrite */))) {
if (OB_HASH_EXIST == ret) {
ret = OB_ENTRY_EXIST;
}
LOG_WARN("add task to set failed, may be in executing", K(ret), "task_id", input->get_ob_task_id());
} else {
task_added = true;
for (int64_t i = 0; OB_SUCC(ret) && reuse_result && i < get_split_task_count(); i++) {
slice_id.set_slice_id(i);
res_key.init(slice_id);
if (OB_FAIL(result_mgr->update_expire_time(res_key, input->get_expire_time()))) {
if (OB_ENTRY_NOT_EXIST == ret) {
reuse_result = false;
ret = OB_SUCCESS;
} else {
LOG_WARN("update expire time failed", K(ret), K(res_key), "expire_time", input->get_expire_time());
}
}
}
if (!reuse_result) {
if (OB_FAIL(delete_all_results(*result_mgr, input->get_ob_task_id(), get_split_task_count()))) {
LOG_WARN("delete all results failed", K(ret));
}
if (OB_SUCC(ret) && OPEN_SELF_ONLY == get_operator_open_order(exec_ctx)) {
if (OB_FAIL(child_op_->open(exec_ctx))) {
LOG_WARN("open child failed", K(ret));
}
op_ctx->close_child_manually_ = true;
}
}
}
// generate interm result
if (OB_SUCC(ret) && !reuse_result) {
ObTableLocation table_location;
Id2IdxMap partition_id2idx_map;
schema::ObSchemaGetterGuard schema_guard;
ObSqlSchemaGuard sql_schema_guard;
const int check_status_per_row = E(EventTable::EN_BKGD_TRANSMIT_CHECK_STATUS_PER_ROW) 0;
if (OB_ISNULL(exec_ctx.get_task_exec_ctx().schema_service_) || OB_ISNULL(get_phy_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL schema service or physical plan", K(ret));
} else if (OB_FAIL(exec_ctx.get_task_exec_ctx().schema_service_->get_tenant_schema_guard(
session->get_effective_tenant_id(), schema_guard, get_phy_plan()->get_tenant_schema_version()))) {
LOG_WARN("get schema guard failed", K(ret));
} else if (FALSE_IT(sql_schema_guard.set_schema_guard(&schema_guard))) {
} else if (OB_FAIL(alloc_result_array(exec_ctx, *result_mgr, get_split_task_count(), results))) {
LOG_WARN("alloc result array failed", K(ret));
} else {
const ObNewRow* row = NULL;
while (OB_SUCC(ret)) {
int64_t slice_idx = -1;
if (0 != check_status_per_row && OB_FAIL(check_status(exec_ctx))) {
LOG_WARN("check status failed", K(ret));
} else if (OB_FAIL(try_check_status(exec_ctx))) {
LOG_WARN("check status failed", K(ret));
} else if (OB_FAIL(get_next_row(exec_ctx, row))) {
if (OB_ITER_END != ret) {
LOG_WARN("get row from child failed", K(ret));
} else {
ret = OB_SUCCESS;
auto& res = *results[0];
res.set_found_rows(plan_ctx->get_found_rows());
res.set_affected_rows(plan_ctx->get_affected_rows());
res.set_matched_rows(plan_ctx->get_row_matched_count());
res.set_duplicated_rows(plan_ctx->get_row_duplicated_count());
res.set_last_insert_id_session(plan_ctx->calc_last_insert_id_session());
if (!plan_ctx->is_result_accurate()) {
res.set_is_result_accurate(false);
}
}
break;
} else if (OB_ISNULL(row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL row returned", K(ret));
} else if (OB_FAIL(shuffle_row(
exec_ctx, sql_schema_guard, table_location, partition_id2idx_map, *row, slice_idx))) {
LOG_WARN("shuffle row failed", K(ret));
} else if (slice_idx < 0 || slice_idx >= get_split_task_count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid slice idx", K(ret), K(slice_idx), "split_task_cnt", get_split_task_count());
} else if (OB_FAIL(results[slice_idx]->add_row(session->get_effective_tenant_id(), *row))) {
LOG_WARN("add row to interm result failed", K(ret));
}
}
}
if (OB_SUCC(ret)) {
for (int64_t i = 0; OB_SUCC(ret) && i < get_split_task_count(); i++) {
slice_id.set_slice_id(i);
res_key.init(slice_id);
if (OB_FAIL(results[i]->complete_add_rows(session->get_effective_tenant_id()))) {
LOG_WARN("complete add rows failed", K(ret));
} else if (OB_FAIL(result_mgr->add_result(res_key, results[i], input->get_expire_time()))) {
LOG_WARN("add result failed", K(ret));
} else {
results[i] = NULL;
}
}
}
}
// build slice events
if (OB_SUCC(ret)) {
auto events = input->get_slice_events_for_update();
if (OB_ISNULL(events)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL slice events", K(ret));
} else {
events->reset();
if (OB_FAIL(events->prepare_allocate(get_split_task_count()))) {
LOG_WARN("array prepare allocate failed", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < get_split_task_count(); i++) {
slice_id.set_slice_id(i);
res_key.init(slice_id);
events->at(i).set_ob_slice_id(slice_id);
}
}
}
}
if (OB_FAIL(ret) && NULL != results) {
int tmp_ret = delete_all_results(*result_mgr, input->get_ob_task_id(), get_split_task_count());
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("delete added results failed", K(tmp_ret));
}
tmp_ret = free_result_array(*result_mgr, get_split_task_count(), results);
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("free result array failed", K(tmp_ret));
}
}
if (task_added) {
int tmp_ret = tasks->erase_refactored(input->get_ob_task_id());
if (OB_SUCCESS != tmp_ret) {
ret = OB_SUCCESS == ret ? tmp_ret : ret;
}
}
return ret;
}
int ObDeterminateTaskTransmit::inner_close(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
auto* op_ctx = GET_PHY_OPERATOR_CTX(ObDeterminateTaskTransmitCtx, ctx, get_id());
if (NULL != op_ctx && NULL != child_op_ && op_ctx->close_child_manually_) {
if (OB_FAIL(child_op_->close(ctx))) {
LOG_WARN("close child failed", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(ObDistributedTransmit::inner_close(ctx))) {
LOG_WARN("distributed transmit close failed", K(ret));
}
}
return ret;
}
int ObDeterminateTaskTransmit::alloc_result_array(
ObExecContext& exec_ctx, ObIntermResultManager& mgr, const int64_t cnt, ObIntermResult**& results) const
{
int ret = OB_SUCCESS;
results = NULL;
if (OB_ISNULL(child_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("transmit op has no child", K(ret));
} else if (cnt > 0) {
results = static_cast<ObIntermResult**>(exec_ctx.get_allocator().alloc(sizeof(ObIntermResult*) * cnt));
if (OB_ISNULL(results)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc memory failed", K(ret));
} else {
MEMSET(results, 0, sizeof(ObIntermResult*) * cnt);
for (int64_t i = 0; OB_SUCC(ret) && i < cnt; i++) {
if (OB_FAIL(mgr.alloc_result(results[i]))) {
LOG_WARN("alloc result failed", K(ret));
} else if (NULL == results[i]) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL result", K(ret));
} else {
results[i]->set_row_reclaim_func(child_op_->reclaim_row_func());
}
}
if (OB_FAIL(ret)) {
int tmp_ret = free_result_array(mgr, cnt, results);
LOG_WARN("free result array failed", K(tmp_ret));
}
}
}
return ret;
}
int ObDeterminateTaskTransmit::free_result_array(
ObIntermResultManager& mgr, const int64_t cnt, ObIntermResult**& results) const
{
int ret = OB_SUCCESS;
for (int64_t i = 0; i < cnt; i++) {
if (NULL != results[i]) {
int tmp_ret = mgr.free_result(results[i]);
if (OB_SUCCESS != tmp_ret) {
LOG_ERROR("free result failed", K(tmp_ret));
ret = OB_SUCCESS == ret ? tmp_ret : ret;
}
results[i] = NULL;
}
}
return ret;
}
int ObDeterminateTaskTransmit::delete_all_results(
ObIntermResultManager& mgr, const ObTaskID& task_id, const int64_t cnt) const
{
int final_ret = OB_SUCCESS;
int ret = OB_SUCCESS;
ObSliceID slice_id;
slice_id.set_ob_task_id(task_id);
ObIntermResultInfo res_key;
for (int64_t i = 0; i < cnt; i++) {
slice_id.set_slice_id(i);
res_key.init(slice_id);
if (OB_FAIL(mgr.delete_result(res_key))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("delete result failed", K(ret));
}
if (OB_FAIL(ret) && OB_SUCCESS == final_ret) {
final_ret = ret;
}
}
}
return final_ret;
}
ERRSIM_POINT_DEF(ERRSIM_DETERMINATE_TRANSMIT_SHUFFLE_ROW);
int ObDeterminateTaskTransmit::shuffle_row(ObExecContext& exec_ctx, ObSqlSchemaGuard& schema_guard,
ObTableLocation& table_location, Id2IdxMap& partition_id2idx_map, const ObNewRow& row, int64_t& slice_idx) const
{
int ret = OB_SUCCESS;
slice_idx = 0;
int64_t part_idx = 0;
if (shuffle_by_part_) {
int64_t part_id = 0;
if (!table_location.is_inited()) {
const schema::ObTableSchema* table = NULL;
if (OB_FAIL(schema_guard.get_table_schema(repartition_table_id_, table))) {
LOG_WARN("get table schema failed", K(ret));
} else if (OB_ISNULL(table)) {
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("table not exist", K(ret), K(repartition_table_id_));
} else if (OB_FAIL(partition_id2idx_map.create(
hash::cal_next_prime(table->get_all_part_num()), ObModIds::OB_HASH_BUCKET))) {
LOG_WARN("create map failed", K(ret));
} else {
bool check_dropped_schema = false;
schema::ObTablePartitionKeyIter keys(*table, check_dropped_schema);
for (int64_t i = 0; OB_SUCC(ret) && i < keys.get_partition_num(); i++) {
if (OB_FAIL(keys.next_partition_id_v2(part_id))) {
LOG_WARN("get partition id failed", K(ret));
} else if (OB_FAIL(partition_id2idx_map.set_refactored(part_id, i))) {
LOG_WARN("add to map failed", K(ret));
}
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(table_location.init_table_location_with_rowkey(
schema_guard, repartition_table_id_, *exec_ctx.get_my_session()))) {
LOG_WARN("init table location failed", K(ret));
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(table_location.calculate_partition_id_by_row(
exec_ctx, schema_guard.get_schema_guard(), row, part_id))) {
LOG_WARN("calculate partition id by row failed", K(ret));
} else if (OB_FAIL(partition_id2idx_map.get_refactored(part_id, part_idx))) {
LOG_WARN("get partition index failed", K(part_id));
}
if (OB_SUCC(ret)) {
if (start_slice_ids_.empty()) {
slice_idx = part_idx;
} else if (part_idx >= start_slice_ids_.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("partitions more than start_slice_ids", K(ret));
} else {
slice_idx = start_slice_ids_.at(part_idx);
}
}
ret = ERRSIM_DETERMINATE_TRANSMIT_SHUFFLE_ROW ?: ret;
}
if (OB_SUCC(ret) && shuffle_by_range_) {
const auto& ranges = shuffle_ranges_.at(shuffle_ranges_.count() > 1 ? part_idx : 0);
if (!ranges.empty()) {
auto begin = &ranges.at(0);
auto end = begin + ranges.count();
RangeStartCompare range_row_cmp(ret);
auto iter = std::lower_bound(begin, end, row, range_row_cmp);
if (OB_FAIL(ret)) {
LOG_WARN("compare range and row failed", K(ret));
} else if (iter == begin) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ranges not start with min value", K(ret), K(row), "range", ranges.at(0));
} else {
slice_idx += iter - begin - 1;
}
}
}
if (OB_SUCC(ret)) {
LOG_DEBUG("shuffle row", K(ret), K(slice_idx), K(row));
}
return ret;
}
ObLatch ObDeterminateTaskTransmit::task_set_init_lock_;
ObDeterminateTaskTransmit::TaskIDSet ObDeterminateTaskTransmit::executing_task_set_instance_;
ObDeterminateTaskTransmit::TaskIDSet* ObDeterminateTaskTransmit::executing_tasks()
{
static volatile bool inited = false;
if (!inited) {
task_set_init_lock_.wrlock(0);
int ret = OB_SUCCESS;
if (!inited) {
if (OB_FAIL(executing_task_set_instance_.create(4096))) {
LOG_WARN("create set failed", K(ret));
} else {
inited = true;
}
}
task_set_init_lock_.unlock();
}
return inited ? &executing_task_set_instance_ : NULL;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,226 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_EXECUTOR_OB_DETERMINATE_TASK_TRANSMIT_H_
#define OCEANBASE_EXECUTOR_OB_DETERMINATE_TASK_TRANSMIT_H_
#include "sql/executor/ob_distributed_transmit.h"
#include "sql/executor/ob_task_info.h"
#include "lib/hash/ob_hashset.h"
namespace oceanbase {
namespace sql {
class ObTableLocation;
// Tasks are determined before execute, we sore those tasks in this operator.
class ObDeterminateTaskTransmit : public ObDistributedTransmit {
OB_UNIS_VERSION_V(1);
public:
class ITaskRouting {
public:
enum Policy {
DATA_REPLICA_PICKER,
INDEX_REPLICA_PICKER,
SAME_WITH_CHILD,
};
ITaskRouting()
{}
virtual ~ITaskRouting()
{}
virtual int route(
Policy policy, const ObTaskInfo& task, const common::ObIArray<common::ObAddr>& previous, ObAddr& server) = 0;
};
struct TaskIndex {
OB_UNIS_VERSION_V(1);
public:
int32_t loc_idx_;
int32_t part_loc_idx_;
TaskIndex() : loc_idx_(0), part_loc_idx_(0)
{}
TO_STRING_KV(K(loc_idx_), K(part_loc_idx_));
};
struct IdRange {
OB_UNIS_VERSION_V(1);
public:
int32_t begin_;
int32_t end_;
IdRange() : begin_(0), end_(0)
{}
TO_STRING_KV(K(begin_), K(end_));
};
struct ResultRange {
OB_UNIS_VERSION_V(1);
public:
IdRange task_range_;
IdRange slice_range_;
TO_STRING_KV(K(task_range_), K(slice_range_));
};
// compare ObNewRange::start_ and ObNewRow
struct RangeStartCompare;
private:
class ObDeterminateTaskTransmitCtx : public ObDistributedTransmitCtx {
public:
explicit ObDeterminateTaskTransmitCtx(ObExecContext& ctx)
: ObDistributedTransmitCtx(ctx), close_child_manually_(false)
{}
virtual ~ObDeterminateTaskTransmitCtx()
{}
virtual void destroy() override
{
ObDistributedTransmitCtx::destroy();
}
public:
bool close_child_manually_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDeterminateTaskTransmitCtx);
};
typedef common::hash::ObHashMap<int64_t, int64_t, common::hash::NoPthreadDefendMode> Id2IdxMap;
typedef common::hash::ObHashSet<ObTaskID> TaskIDSet;
public:
explicit ObDeterminateTaskTransmit(common::ObIAllocator& alloc);
virtual ~ObDeterminateTaskTransmit()
{}
virtual int init_op_ctx(ObExecContext& exec_ctx) const override;
virtual int inner_open(ObExecContext& exec_ctx) const override;
virtual int inner_close(ObExecContext& ctx) const override;
virtual OperatorOpenOrder get_operator_open_order(ObExecContext& ctx) const;
typedef common::ObFixedArray<ObTaskInfo::ObRangeLocation, common::ObIAllocator> RangeLocations;
typedef common::ObFixedArray<TaskIndex, common::ObIAllocator> Tasks;
typedef common::ObFixedArray<common::ObFixedArray<common::ObNewRange, common::ObIAllocator>, common::ObIAllocator>
ShuffleRanges;
typedef common::ObFixedArray<int64_t, common::ObIAllocator> StartSliceIds;
typedef common::ObFixedArray<ResultRange, common::ObIAllocator> ResultMapping;
void set_result_reusable(bool reusable)
{
result_reusable_ = reusable;
}
RangeLocations& get_range_locations()
{
return range_locations_;
}
Tasks& get_tasks()
{
return tasks_;
}
ShuffleRanges& get_shuffle_ranges()
{
return shuffle_ranges_;
}
StartSliceIds& get_start_slice_ids()
{
return start_slice_ids_;
}
ResultMapping& get_result_mapping()
{
return result_mapping_;
}
void set_shuffle_by_part()
{
shuffle_by_part_ = true;
}
void set_shuffle_by_range()
{
shuffle_by_range_ = true;
}
ITaskRouting* get_task_routing() const
{
return task_routing_;
}
void set_task_routing(ITaskRouting* routing)
{
task_routing_ = routing;
}
ITaskRouting::Policy get_task_route_policy() const
{
return task_route_policy_;
}
void set_task_routing_policy(ITaskRouting::Policy policy)
{
task_route_policy_ = policy;
}
void set_background(const bool v)
{
background_ = v;
}
bool is_background() const
{
return background_;
}
private:
int alloc_result_array(
ObExecContext& exec_ctx, ObIntermResultManager& mgr, const int64_t cnt, ObIntermResult**& results) const;
int free_result_array(ObIntermResultManager& mgr, const int64_t cnt, ObIntermResult**& results) const;
// delete all result, return the first error.
int delete_all_results(ObIntermResultManager& mgr, const ObTaskID& task_id, const int64_t cnt) const;
int shuffle_row(ObExecContext& exec_ctx, ObSqlSchemaGuard& schema_guard, ObTableLocation& table_location,
Id2IdxMap& partition_id2idx_map, const common::ObNewRow& row, int64_t& slice_idx) const;
private:
static common::ObLatch task_set_init_lock_;
static TaskIDSet executing_task_set_instance_;
static TaskIDSet* executing_tasks();
private:
bool result_reusable_;
RangeLocations range_locations_;
Tasks tasks_;
// shuffle info
// 1. shuffle by table partition
// 2. shuffle by range
bool shuffle_by_part_;
bool shuffle_by_range_;
// ranges must be: left open right close `(left, right]`,
// and cover the whole range (min, max), and has no overlay
ShuffleRanges shuffle_ranges_;
StartSliceIds start_slice_ids_;
// describe how the result are mapped to the receiver
ResultMapping result_mapping_;
ITaskRouting::Policy task_route_policy_;
ITaskRouting* task_routing_;
// run task in background threads
bool background_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_EXECUTOR_OB_DETERMINATE_TASK_TRANSMIT_H_

View File

@ -0,0 +1,405 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_direct_receive.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/executor/ob_task_executor_ctx.h"
#include "sql/executor/ob_executor_rpc_impl.h"
#include "share/ob_scanner.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/monitor/ob_exec_stat_collector.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObDirectReceive::ObDirectReceiveCtx::ObDirectReceiveCtx(ObExecContext& ctx)
: ObPhyOperatorCtx(ctx),
scanner_(NULL),
scanner_iter_(),
all_data_empty_(false),
cur_data_empty_(true),
first_request_received_(false),
found_rows_(0)
{}
ObDirectReceive::ObDirectReceiveCtx::~ObDirectReceiveCtx()
{}
//
//
//
///////////////////// end Context /////////////////////////////
//
//
ObDirectReceiveInput::ObDirectReceiveInput() : ObReceiveInput()
{}
ObDirectReceiveInput::~ObDirectReceiveInput()
{}
void ObDirectReceiveInput::reset()
{
ObReceiveInput::reset();
}
int ObDirectReceiveInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
{
int ret = OB_SUCCESS;
UNUSED(ctx);
UNUSED(task_info);
UNUSED(op);
return ret;
}
ObPhyOperatorType ObDirectReceiveInput::get_phy_op_type() const
{
return PHY_DIRECT_RECEIVE;
}
OB_SERIALIZE_MEMBER((ObDirectReceiveInput, ObReceiveInput));
//
//
///////////////////// End Input /////////////////////////////////
//
//
ObDirectReceive::ObDirectReceive(ObIAllocator& alloc) : ObReceive(alloc)
{}
ObDirectReceive::~ObDirectReceive()
{}
int ObDirectReceive::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObDirectReceiveCtx, ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("create physical operator context failed", K(ret));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ssucc to create op ctx, but op ctx is NULL", K(ret));
} else if (OB_FAIL(init_cur_row(*op_ctx, true))) {
LOG_WARN("init current row failed", K(ret));
}
return ret;
}
int ObDirectReceive::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObDirectReceiveCtx* recv_ctx = NULL;
if (OB_FAIL(init_op_ctx(ctx))) {
LOG_WARN("initialize operator context failed", K(ret));
} else if (OB_ISNULL(recv_ctx = GET_PHY_OPERATOR_CTX(ObDirectReceiveCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail get phy op ctx", "op_id", get_id(), "op_type", get_type());
} else {
recv_ctx->all_data_empty_ = false;
recv_ctx->cur_data_empty_ = true;
recv_ctx->first_request_received_ = false;
if (OB_FAIL(setup_next_scanner(ctx))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("failed to setup first scanner", K(ret));
} else {
recv_ctx->all_data_empty_ = true;
}
} else {
recv_ctx->cur_data_empty_ = false;
}
}
return ret;
}
int ObDirectReceive::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
bool has_got_a_row = false;
RemoteExecuteStreamHandle* resp_handler = NULL;
ObDirectReceiveCtx* recv_ctx = GET_PHY_OPERATOR_CTX(ObDirectReceiveCtx, ctx, get_id());
// Sometimes we need send user variables to remote end to complete query successfully
// And, of course, we will get the new value for user variable.
// Scanner contains the updated values.
// so we update the user variables in terms of scanners here.
if (OB_ISNULL(recv_ctx)) {
ret = OB_ERR_UNEXPECTED;
} else if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(ctx, resp_handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_ISNULL(resp_handler)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("resp_handler is NULL", K(ret));
} else if (OB_FAIL(THIS_WORKER.check_status())) {
LOG_WARN("check physical plan status failed", K(ret));
} else if (OB_ERR_TASK_SKIPPED == resp_handler->get_result_code()) {
ret = OB_ITER_END;
LOG_WARN("this remote task is skipped", K(ret));
}
/* following is an state machine */
while (OB_SUCC(ret) && false == has_got_a_row) {
if (recv_ctx->all_data_empty_) { /* all data is read */
ret = OB_ITER_END;
} else if (recv_ctx->cur_data_empty_) { /* current scanner is read */
/* send RPC request, remote returns a scanner */
if (OB_FAIL(setup_next_scanner(ctx))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail to setup next scanner", K(ret));
} else {
recv_ctx->all_data_empty_ = true; /* no more scanner */
}
} else {
recv_ctx->cur_data_empty_ = false; /* scanner is filled once again */
}
} else { /* current scanner is readable */
if (OB_FAIL(get_next_row_from_cur_scanner(*recv_ctx, row))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail to get next row from cur scanner", K(ret));
} else {
// current scanner is read
recv_ctx->cur_data_empty_ = true;
ret = OB_SUCCESS;
}
} else {
// retrive one row, exit loop
has_got_a_row = true;
}
}
}
if (OB_ITER_END == ret) {
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx);
if (OB_ISNULL(plan_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("plan ctx is NULL", K(ret));
} else {
plan_ctx->set_found_rows(recv_ctx->found_rows_);
}
}
return ret;
}
int ObDirectReceive::inner_close(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
RemoteExecuteStreamHandle* resp_handler = NULL;
if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(ctx, resp_handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_ISNULL(resp_handler)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("resp_handler is NULL", K(ret));
} else {
if (resp_handler->has_more()) {
if (OB_FAIL(resp_handler->abort())) {
LOG_WARN("fail to abort", K(ret));
} else {
ObSQLSessionInfo* session = ctx.get_my_session();
ObPhysicalPlanCtx* plan_ctx = ctx.get_physical_plan_ctx();
ObExecutorRpcImpl* rpc = NULL;
if (OB_FAIL(ObTaskExecutorCtxUtil::get_task_executor_rpc(ctx, rpc))) {
LOG_WARN("get task executor rpc failed", K(ret));
} else if (OB_ISNULL(session) || OB_ISNULL(plan_ctx) || OB_ISNULL(rpc)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session or plan ctx or rpc is NULL", K(ret));
} else {
ObQueryRetryInfo retry_info;
ObExecutorRpcCtx rpc_ctx(session->get_effective_tenant_id(),
plan_ctx->get_timeout_timestamp(),
ctx.get_task_exec_ctx().get_min_cluster_version(),
&retry_info,
session,
plan_ctx->is_plain_select_stmt());
int tmp_ret = rpc->task_kill(rpc_ctx, resp_handler->get_task_id(), resp_handler->get_dst_addr());
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("kill task failed", K(tmp_ret), K(resp_handler->get_task_id()), K(resp_handler->get_dst_addr()));
}
}
}
} else {
}
}
return ret;
}
int ObDirectReceive::setup_next_scanner(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhysicalPlanCtx* plan_ctx = NULL;
RemoteExecuteStreamHandle* resp_handler = NULL;
ObSQLSessionInfo* my_session = NULL;
ObDirectReceiveCtx* recv_ctx = GET_PHY_OPERATOR_CTX(ObDirectReceiveCtx, ctx, get_id());
if (OB_ISNULL(recv_ctx)) {
LOG_WARN("fail get phy op ctx");
ret = OB_ERR_UNEXPECTED;
} else if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(ctx, resp_handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_ISNULL(resp_handler)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("resp_handler is NULL", K(ret));
} else if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail get phy plan ctx", K(ret));
} else if (OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail get my session", K(ret));
}
if (OB_SUCC(ret)) {
/* reads data first time, result has been retrived in task_submit called by Scheduler */
if (!recv_ctx->first_request_received_) {
ObScanner* scanner = resp_handler->get_result();
if (OB_ISNULL(scanner)) {
ret = OB_ERR_UNEXPECTED;
} else {
// set last_insert_id no matter success or fail
plan_ctx->set_last_insert_id_to_client(scanner->get_last_insert_id_to_client());
plan_ctx->set_last_insert_id_session(scanner->get_last_insert_id_session());
plan_ctx->set_last_insert_id_changed(scanner->get_last_insert_id_changed());
int tmp_ret = OB_SUCCESS;
ObExecStatCollector& collector = ctx.get_exec_stat_collector();
if (OB_SUCCESS != (tmp_ret = collector.add_raw_stat(scanner->get_extend_info()))) {
LOG_WARN("fail to collected raw extend info in scanner", K(tmp_ret));
}
if (OB_FAIL(scanner->get_err_code())) {
int add_ret = OB_SUCCESS;
const char* err_msg = scanner->get_err_msg();
FORWARD_USER_ERROR(ret, err_msg);
LOG_WARN("while fetching first scanner, the remote rcode is not OB_SUCCESS",
K(ret),
K(err_msg),
"dst_addr",
to_cstring(resp_handler->get_dst_addr()));
if (is_data_not_readable_err(ret)) {
// slave replays log's copy
ObQueryRetryInfo& retry_info = my_session->get_retry_info_for_update();
if (OB_UNLIKELY(OB_SUCCESS !=
(add_ret = retry_info.add_invalid_server_distinctly(resp_handler->get_dst_addr(), true)))) {
LOG_WARN("fail to add remote addr to invalid servers distinctly",
K(ret),
K(add_ret),
K(resp_handler->get_dst_addr()),
K(retry_info));
}
}
} else {
recv_ctx->scanner_ = scanner;
recv_ctx->scanner_iter_ = scanner->begin();
recv_ctx->first_request_received_ = true;
plan_ctx->set_affected_rows(scanner->get_affected_rows());
recv_ctx->found_rows_ += scanner->get_found_rows();
if (OB_FAIL(plan_ctx->set_row_matched_count(scanner->get_row_matched_count()))) {
LOG_WARN("fail to set row matched count", K(ret), K(scanner->get_row_matched_count()));
} else if (OB_FAIL(plan_ctx->set_row_duplicated_count(scanner->get_row_duplicated_count()))) {
LOG_WARN("fail to set row duplicate count", K(ret), K(scanner->get_row_duplicated_count()));
// } else if (OB_FAIL(my_session->get_trans_result().merge_result(scanner->get_trans_result()))) {
// LOG_WARN("merge trans result to session failed", K(ret));
/**
* ObRemoteTaskExecutor::execute() has called merge_result() before here, that is a
* better place to call merge_result(), especially when any operation failed between
* there and here.
*/
} else if (OB_FAIL(plan_ctx->merge_implicit_cursors(scanner->get_implicit_cursors()))) {
LOG_WARN("merge implicit cursors failed", K(ret), K(scanner->get_implicit_cursors()));
}
}
}
} else {
ObScanner* result_scanner = NULL;
if (resp_handler->has_more()) {
if (OB_FAIL(resp_handler->reset_and_init_result())) {
LOG_WARN("fail reset and init result", K(ret));
} else if (OB_ISNULL(result_scanner = resp_handler->get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succ to alloc result, but result scanner is NULL", K(ret));
} else if (OB_FAIL(resp_handler->get_more(*result_scanner))) {
LOG_WARN("fail wait response", K(ret), "dst_addr", to_cstring(resp_handler->get_dst_addr()));
} else if (OB_FAIL(result_scanner->get_err_code())) {
int add_ret = OB_SUCCESS;
const char* err_msg = result_scanner->get_err_msg();
FORWARD_USER_ERROR(ret, err_msg);
LOG_WARN("while getting more scanner, the remote rcode is not OB_SUCCESS",
K(ret),
K(err_msg),
"dst_addr",
to_cstring(resp_handler->get_dst_addr()));
if (is_data_not_readable_err(ret)) {
ObQueryRetryInfo& retry_info = my_session->get_retry_info_for_update();
if (OB_UNLIKELY(OB_SUCCESS !=
(add_ret = retry_info.add_invalid_server_distinctly(resp_handler->get_dst_addr(), true)))) {
LOG_WARN("fail to add remote addr to invalid servers distinctly",
K(ret),
K(add_ret),
K(resp_handler->get_dst_addr()),
K(retry_info));
}
}
} else {
recv_ctx->scanner_ = result_scanner;
recv_ctx->scanner_iter_ = recv_ctx->scanner_->begin();
recv_ctx->found_rows_ += recv_ctx->scanner_->get_found_rows();
}
} else {
ret = OB_ITER_END;
// only successful select affect last_insert_id
// for select, last_insert_id may changed because last_insert_id(#) called
// last_insert_id values should be the last row calling last_insert_id(#)
plan_ctx->set_last_insert_id_session(recv_ctx->scanner_->get_last_insert_id_session());
plan_ctx->set_last_insert_id_changed(recv_ctx->scanner_->get_last_insert_id_changed());
int tmp_ret = OB_SUCCESS;
ObExecStatCollector& collector = ctx.get_exec_stat_collector();
if (OB_SUCCESS != (tmp_ret = collector.add_raw_stat(recv_ctx->scanner_->get_extend_info()))) {
LOG_WARN("fail to collected raw extend info in scanner", K(tmp_ret));
}
if (OB_SUCCESS !=
(tmp_ret = plan_ctx->get_table_row_count_list().assign(recv_ctx->scanner_->get_table_row_counts()))) {
LOG_WARN("fail to set table row count", K(ret), K(recv_ctx->scanner_->get_table_row_counts()));
}
LOG_DEBUG("remote table row counts",
K(recv_ctx->scanner_->get_table_row_counts()),
K(plan_ctx->get_table_row_count_list()));
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(my_session->replace_user_variables(ctx, recv_ctx->scanner_->get_session_var_map()))) {
LOG_WARN("replace user variables failed", K(ret));
}
}
return ret;
}
int ObDirectReceive::get_next_row_from_cur_scanner(ObDirectReceiveCtx& op_ctx, const common::ObNewRow*& row) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(op_ctx.scanner_iter_.get_next_row(op_ctx.get_cur_row()))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail get next row", K(ret));
} else {
}
} else {
row = &op_ctx.get_cur_row();
}
return ret;
}
int ObDirectReceive::rescan(ObExecContext& ctx) const
{
UNUSED(ctx);
int ret = OB_NOT_SUPPORTED;
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Distributed rescan");
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,98 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DIRECT_RECEIVE_
#define OCEANBASE_SQL_EXECUTOR_OB_DIRECT_RECEIVE_
#include "sql/executor/ob_receive.h"
#include "sql/engine/ob_phy_operator_type.h"
#include "sql/engine/ob_phy_operator.h"
#include "share/ob_scanner.h"
namespace oceanbase {
namespace common {
class ObNewRow;
}
namespace sql {
class ObExecContext;
class ObTaskInfo;
class ObDirectReceiveInput : public ObReceiveInput {
OB_UNIS_VERSION_V(1);
public:
ObDirectReceiveInput();
virtual ~ObDirectReceiveInput();
virtual void reset() override;
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op) override;
virtual ObPhyOperatorType get_phy_op_type() const;
private:
/* functions */
/* variables */
DISALLOW_COPY_AND_ASSIGN(ObDirectReceiveInput);
};
class ObDirectReceive : public ObReceive {
private:
class ObDirectReceiveCtx : public ObPhyOperatorCtx {
friend class ObDirectReceive;
public:
explicit ObDirectReceiveCtx(ObExecContext& ctx);
virtual ~ObDirectReceiveCtx();
virtual void destroy()
{
ObPhyOperatorCtx::destroy_base();
}
/* variables */
common::ObScanner* scanner_;
common::ObScanner::Iterator scanner_iter_;
bool all_data_empty_;
bool cur_data_empty_;
bool first_request_received_;
int64_t found_rows_;
private:
/* functions */
/* variables */
DISALLOW_COPY_AND_ASSIGN(ObDirectReceiveCtx);
};
public:
explicit ObDirectReceive(common::ObIAllocator& alloc);
virtual ~ObDirectReceive();
virtual int rescan(ObExecContext& ctx) const;
private:
/* functions */
int setup_next_scanner(ObExecContext& ctx) const;
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
virtual int inner_open(ObExecContext& ctx) const;
virtual int inner_close(ObExecContext& ctx) const;
int get_next_row_from_cur_scanner(ObDirectReceiveCtx& ctx, const common::ObNewRow*& row) const;
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
private:
/* macros */
DISALLOW_COPY_AND_ASSIGN(ObDirectReceive);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DIRECT_RECEIVE_ */
//// end of header file

View File

@ -0,0 +1,326 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_direct_receive_op.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/executor/ob_task_executor_ctx.h"
#include "sql/executor/ob_executor_rpc_impl.h"
#include "share/ob_scanner.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/monitor/ob_exec_stat_collector.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
OB_SERIALIZE_MEMBER((ObDirectReceiveSpec, ObOpSpec));
ObDirectReceiveOp::ObDirectReceiveOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObReceiveOp(exec_ctx, spec, input),
scanner_(NULL),
scanner_iter_(),
all_data_empty_(false),
cur_data_empty_(true),
first_request_received_(false),
found_rows_(0)
{}
int ObDirectReceiveOp::inner_open()
{
int ret = OB_SUCCESS;
all_data_empty_ = false; /* Whether all the scanner data has been read */
cur_data_empty_ = true; /* Whether the current scanner data has been read */
first_request_received_ = false; /* Whether the plan has been sent to the remote server */
// Receive the first scanner. For DML such as insert and update, need to get affected_rows etc.
if (OB_FAIL(setup_next_scanner())) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("failed to setup first scanner", K(ret));
} else {
all_data_empty_ = true; /* no more scanner */
}
} else {
cur_data_empty_ = false; /* scanner is filled up again, can continue reading */
}
return ret;
}
/*
* state:cur_data_empty_, all_data_empty_
*/
int ObDirectReceiveOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
bool has_got_a_row = false;
RemoteExecuteStreamHandle* resp_handler = NULL;
// Sometimes we need send user variables to remote end to complete query successfully
// And, of course, we will get the new value for user variable.
// Scanner contains the updated values.
// so we update the user variables in terms of scanners here.
if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(ctx_, resp_handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_ISNULL(resp_handler)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("resp_handler is NULL", K(ret));
} else if (OB_FAIL(THIS_WORKER.check_status())) {
LOG_WARN("check physical plan status failed", K(ret));
} else if (OB_ERR_TASK_SKIPPED == resp_handler->get_result_code()) {
// skip
ret = OB_ITER_END;
LOG_WARN("this remote task is skipped", K(ret));
}
/* It will be easier to understand the following code with state machine thinking */
while (OB_SUCC(ret) && false == has_got_a_row) {
if (all_data_empty_) { /* All data has been read */
ret = OB_ITER_END;
} else if (cur_data_empty_) { /* The current scanner has been read over */
/* send RPC req and remote server response Scanner */
if (OB_FAIL(setup_next_scanner())) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail to setup next scanner", K(ret));
} else {
all_data_empty_ = true; /* no more scanner */
}
} else {
cur_data_empty_ = false; /* scanner is filled up again, can continue reading */
}
} else { /* current scanner is readable */
if (OB_FAIL(get_next_row_from_cur_scanner())) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail to get next row from cur scanner", K(ret));
} else {
// curr scanner read over
cur_data_empty_ = true;
ret = OB_SUCCESS; // set ret be OB_SUCCESS for loop
}
} else {
// get one row then break loop
has_got_a_row = true;
}
}
}
if (OB_ITER_END == ret) {
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
plan_ctx->set_found_rows(found_rows_);
}
return ret;
}
int ObDirectReceiveOp::inner_close()
{
int ret = OB_SUCCESS;
RemoteExecuteStreamHandle* resp_handler = NULL;
if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(ctx_, resp_handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_ISNULL(resp_handler)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("resp_handler is NULL", K(ret));
} else {
if (resp_handler->has_more()) {
if (OB_FAIL(resp_handler->abort())) {
LOG_WARN("fail to abort", K(ret));
} else {
ObSQLSessionInfo* session = ctx_.get_my_session();
ObPhysicalPlanCtx* plan_ctx = ctx_.get_physical_plan_ctx();
ObExecutorRpcImpl* rpc = NULL;
if (OB_FAIL(ObTaskExecutorCtxUtil::get_task_executor_rpc(ctx_, rpc))) {
LOG_WARN("get task executor rpc failed", K(ret));
} else if (OB_ISNULL(session) || OB_ISNULL(plan_ctx) || OB_ISNULL(rpc)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session or plan ctx or rpc is NULL", K(ret));
} else {
ObQueryRetryInfo retry_info;
ObExecutorRpcCtx rpc_ctx(session->get_effective_tenant_id(),
plan_ctx->get_timeout_timestamp(),
ctx_.get_task_exec_ctx().get_min_cluster_version(),
&retry_info,
session,
plan_ctx->is_plain_select_stmt());
int tmp_ret = rpc->task_kill(rpc_ctx, resp_handler->get_task_id(), resp_handler->get_dst_addr());
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("kill task failed", K(tmp_ret), K(resp_handler->get_task_id()), K(resp_handler->get_dst_addr()));
}
}
}
} else {
}
}
return ret;
}
int ObDirectReceiveOp::setup_next_scanner()
{
int ret = OB_SUCCESS;
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
RemoteExecuteStreamHandle* resp_handler = NULL;
ObSQLSessionInfo* my_session = GET_MY_SESSION(ctx_);
if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(ctx_, resp_handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_ISNULL(resp_handler)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("resp_handler is NULL", K(ret));
}
if (OB_SUCC(ret)) {
// Read the data the first time, and the result has been obtained when task_submit() is called by the Scheduler
if (!first_request_received_) {
ObScanner* scanner = resp_handler->get_result();
if (OB_ISNULL(scanner)) {
ret = OB_ERR_UNEXPECTED;
} else {
// set last_insert_id no matter success or fail
plan_ctx->set_last_insert_id_to_client(scanner->get_last_insert_id_to_client());
plan_ctx->set_last_insert_id_session(scanner->get_last_insert_id_session());
plan_ctx->set_last_insert_id_changed(scanner->get_last_insert_id_changed());
int tmp_ret = OB_SUCCESS;
ObExecStatCollector& collector = ctx_.get_exec_stat_collector();
if (OB_SUCCESS != (tmp_ret = collector.add_raw_stat(scanner->get_extend_info()))) {
LOG_WARN("fail to collected raw extend info in scanner", K(tmp_ret));
}
if (OB_FAIL(scanner->get_err_code())) {
int add_ret = OB_SUCCESS;
const char* err_msg = scanner->get_err_msg();
// after FORWARD_USER_ERROR(ret, err_msg),if err_msg length > 0,
// then return err_msg
// Otherwise, the err_msg returned is the default error message corresponding to ret.
// use FORWARD_USER_ERROR(ret, err_msg) can qualify.
FORWARD_USER_ERROR(ret, err_msg);
LOG_WARN("while fetching first scanner, the remote rcode is not OB_SUCCESS",
K(ret),
K(err_msg),
"dst_addr",
to_cstring(resp_handler->get_dst_addr()));
if (is_data_not_readable_err(ret)) {
ObQueryRetryInfo& retry_info = my_session->get_retry_info_for_update();
if (OB_UNLIKELY(OB_SUCCESS !=
(add_ret = retry_info.add_invalid_server_distinctly(resp_handler->get_dst_addr(), true)))) {
LOG_WARN("fail to add remote addr to invalid servers distinctly",
K(ret),
K(add_ret),
K(resp_handler->get_dst_addr()),
K(retry_info));
}
}
} else {
scanner_ = scanner;
first_request_received_ = true;
// INSERT,UPDATE,DELETE,The Scanner returned for the first time contains the affected row
plan_ctx->set_affected_rows(scanner->get_affected_rows());
found_rows_ += scanner->get_found_rows();
if (OB_FAIL(scanner->get_datum_store().begin(scanner_iter_))) {
LOG_WARN("fail to init datum store iter", K(ret));
} else if (OB_FAIL(plan_ctx->set_row_matched_count(scanner->get_row_matched_count()))) {
LOG_WARN("fail to set row matched count", K(ret), K(scanner->get_row_matched_count()));
} else if (OB_FAIL(plan_ctx->set_row_duplicated_count(scanner->get_row_duplicated_count()))) {
LOG_WARN("fail to set row duplicate count", K(ret), K(scanner->get_row_duplicated_count()));
/**
* ObRemoteTaskExecutor::execute() has called merge_result() before here, that is a
* better place to call merge_result(), especially when any operation failed between
* there and here.
*/
} else if (OB_FAIL(plan_ctx->merge_implicit_cursors(scanner->get_implicit_cursors()))) {
LOG_WARN("merge implicit cursors failed", K(ret), K(scanner->get_implicit_cursors()));
}
}
}
} else { /* Subsequent request, send SESSION_NEXT to the remote end through Handle */
ObScanner* result_scanner = NULL;
if (resp_handler->has_more()) {
if (OB_FAIL(resp_handler->reset_and_init_result())) {
LOG_WARN("fail reset and init result", K(ret));
} else if (OB_ISNULL(result_scanner = resp_handler->get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succ to alloc result, but result scanner is NULL", K(ret));
} else if (OB_FAIL(resp_handler->get_more(*result_scanner))) {
LOG_WARN("fail wait response", K(ret), "dst_addr", to_cstring(resp_handler->get_dst_addr()));
} else if (OB_FAIL(result_scanner->get_err_code())) {
int add_ret = OB_SUCCESS;
const char* err_msg = result_scanner->get_err_msg();
FORWARD_USER_ERROR(ret, err_msg);
LOG_WARN("while getting more scanner, the remote rcode is not OB_SUCCESS",
K(ret),
K(err_msg),
"dst_addr",
to_cstring(resp_handler->get_dst_addr()));
if (is_data_not_readable_err(ret)) {
// Add the remote observer to the invalid servers of retry info
ObQueryRetryInfo& retry_info = my_session->get_retry_info_for_update();
if (OB_UNLIKELY(OB_SUCCESS !=
(add_ret = retry_info.add_invalid_server_distinctly(resp_handler->get_dst_addr(), true)))) {
LOG_WARN("fail to add remote addr to invalid servers distinctly",
K(ret),
K(add_ret),
K(resp_handler->get_dst_addr()),
K(retry_info));
}
}
} else {
scanner_ = result_scanner;
found_rows_ += scanner_->get_found_rows();
if (OB_FAIL(scanner_->get_datum_store().begin(scanner_iter_))) {
LOG_WARN("fail to init datum store iter", K(ret));
}
}
} else {
ret = OB_ITER_END;
// only successful select affect last_insert_id
// for select, last_insert_id may changed because last_insert_id(#) called
// last_insert_id values should be the last row calling last_insert_id(#)
plan_ctx->set_last_insert_id_session(scanner_->get_last_insert_id_session());
plan_ctx->set_last_insert_id_changed(scanner_->get_last_insert_id_changed());
int tmp_ret = OB_SUCCESS;
ObExecStatCollector& collector = ctx_.get_exec_stat_collector();
if (OB_SUCCESS != (tmp_ret = collector.add_raw_stat(scanner_->get_extend_info()))) {
LOG_WARN("fail to collected raw extend info in scanner", K(tmp_ret));
}
if (OB_SUCCESS != (tmp_ret = plan_ctx->get_table_row_count_list().assign(scanner_->get_table_row_counts()))) {
LOG_WARN("fail to set table row count", K(ret), K(scanner_->get_table_row_counts()));
}
LOG_DEBUG(
"remote table row counts", K(scanner_->get_table_row_counts()), K(plan_ctx->get_table_row_count_list()));
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(my_session->replace_user_variables(scanner_->get_session_var_map()))) {
LOG_WARN("replace user variables failed", K(ret));
}
}
return ret;
}
int ObDirectReceiveOp::get_next_row_from_cur_scanner()
{
int ret = OB_SUCCESS;
if (OB_FAIL(scanner_iter_.get_next_row(MY_SPEC.output_, eval_ctx_))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail get next row", K(ret));
} else {
}
} else {
LOG_DEBUG("direct receive next row", "row", ROWEXPR2STR(eval_ctx_, MY_SPEC.output_));
}
return ret;
}
int ObDirectReceiveOp::rescan()
{
// not support the rescan operation of the remote operator
int ret = OB_NOT_SUPPORTED;
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Distributed rescan");
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,68 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DIRECT_RECEIVE_OP_
#define OCEANBASE_SQL_EXECUTOR_OB_DIRECT_RECEIVE_OP_
#include "sql/engine/px/exchange/ob_receive_op.h"
#include "share/ob_scanner.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObDirectReceiveSpec : public ObReceiveSpec {
OB_UNIS_VERSION_V(1);
public:
ObDirectReceiveSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type) : ObReceiveSpec(alloc, type)
{}
virtual ~ObDirectReceiveSpec(){};
};
class ObDirectReceiveOp : public ObReceiveOp {
public:
ObDirectReceiveOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
virtual ~ObDirectReceiveOp()
{}
virtual int inner_open() override;
virtual int inner_get_next_row() override;
int inner_close();
virtual int rescan() override;
virtual void destroy() override
{
ObReceiveOp::destroy();
}
private:
int setup_next_scanner();
int get_next_row_from_cur_scanner();
int update_user_var();
private:
common::ObScanner* scanner_;
ObChunkDatumStore::Iterator scanner_iter_;
bool all_data_empty_;
bool cur_data_empty_;
bool first_request_received_;
int64_t found_rows_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDirectReceiveOp);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DIRECT_RECEIVE_OP_ */
//// end of header file

View File

@ -0,0 +1,104 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_direct_transmit.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
namespace sql {
using namespace oceanbase::common;
ObDirectTransmitInput::ObDirectTransmitInput() : ObTransmitInput()
{}
ObDirectTransmitInput::~ObDirectTransmitInput()
{}
int ObDirectTransmitInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
{
UNUSED(ctx);
UNUSED(task_info);
UNUSED(op);
return OB_SUCCESS;
}
OB_SERIALIZE_MEMBER((ObDirectTransmitInput, ObTransmitInput));
ObDirectTransmit::ObDirectTransmit(common::ObIAllocator& alloc) : ObTransmit(alloc)
{}
ObDirectTransmit::~ObDirectTransmit()
{}
int ObDirectTransmit::get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ret = ObPhyOperator::get_next_row(ctx, row);
return ret;
}
int ObDirectTransmit::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObDirectTransmitCtx* direct_ctx = NULL;
if (OB_ISNULL(child_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child_op_ is NULL", K(ret));
} else if (OB_ISNULL(direct_ctx = GET_PHY_OPERATOR_CTX(ObDirectTransmitCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("direct ctx is NULL", K(ret));
} else if (OB_FAIL(child_op_->get_next_row(ctx, row))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail to get next row from child op", K(ret), K(*child_op_));
}
} else if (OB_FAIL(copy_cur_row(*direct_ctx, row))) {
LOG_WARN("fail to copy current row", "op_type", ob_phy_operator_type_str(child_op_->get_type()), K(ret));
}
return ret;
}
int ObDirectTransmit::create_operator_input(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObIPhyOperatorInput* input = NULL;
if (OB_FAIL(CREATE_PHY_OP_INPUT(ObDirectTransmitInput, ctx, get_id(), get_type(), input))) {
LOG_WARN("fail to create phy op input", K(ret), K(get_id()), K(get_type()));
}
UNUSED(input);
return ret;
}
int ObDirectTransmit::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_UNLIKELY(calc_exprs_.get_size() > 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("calc exprs should be empty", K(ret), K(calc_exprs_.get_size()));
} else if (OB_UNLIKELY(filter_exprs_.get_size() > 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("filter exprs should be empty", K(ret), K(filter_exprs_.get_size()));
} else if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObDirectTransmitCtx, ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
} else if (OB_ISNULL((op_ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op ctx is NULL", K(ret));
} else if (OB_FAIL(init_cur_row(*op_ctx, false))) {
LOG_WARN("fail to init cur row", K(ret));
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,84 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DIRECT_TRANSMIT_
#define OCEANBASE_SQL_EXECUTOR_OB_DIRECT_TRANSMIT_
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_single_child_phy_operator.h"
#include "sql/executor/ob_transmit.h"
namespace oceanbase {
namespace sql {
class ObDirectTransmitInput : public ObTransmitInput {
OB_UNIS_VERSION_V(1);
public:
ObDirectTransmitInput();
virtual ~ObDirectTransmitInput();
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
virtual ObPhyOperatorType get_phy_op_type() const
{
return PHY_DIRECT_TRANSMIT;
}
private:
DISALLOW_COPY_AND_ASSIGN(ObDirectTransmitInput);
};
class ObDirectTransmit : public ObTransmit {
private:
class ObDirectTransmitCtx : public ObTransmitCtx {
friend class ObDirectTransmit;
public:
explicit ObDirectTransmitCtx(ObExecContext& ctx) : ObTransmitCtx(ctx)
{}
virtual ~ObDirectTransmitCtx()
{}
virtual void destroy()
{
ObTransmitCtx::destroy();
}
private:
DISALLOW_COPY_AND_ASSIGN(ObDirectTransmitCtx);
};
public:
explicit ObDirectTransmit(common::ObIAllocator& alloc);
virtual ~ObDirectTransmit();
virtual int create_operator_input(ObExecContext& ctx) const;
protected:
int get_next_row(ObExecContext& ctx, const ObNewRow*& row) const override;
private:
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
private:
// disallow copy assign
DISALLOW_COPY_AND_ASSIGN(ObDirectTransmit);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DIRECT_TRANSMIT_ */
//// end of header file

View File

@ -0,0 +1,26 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_direct_transmit_op.h"
namespace oceanbase {
namespace sql {
using namespace oceanbase::common;
OB_SERIALIZE_MEMBER((ObDirectTransmitOpInput, ObTransmitOpInput));
OB_SERIALIZE_MEMBER((ObDirectTransmitSpec, ObTransmitSpec));
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,73 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DIRECT_TRANSMIT_OP_
#define OCEANBASE_SQL_EXECUTOR_OB_DIRECT_TRANSMIT_OP_
#include "sql/engine/px/exchange/ob_transmit_op.h"
namespace oceanbase {
namespace sql {
class ObDirectTransmitOpInput : public ObTransmitOpInput {
OB_UNIS_VERSION_V(1);
public:
ObDirectTransmitOpInput(ObExecContext& ctx, const ObOpSpec& spec) : ObTransmitOpInput(ctx, spec)
{}
virtual ~ObDirectTransmitOpInput(){};
virtual int init(ObTaskInfo& task_info) override
{
UNUSED(task_info);
return common::OB_NOT_SUPPORTED;
}
private:
DISALLOW_COPY_AND_ASSIGN(ObDirectTransmitOpInput);
};
class ObDirectTransmitSpec : public ObTransmitSpec {
OB_UNIS_VERSION_V(1);
public:
ObDirectTransmitSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type) : ObTransmitSpec(alloc, type)
{}
virtual ~ObDirectTransmitSpec(){};
};
class ObDirectTransmitOp : public ObTransmitOp {
public:
ObDirectTransmitOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObTransmitOp(exec_ctx, spec, input)
{}
virtual ~ObDirectTransmitOp()
{}
virtual int inner_open() override
{
return common::OB_NOT_SUPPORTED;
}
virtual int inner_get_next_row()
{
return common::OB_NOT_SUPPORTED;
}
private:
DISALLOW_COPY_AND_ASSIGN(ObDirectTransmitOp);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DIRECT_TRANSMIT_OP_ */
//// end of header file

View File

@ -0,0 +1,100 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_distributed_job_control.h"
#include "lib/utility/ob_tracepoint.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObDistributedJobControl::ObDistributedJobControl()
{}
ObDistributedJobControl::~ObDistributedJobControl()
{}
int ObDistributedJobControl::get_ready_jobs(ObIArray<ObJob*>& jobs, bool serial_sched) const
{
int ret = OB_SUCCESS;
UNUSED(serial_sched);
bool all_finish = true;
jobs.reset();
for (int64_t i = 1; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
bool can_exec = true;
if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
SQL_EXE_LOG(WARN, "job is NULL", K(ret), K(i), K(jobs_.count()));
} else if (OB_JOB_STATE_FINISHED != job->get_state()) {
all_finish = false;
if (OB_FAIL(job->job_can_exec(can_exec))) {
LOG_WARN("fail to get job can exec", K(ret));
} else if (!can_exec) {
// nothing.
} else if (OB_FAIL(OB_I(t1) jobs.push_back(job))) {
LOG_WARN("fail to push job into array", K(ret));
} else if (serial_sched) {
break;
}
}
}
if (all_finish && OB_SUCCESS == ret) {
ret = OB_ITER_END;
}
return ret;
}
int ObDistributedJobControl::sort_job_scan_part_locs(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job is NULL", K(ret), K(i));
} else if (OB_FAIL(job->sort_scan_partition_locations(ctx))) {
LOG_WARN("fail to sort scan partition locations", K(ret), K(i), K(*job));
}
}
return ret;
}
int ObDistributedJobControl::init_job_finish_queue(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job is NULL", K(ret), K(i));
} else if (OB_FAIL(job->init_finish_queue(ctx))) {
LOG_WARN("fail to sort scan partition locations", K(ret), K(i), K(*job));
}
}
return ret;
}
int ObDistributedJobControl::get_root_job(ObJob*& root_job) const
{
int ret = OB_SUCCESS;
root_job = jobs_.at(0);
if (OB_ISNULL(root_job) || OB_UNLIKELY(!root_job->is_root_job())) {
ret = OB_ERR_UNEXPECTED;
SQL_EXE_LOG(WARN, "root job is NULL or invalid", K(ret), K(jobs_.count()));
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,35 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_DISTRIBUTED_JOB_CONTROL_
#define OCEANBASE_SQL_EXECUTOR_DISTRIBUTED_JOB_CONTROL_
#include "sql/executor/ob_job_control.h"
namespace oceanbase {
namespace sql {
class ObDistributedJobControl : public ObJobControl {
public:
ObDistributedJobControl();
virtual ~ObDistributedJobControl();
virtual int get_ready_jobs(common::ObIArray<ObJob*>& jobs, bool serial_sched = false) const;
virtual int sort_job_scan_part_locs(ObExecContext& ctx) override;
virtual int init_job_finish_queue(ObExecContext& ctx) override;
int get_root_job(ObJob*& root_job) const;
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedJobControl);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_DISTRIBUTED_JOB_CONTROL_ */

View File

@ -0,0 +1,191 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_distributed_job_executor.h"
#include "sql/executor/ob_distributed_scheduler.h"
#include "sql/executor/ob_job.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/queue/ob_lighty_queue.h"
#include "lib/utility/ob_tracepoint.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObDistributedJobExecutor::ObDistributedJobExecutor() : job_(NULL), executor_(NULL)
{}
ObDistributedJobExecutor::~ObDistributedJobExecutor()
{}
int ObDistributedJobExecutor::execute_step(ObExecContext& ctx)
{
NG_TRACE(job_exec_step_begin);
int ret = OB_SUCCESS;
ObArray<ObTaskInfo*> ready_tasks;
ObTaskInfo* task_info = NULL;
ObDistributedSchedulerManager* sched_mgr = ObDistributedSchedulerManager::get_instance();
if (OB_ISNULL(sched_mgr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail get ObDistributedSchedulerManager instance", K(ret));
} else if (OB_ISNULL(job_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job_ is NULL", K(ret));
} else if (OB_ISNULL(executor_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("executor_ is NULL", K(ret));
} else if (OB_I(t1)
OB_UNLIKELY(OB_JOB_STATE_INITED != job_->get_state() && OB_JOB_STATE_RUNNING != job_->get_state())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job state must be inited or running", "job_state", job_->get_state());
} else {
if (OB_JOB_STATE_INITED == job_->get_state()) {
job_->set_state(OB_JOB_STATE_RUNNING);
} else {
// job state is OB_JOB_STATE_RUNNING, do nothing
}
ready_tasks.reset();
if (OB_FAIL(get_executable_tasks(ctx, ready_tasks))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail get executable tasks.", K(ret));
}
} else {
ObTaskCompleteEvent task_event;
for (int64_t i = 0; OB_SUCC(ret) && i < ready_tasks.count(); ++i) {
if (OB_ISNULL(task_info = ready_tasks.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("task_info is NULL", K(ret), K(i), K(*job_));
} else if (OB_FAIL(executor_->execute(ctx, job_, task_info))) {
if (OB_ERR_TASK_SKIPPED == ret) {
// The task is skipped, construct a virtual ObTaskCompleteEvent and record it
// set ret to OB_SUCCESS to continue the loop
int inner_ret = OB_SUCCESS;
task_event.reset();
if (OB_SUCCESS != (inner_ret = task_event.init(task_info->get_task_location(), OB_ERR_TASK_SKIPPED))) {
LOG_WARN("fail to init task event", K(ret), K(inner_ret), K(*job_));
ret = inner_ret;
} else if (OB_SUCCESS != (inner_ret = sched_mgr->signal_scheduler(task_event))) {
LOG_WARN("fail to signal scheduler", K(ret), K(inner_ret), K(task_event), K(*job_));
ret = inner_ret;
} else {
ret = OB_SUCCESS;
}
} else {
LOG_WARN("fail execute task. ret", K(ret));
}
}
} // for
}
}
NG_TRACE(job_exec_step_end);
return ret;
}
int ObDistributedJobExecutor::get_executable_tasks(const ObExecContext& ctx, ObArray<ObTaskInfo*>& ready_tasks)
{
int ret = OB_SUCCESS;
ObTaskControl* task_ctrl = NULL;
if (OB_I(t1) OB_ISNULL(job_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job executor is not init, job is NULL", K(ret));
} else if (OB_FAIL(OB_I(t2) job_->get_task_control(ctx, task_ctrl))) {
LOG_WARN("fail get task control.", K(ret));
} else if (OB_ISNULL(task_ctrl)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succ to get task ctrl, but task_ctrl is NULL", K(ret), K(*job_));
} else if (OB_FAIL(OB_I(t3) task_ctrl->get_ready_tasks(ready_tasks))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail get ready task.", K(ret));
}
} else {
// empty
}
return ret;
}
int ObDistributedJobExecutor::kill_job(ObExecContext& query_ctx)
{
int ret = OB_SUCCESS;
int kill_ret = OB_SUCCESS;
ObArray<ObTaskInfo*> running_tasks;
ObTaskControl* task_ctrl = NULL;
if (OB_ISNULL(job_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job executor is not init, job is NULL", K(ret));
} else if (OB_ISNULL(executor_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("executor_ is NULL", K(ret));
} else if (OB_FAIL(OB_I(t1) job_->get_task_control(query_ctx, task_ctrl))) {
LOG_WARN("fail get task control.", K(ret));
} else if (OB_ISNULL(task_ctrl)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succ to get task ctrl, but task_ctrl is NULL", K(ret), K(*job_));
} else {
if (OB_FAIL(task_ctrl->get_running_tasks(running_tasks))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail to get running task.", K(ret));
} else {
ret = OB_SUCCESS;
}
} else {
}
for (int64_t i = 0; OB_SUCC(ret) && i < running_tasks.count(); ++i) {
if (OB_SUCCESS != (kill_ret = executor_->kill(query_ctx, job_, running_tasks.at(i)))) {
LOG_WARN("fail to kill task", K(kill_ret)); // ignore error
}
}
}
return ret;
}
int ObDistributedJobExecutor::close_all_results(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
ObArray<ObTaskInfo*> tasks;
ObTaskControl* task_ctrl = NULL;
if (OB_ISNULL(job_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job executor is not init, job is NULL", K(ret));
} else if (OB_ISNULL(executor_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("executor_ is NULL", K(ret));
} else if (OB_FAIL(job_->get_task_control(ctx, task_ctrl))) {
LOG_WARN("fail get task control.", K(ret));
} else if (OB_ISNULL(task_ctrl)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succ to get task ctrl, but task_ctrl is NULL", K(ret), K(*job_));
} else if (OB_FAIL(task_ctrl->get_begin_running_tasks(tasks))) {
LOG_WARN("fail get begin running tasks.", K(ret));
} else {
int close_ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < tasks.count(); ++i) {
const ObTaskInfo* task = tasks.at(i);
if (OB_SUCCESS != (close_ret = executor_->close_result(ctx, task))) {
// ignore error
if (OB_ISNULL(task)) {
LOG_WARN("fail to close result, and task is NULL", K(close_ret), K(i));
} else {
LOG_WARN("fail to close result", K(close_ret), K(i), K(*task));
}
} else {
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,65 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_JOB_EXECUTOR_
#define OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_JOB_EXECUTOR_
#include "sql/executor/ob_distributed_task_executor.h"
#include "sql/executor/ob_task_event.h"
#include "lib/container/ob_array.h"
namespace oceanbase {
namespace common {
class ObLightyQueue;
}
namespace sql {
class ObJob;
class ObExecContext;
class ObDistributedJobExecutor {
public:
ObDistributedJobExecutor();
virtual ~ObDistributedJobExecutor();
int execute_step(ObExecContext& ctx);
int kill_job(ObExecContext& ctx);
int close_all_results(ObExecContext& ctx);
inline ObJob* get_job()
{
return job_;
}
inline void set_job(ObJob& job)
{
job_ = &job;
}
inline void set_task_executor(ObDistributedTaskExecutor& executor)
{
executor_ = &executor;
}
inline void reset()
{
job_ = NULL;
executor_ = NULL;
}
private:
int get_executable_tasks(const ObExecContext& ctx, common::ObArray<ObTaskInfo*>& ready_tasks);
ObJob* job_;
ObDistributedTaskExecutor* executor_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedJobExecutor);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_JOB_EXECUTOR_ */

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,288 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_SCHEDULER_
#define OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_SCHEDULER_
#include "lib/queue/ob_lighty_queue.h"
#include "lib/lock/ob_spin_lock.h"
#include "lib/container/ob_id_map.h"
#include "lib/thread/ob_simple_thread_pool.h"
#include "rpc/obrpc/ob_rpc_packet.h"
#include "common/ob_queue_thread.h"
#include "sql/executor/ob_job_parser.h"
#include "sql/executor/ob_distributed_job_control.h"
#include "sql/executor/ob_task_spliter_factory.h"
#include "sql/executor/ob_addrs_provider_factory.h"
#include "sql/executor/ob_sql_scheduler.h"
#include "sql/executor/ob_local_job_control.h"
#include "sql/executor/ob_task_event.h"
#include "sql/engine/ob_des_exec_context.h"
#include "sql/executor/ob_sql_execution_id_map.h"
#include "sql/executor/ob_trans_result_collector.h"
#include "sql/ob_sql_trans_util.h"
namespace oceanbase {
namespace sql {
class ObDistributedJobExecutor;
class ObExecStatCollector;
class ObDistributedSchedulerManager;
class ObDistributedSchedulerCtx;
class ObDistributedExecContext;
class ObDistributedScheduler : public ObSqlScheduler {
public:
friend class ObSignalFinishQueue;
friend class ObDistributedSchedulerManager;
ObDistributedScheduler();
virtual ~ObDistributedScheduler();
virtual void reset();
// called by query thread
virtual int schedule(ObExecContext& ctx, ObPhysicalPlan* phy_plan);
int parse_all_jobs_and_start_root_job(ObExecContext& ctx, ObPhysicalPlan* phy_plan);
int kill_all_jobs(ObExecContext& ctx, ObJobControl& jc);
int close_all_results(ObExecContext& ctx);
int init();
int stop();
inline void set_exec_stat_collector(ObExecStatCollector* collector);
inline ObExecStatCollector* get_exec_stat_collector();
inline int init_trans_result(ObSQLSessionInfo& session, ObExecutorRpcImpl* exec_rpc);
inline uint64_t get_execution_id()
{
return execution_id_;
}
int pop_task_result_for_root(
ObExecContext& ctx, uint64_t root_op_id, ObTaskResult& task_result, int64_t timeout_timestamp);
int pop_task_event_for_sche(const ObExecContext& ctx, ObTaskCompleteEvent*& task_event, int64_t timeout_timestamp);
int signal_root_finish();
int signal_job_iter_end(common::ObLightyQueue& finish_queue);
int signal_schedule_error(int sche_ret);
int signal_schedule_finish();
int signal_can_serial_exec();
int check_root_finish()
{
return root_finish_ ? common::OB_ERR_INTERRUPTED : common::OB_SUCCESS;
}
int check_schedule_error()
{
return sche_ret_;
}
int wait_root_use_up_data(ObExecContext& ctx);
int wait_schedule_finish(/*int64_t timeout_timestamp*/);
int wait_can_serial_exec(ObExecContext& ctx, int64_t timeout_timestamp);
int wait_all_task(int64_t timeout, const bool is_build_index)
{
return trans_result_.wait_all_task(timeout, is_build_index);
}
int get_schedule_ret()
{
return sche_ret_;
}
void set_sche_thread_started(bool sche_thread_started)
{
sche_thread_started_ = sche_thread_started;
}
uint64_t get_scheduler_id() const
{
return scheduler_id_;
}
int atomic_push_err_rpc_addr(const common::ObAddr& addr);
private:
static const int64_t OB_MAX_SKIPPED_TASK_EVENTS_QUEUE_CAPACITY = 1024L * 16L;
typedef int (ObDistributedScheduler::*ObCheckStatus)();
int merge_trans_result(const ObTaskCompleteEvent& task_event);
int set_task_status(const ObTaskID& task_id, ObTaskStatus status);
int signal_finish_queue(const ObTaskCompleteEvent& task_event);
int pop_task_idx(const ObExecContext& ctx, common::ObLightyQueue& finish_queue, int64_t timeout_timestamp,
ObCheckStatus check_func, int64_t& task_idx);
int get_task_event(int64_t task_event_idx, ObTaskCompleteEvent*& task_event);
inline void set_execution_id(uint64_t execution_id)
{
execution_id_ = execution_id;
}
inline void* idx_to_ptr(int64_t idx)
{
return reinterpret_cast<void*>(idx + 1);
}
inline int64_t ptr_to_idx(void* ptr)
{
return reinterpret_cast<int64_t>(ptr) - 1;
}
uint64_t next_scheduler_id();
private:
static const int64_t MAX_FINISH_QUEUE_CAPACITY = 4096;
static const int64_t NOP_EVENT = INT64_MIN + 1;
static const int64_t SCHE_ITER_END = INT64_MIN + 2;
common::ObArenaAllocator allocator_;
uint64_t execution_id_;
common::ObLightyQueue finish_queue_;
common::ObSEArray<ObTaskCompleteEvent*, 64> response_task_events_;
common::ObSpinLock lock_;
ObTaskSpliterFactory spfactory_;
ObDistributedJobControl job_control_;
ObJobParser parser_;
ObExecStatCollector* exec_stat_collector_;
ObTransResultCollector trans_result_;
volatile bool should_stop_;
volatile bool root_finish_;
volatile bool sche_finish_;
volatile int sche_ret_;
common::ObCond sche_finish_cond_;
volatile bool can_serial_exec_;
common::ObCond can_serial_exec_cond_;
//===================main thread vars===================
bool sche_thread_started_;
// check scheduler id to discard message of other scheduler.
// (index building may retry with same execution_id, may receive message of previous scheduler)
uint64_t scheduler_id_;
common::ObArray<ObAddr> rpc_error_addrs_;
DISALLOW_COPY_AND_ASSIGN(ObDistributedScheduler);
}; /* end ObDistributedScheduler */
class ObSchedulerThreadPool : public lib::TGTaskHandler {
virtual void handle(void* task);
};
class ObDistributedSchedulerManager {
public:
// private static variable
static const int64_t DEFAULT_ID_MAP_SIZE = (1 << 20);
static const int64_t MINI_MODE_ID_MAP_SIZE = (128 << 10);
static const int64_t SCHEDULER_THREAD_NUM = 128;
static const int64_t MINI_MODE_SCHEDULER_THREAD_NUM = 4;
static const int64_t SCHEDULER_THREAD_QUEUE = 256;
private:
// private static variable
static ObDistributedSchedulerManager* instance_;
typedef ObSqlExecutionIDMap ExecutionIDMap;
public:
class ObDistributedSchedulerHolder {
public:
ObDistributedSchedulerHolder();
virtual ~ObDistributedSchedulerHolder();
void reset();
int init(ObDistributedScheduler* scheduler, uint64_t execution_id, ExecutionIDMap& execution_id_map);
int get_scheduler(ObDistributedScheduler*& scheduler);
private:
bool inited_;
uint64_t execution_id_;
ObDistributedScheduler* scheduler_;
ExecutionIDMap* execution_id_map_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedSchedulerHolder);
};
class ObDistributedSchedulerKiller {
public:
ObDistributedSchedulerKiller()
{}
virtual ~ObDistributedSchedulerKiller()
{}
void reset()
{}
void operator()(const uint64_t execution_id);
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedSchedulerKiller);
};
public:
friend class ObDistributedSchedulerKiller;
static int build_instance();
static ObDistributedSchedulerManager* get_instance();
ObDistributedSchedulerManager();
virtual ~ObDistributedSchedulerManager();
void reset();
int alloc_scheduler(ObExecContext& ctx, uint64_t& execution_id);
int free_scheduler(uint64_t execution_id);
int close_scheduler(ObExecContext& ctx, uint64_t execution_id);
int get_scheduler(uint64_t execution_id, ObDistributedSchedulerHolder& scheduler_holder);
int parse_jobs_and_start_sche_thread(
uint64_t execution_id, ObExecContext& ctx, ObPhysicalPlan* phy_plan, int64_t timeout_timestamp);
int do_schedule(ObDistributedSchedulerCtx& sched_ctx, ObDistributedExecContext& dis_exec_ctx);
int signal_scheduler(ObTaskCompleteEvent& task_event, const uint64_t scheduler_id = 0);
int signal_schedule_error(uint64_t execution_id, int sche_ret, const ObAddr addr, const uint64_t scheduler_id = 0);
int collect_extent_info(ObTaskCompleteEvent& task_event);
int merge_trans_result(const ObTaskCompleteEvent& task_event);
int set_task_status(const ObTaskID& task_id, ObTaskStatus status);
int stop();
private:
// private function
int init();
private:
// private common variable
bool inited_;
ExecutionIDMap execution_id_map_;
volatile bool is_stopping_;
ObDistributedSchedulerKiller distributed_scheduler_killer_;
ObSchedulerThreadPool scheduler_pool_;
DISALLOW_COPY_AND_ASSIGN(ObDistributedSchedulerManager);
};
inline void ObDistributedScheduler::set_exec_stat_collector(ObExecStatCollector* collector)
{
exec_stat_collector_ = collector;
}
inline ObExecStatCollector* ObDistributedScheduler::get_exec_stat_collector()
{
return exec_stat_collector_;
}
inline int ObDistributedScheduler::init_trans_result(ObSQLSessionInfo& session, ObExecutorRpcImpl* exec_rpc)
{
ObDistributedSchedulerManager* dist_task_mgr = ObDistributedSchedulerManager::get_instance();
return trans_result_.init(session, exec_rpc, dist_task_mgr, NULL /*mini_task_mgr*/);
}
class ObDistributedSchedulerCtx {
public:
const uint64_t* trace_id_;
uint64_t execution_id_;
ObExecContext* exec_ctx_;
char* exec_ctx_buf_;
int64_t buf_len_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedSchedulerCtx);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_SCHEDULER_ */
//// end of header file

View File

@ -0,0 +1,377 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/session/ob_sql_session_info.h"
#include "sql/executor/ob_distributed_task_executor.h"
#include "sql/executor/ob_distributed_transmit.h"
#include "sql/executor/ob_receive.h"
#include "sql/executor/ob_job.h"
#include "sql/executor/ob_task_executor_ctx.h"
#include "sql/executor/ob_executor_rpc_impl.h"
#include "sql/executor/ob_trans_result_collector.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/ob_des_exec_context.h"
#include "lib/utility/ob_tracepoint.h"
#include "sql/executor/ob_bkgd_dist_task.h"
#include "sql/executor/ob_executor_rpc_processor.h"
#include "rootserver/ob_root_service.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObDistributedTaskExecutor::ObDistributedTaskExecutor(const uint64_t scheduler_id)
: scheduler_id_(scheduler_id), trans_result_(NULL)
{}
ObDistributedTaskExecutor::~ObDistributedTaskExecutor()
{}
int ObDistributedTaskExecutor::execute(ObExecContext& query_ctx, ObJob* job, ObTaskInfo* task_info)
{
int ret = OB_SUCCESS;
ObPhyOperator* root_op = NULL;
ObDistributedTransmitInput* trans_input = NULL;
ObIPhyOperatorInput* op_input = NULL;
ObExecutorRpcImpl* rpc = NULL;
ObExecContext* exec_ctx_snap = NULL;
ObSQLSessionInfo* session_snap = NULL;
ObPhysicalPlanCtx* plan_ctx_snap = NULL;
ObTask task;
if (OB_ISNULL(exec_ctx_snap = query_ctx.get_scheduler_thread_ctx().get_dis_exec_ctx_for_update())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("exec ctx snap is NULL", K(ret));
} else if (OB_ISNULL(session_snap = exec_ctx_snap->get_my_session()) ||
OB_ISNULL(plan_ctx_snap = exec_ctx_snap->get_physical_plan_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("session snap or plan ctx snap is NULL", K(ret), K(session_snap), K(plan_ctx_snap));
} else if (OB_I(t1)(OB_ISNULL(job) || OB_ISNULL(task_info))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job or taskinfo is not set", K(ret), K(job), K(task_info));
} else if (OB_I(t2)(OB_ISNULL(root_op = job->get_root_op()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail execute task. no root op found", K(ret), K(root_op));
} else if (OB_I(t3)(OB_UNLIKELY(!IS_DIST_TRANSMIT(root_op->get_type())))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("valid check fail. root op type must be ObTransmit", K(ret), K(root_op->get_type()));
} else if (OB_I(t4)(OB_ISNULL(op_input = exec_ctx_snap->get_phy_op_input(root_op->get_id())))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get op input", K(ret), "op_id", root_op->get_id());
} else if (OB_UNLIKELY(!IS_DIST_TRANSMIT(op_input->get_phy_op_type()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Optimizer should generate PHY_DISTRIBUTED_TRANSMIT for this plan",
K(ret),
"input_type",
op_input->get_phy_op_type());
} else if (OB_ISNULL(trans_input = static_cast<ObDistributedTransmitInput*>(op_input))) {
ret = OB_ERR_UNEXPECTED; // should never reach here
LOG_WARN("fail cast op", K(ret), "trans_input", trans_input);
} else if (OB_FAIL(OB_I(t6) ObTaskExecutorCtxUtil::get_task_executor_rpc(query_ctx, rpc))) {
LOG_WARN("fail get rpc", K(ret));
} else if (OB_ISNULL(rpc)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("rpc is NULL", K(ret));
} else if (OB_FAIL(build_task(*exec_ctx_snap, *job, *task_info, task))) {
LOG_WARN("fail to build task", K(ret), K(*job), K(*task_info));
} else {
ObExecutorRpcCtx rpc_ctx(session_snap->get_rpc_tenant_id(),
plan_ctx_snap->get_timeout_timestamp(),
exec_ctx_snap->get_task_exec_ctx().get_min_cluster_version(),
&query_ctx.get_scheduler_thread_ctx().get_scheduler_retry_info_for_update(),
query_ctx.get_my_session(),
query_ctx.get_scheduler_thread_ctx().is_plain_select_stmt());
task_info->set_task_send_begin(ObTimeUtility::current_time());
task_info->set_state(OB_TASK_STATE_RUNNING);
trans_input->set_ob_task_id(task_info->get_task_location().get_ob_task_id());
if (OB_FAIL(task_dispatch(*exec_ctx_snap, *rpc, rpc_ctx, task, *task_info))) {
bool skip_failed_tasks = false;
int check_ret = OB_SUCCESS;
if (OB_SUCCESS != (check_ret = should_skip_failed_tasks(*task_info, skip_failed_tasks))) {
// check fail, set ret to check_ret
LOG_WARN("fail to check if should skip failed tasks", K(ret), K(check_ret), K(*job), K(rpc_ctx));
ret = check_ret;
} else if (true == skip_failed_tasks) {
// should skip failed tasks, log user warning and skip it, than return OB_ERR_TASK_SKIPPED
LOG_WARN("fail to do task on some server, log user warning and skip it",
K(ret),
K(task_info->get_task_location().get_server()),
K(*job),
K(rpc_ctx));
LOG_USER_WARN(OB_ERR_TASK_SKIPPED,
to_cstring(task_info->get_task_location().get_server()),
common::ob_errpkt_errno(ret, lib::is_oracle_mode()));
ret = OB_ERR_TASK_SKIPPED;
} else {
// let user see this ret
LOG_WARN("fail to submit task", K(ret), K(*task_info), K(rpc_ctx));
}
} else {
}
NG_TRACE_EXT(distributed_task_submited,
OB_ID(ret),
ret,
OB_ID(runner_svr),
task_info->get_task_location().get_server(),
OB_ID(task),
task);
}
return ret;
}
int ObDistributedTaskExecutor::kill(ObExecContext& query_ctx, ObJob* job, ObTaskInfo* task_info)
{
int ret = OB_SUCCESS;
ObPhyOperator* root_op = NULL;
ObExecutorRpcImpl* rpc = NULL;
const ObExecContext* exec_ctx_snap = NULL;
ObSQLSessionInfo* session_snap = NULL;
ObPhysicalPlanCtx* plan_ctx_snap = NULL;
if (OB_ISNULL(exec_ctx_snap = query_ctx.get_scheduler_thread_ctx().get_dis_exec_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("exec ctx snap is NULL", K(ret));
} else if (OB_ISNULL(session_snap = exec_ctx_snap->get_my_session()) ||
OB_ISNULL(plan_ctx_snap = exec_ctx_snap->get_physical_plan_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("session snap or plan ctx snap is NULL", K(ret), K(session_snap), K(plan_ctx_snap));
} else if (OB_I(t1)(OB_ISNULL(task_info) || OB_ISNULL(job))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("param is NULL", K(ret), K(task_info), K(job));
} else if (OB_I(t2)(OB_ISNULL(root_op = job->get_root_op()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail execute task. no root op found", K(ret), K(*job));
} else if (OB_I(t3) OB_UNLIKELY(!IS_TRANSMIT(root_op->get_type()))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("valid check fail. root op type must be ObTransmit", K(ret), K(root_op->get_type()));
} else if (OB_FAIL(OB_I(t5) ObTaskExecutorCtxUtil::get_task_executor_rpc(query_ctx, rpc))) {
LOG_WARN("fail get rpc", K(ret));
} else if (OB_ISNULL(rpc)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("rpc is NULL", K(ret));
} else {
ObExecutorRpcCtx rpc_ctx(session_snap->get_rpc_tenant_id(),
plan_ctx_snap->get_timeout_timestamp(),
exec_ctx_snap->get_task_exec_ctx().get_min_cluster_version(),
&query_ctx.get_scheduler_thread_ctx().get_scheduler_retry_info_for_update(),
query_ctx.get_my_session(),
query_ctx.get_scheduler_thread_ctx().is_plain_select_stmt());
if (OB_FAIL(rpc->task_kill(
rpc_ctx, task_info->get_task_location().get_ob_task_id(), task_info->get_task_location().get_server()))) {
LOG_WARN("fail to kill task", K(ret), K(*task_info), K(rpc_ctx));
} else {
}
}
return ret;
}
int ObDistributedTaskExecutor::close_result(ObExecContext& ctx, const ObTaskInfo* task_info)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("task info is NULL", K(ret));
} else {
const ObIArray<ObSliceEvent>& slice_events = task_info->get_slice_events();
if (OB_UNLIKELY(slice_events.count() < 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("slice events count < 0", K(ret), K(slice_events.count()));
} else {
// const ObSliceEvent &slice_event = slice_events.at(0);
// if (OB_ISNULL(slice_event)) {
// ret = OB_ERR_UNEXPECTED;
// LOG_ERROR("slice event is NULL", K(ret), K(slice_events.count()));
// } else {
// The result has been pulled back to the local situation,
// and the remote end has released the result,
// so there is no need to send the rpc to release the result;
// if the results are not pulled back to the local situation,
// the remote end still saves the results, so rpc must be sent to release the remote results
// const ObTaskSmallResult sr = slice_event->get_small_result();
// if (!sr.has_data()) {
if (OB_FAIL(send_close_result_rpc(ctx, task_info))) {
LOG_WARN("fail to send close result rpc", K(ret), K(*task_info));
}
//}
// }
}
}
return ret;
}
int ObDistributedTaskExecutor::send_close_result_rpc(ObExecContext& ctx, const ObTaskInfo* task_info)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("task info is NULL", K(ret));
} else {
const ObExecContext* exec_ctx_snap = NULL;
ObSQLSessionInfo* session_snap = NULL;
ObPhysicalPlanCtx* plan_ctx_snap = NULL;
ObExecutorRpcImpl* rpc = NULL;
const ObTaskLocation& task_loc = task_info->get_task_location();
ObSliceID ob_slice_id;
int bak_ret = OB_SUCCESS;
// When schedule error or early terminate (statement with limit clause), result events
// are not processed (still in %response_task_events_ queue), there is no slice event in task.
// In this case, we remove intermediate result with slice_id 0.
const ObIArray<ObSliceEvent>& slices = task_info->get_slice_events();
for (int64_t i = 0; i < std::max(slices.count(), 1L); i++) {
ret = OB_SUCCESS;
ob_slice_id.set_ob_task_id(task_loc.get_ob_task_id());
ob_slice_id.set_slice_id(slices.empty() ? 0 : slices.at(i).get_ob_slice_id().get_slice_id());
if (OB_ISNULL(exec_ctx_snap = ctx.get_scheduler_thread_ctx().get_dis_exec_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("exec ctx snap is NULL", K(ret));
} else if (OB_ISNULL(session_snap = exec_ctx_snap->get_my_session()) ||
OB_ISNULL(plan_ctx_snap = exec_ctx_snap->get_physical_plan_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("session snap or plan ctx snap is NULL", K(ret), K(session_snap), K(plan_ctx_snap));
} else if (OB_FAIL(ObTaskExecutorCtxUtil::get_task_executor_rpc(ctx, rpc))) {
LOG_ERROR("fail get rpc", K(ret));
} else if (OB_ISNULL(rpc)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("rpc is NULL", K(ret));
} else {
ObExecutorRpcCtx rpc_ctx(session_snap->get_rpc_tenant_id(),
plan_ctx_snap->get_timeout_timestamp(),
exec_ctx_snap->get_task_exec_ctx().get_min_cluster_version(),
&ctx.get_scheduler_thread_ctx().get_scheduler_retry_info_for_update(),
ctx.get_my_session(),
ctx.get_scheduler_thread_ctx().is_plain_select_stmt());
if (OB_FAIL(rpc->close_result(rpc_ctx, ob_slice_id, task_loc.get_server()))) {
LOG_WARN("fail to rpc call close_result", K(ret), K(ob_slice_id), K(task_loc), K(rpc_ctx));
}
}
if (OB_FAIL(ret)) {
bak_ret = ret;
}
}
ret = bak_ret;
}
return ret;
}
int ObDistributedTaskExecutor::build_task(ObExecContext& query_ctx, ObJob& job, ObTaskInfo& task_info, ObTask& task)
{
int ret = OB_SUCCESS;
ObPhyOperator* root_op = NULL;
const ObPhysicalPlan* phy_plan = NULL;
if (OB_ISNULL(root_op = job.get_root_op())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op is NULL", K(ret), K(job), K(task_info));
} else if (OB_ISNULL(phy_plan = root_op->get_phy_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("phy plan is NULL", K(ret), K(job), K(task_info));
} else if (OB_FAIL(OB_I(t1) build_task_op_input(query_ctx, task_info, *root_op))) {
LOG_WARN("fail to build op input", K(ret), K(task_info));
} else if (OB_ISNULL(query_ctx.get_physical_plan_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("physical plan ctx is null", K(ret));
} else {
query_ctx.get_physical_plan_ctx()->set_phy_plan(phy_plan);
const ObTaskInfo::ObRangeLocation& range_loc = task_info.get_range_location();
for (int64_t i = 0; OB_SUCC(ret) && i < range_loc.part_locs_.count(); ++i) {
const ObTaskInfo::ObPartLoc& part_loc = range_loc.part_locs_.at(i);
if (OB_FAIL(task.add_partition_key(part_loc.partition_key_))) {
LOG_WARN("fail to add partition key into ObTask", K(ret), K(i), K(part_loc.partition_key_));
} else if (OB_FAIL(task.assign_ranges(part_loc.scan_ranges_))) {
LOG_WARN("assign range failed", K(ret));
} else {
// Add the partition key of the right table of mv
// so that it can be passed to the start participant interface at the remote end
for (int64_t j = 0; OB_SUCC(ret) && j < part_loc.depend_table_keys_.count(); ++j) {
if (OB_FAIL(task.add_partition_key(part_loc.depend_table_keys_.at(j)))) {
LOG_WARN("fail to add partition key into ObTask", K(ret), K(j), K(part_loc.depend_table_keys_.at(j)));
}
}
}
}
if (OB_SUCC(ret)) {
LOG_DEBUG("build task",
"ctrl_svr",
job.get_ob_job_id().get_server(),
"ob_task_id",
task_info.get_task_location().get_ob_task_id(),
"runner_svr",
task_info.get_task_location().get_server(),
"range_loc",
task_info.get_range_location());
task.set_ctrl_server(job.get_ob_job_id().get_server());
task.set_ob_task_id(task_info.get_task_location().get_ob_task_id());
task.set_location_idx(task_info.get_location_idx());
task.set_runner_server(task_info.get_task_location().get_server());
task.set_serialize_param(query_ctx, *root_op, *phy_plan);
}
}
return ret;
}
int ObDistributedTaskExecutor::task_dispatch(
ObExecContext& exec_ctx, ObExecutorRpcImpl& rpc, ObExecutorRpcCtx& rpc_ctx, ObTask& task, ObTaskInfo& task_info)
{
int ret = OB_SUCCESS;
if (!task_info.is_background()) {
if (OB_ISNULL(trans_result_)) {
ret = OB_NOT_INIT;
LOG_WARN("trans result is NULL", K(ret));
} else if (OB_FAIL(trans_result_->send_task(task_info))) {
LOG_WARN("send task failed", K(ret), K(task_info));
} else if (OB_FAIL(rpc.task_submit(
rpc_ctx, task, task_info.get_task_location().get_server(), trans_result_->get_trans_result()))) {
LOG_WARN("task submit failed", K(ret));
}
} else {
const int64_t size = task.get_serialize_size();
int64_t pos = 0;
char* buf = NULL;
if (OB_ISNULL(exec_ctx.get_my_session())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is NULL", K(ret));
} else if (OB_ISNULL(buf = static_cast<char*>(exec_ctx.get_allocator().alloc(size)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else if (OB_FAIL(task.serialize(buf, size, pos))) {
LOG_WARN("task serialize failed", K(ret), K(task));
} else {
ObString serialized_task(static_cast<int32_t>(size), buf);
ObSchedBKGDDistTask sched_task;
if (OB_FAIL(sched_task.init(exec_ctx.get_my_session()->get_effective_tenant_id(),
rpc_ctx.get_timeout_timestamp(),
task.get_ob_task_id(),
scheduler_id_,
task_info.get_range_location().part_locs_.at(0).partition_key_,
task_info.get_task_location().get_server(),
serialized_task))) {
LOG_WARN("init task failed", K(ret));
} else if (OB_ISNULL(GCTX.root_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root service is NULL", K(ret));
} else if (OB_FAIL(GCTX.root_service_->schedule_sql_bkgd_task(sched_task))) {
LOG_WARN("schedule background task failed", K(ret), K(sched_task));
} else {
LOG_INFO("start schedule background task", K(sched_task));
}
}
if (NULL != buf) {
exec_ctx.get_allocator().free(buf);
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,57 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TASK_EXECUTOR_
#define OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TASK_EXECUTOR_
#include "sql/executor/ob_task_executor.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObJob;
class ObTask;
class ObExecutorRpcImpl;
class ObExecutorRpcCtx;
class ObTransResultCollector;
class ObDistributedTaskExecutor : public ObTaskExecutor {
public:
explicit ObDistributedTaskExecutor(const uint64_t scheduler_id);
virtual ~ObDistributedTaskExecutor();
virtual int execute(ObExecContext& query_ctx, ObJob* job, ObTaskInfo* task_info);
virtual int kill(ObExecContext& ctx, ObJob* job, ObTaskInfo* task_info);
virtual void reset()
{
ObTaskExecutor::reset();
}
int close_result(ObExecContext& ctx, const ObTaskInfo* task_info);
void set_trans_result(ObTransResultCollector* trans_result)
{
trans_result_ = trans_result;
}
private:
int send_close_result_rpc(ObExecContext& ctx, const ObTaskInfo* task_info);
int build_task(ObExecContext& query_ctx, ObJob& job, ObTaskInfo& task_info, ObTask& task);
int task_dispatch(
ObExecContext& exec_ctx, ObExecutorRpcImpl& rpc, ObExecutorRpcCtx& rpc_ctx, ObTask& task, ObTaskInfo& task_info);
private:
uint64_t scheduler_id_;
ObTransResultCollector* trans_result_;
DISALLOW_COPY_AND_ASSIGN(ObDistributedTaskExecutor);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TASK_EXECUTOR_ */

View File

@ -0,0 +1,72 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_distributed_task_runner.h"
#include "sql/executor/ob_distributed_transmit.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/utility/ob_tracepoint.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObDistributedTaskRunner::ObDistributedTaskRunner()
{}
ObDistributedTaskRunner::~ObDistributedTaskRunner()
{}
/*
* Implementation process:
* 1. The Server receives the OB_DISTRIBUTE_TASK message and immediately responds to
the Scheduler receiving the Task message
* 2. Construct three objects of ObTask, ObExecContext and ObPhysicalPlan on the stack
* 3. Initialize ObTask: ObTask.init(ObExecContext &ctx, ObPhysicalPlan &plan)
* 4. Deserialize ObTask from Packet
* 5. (*) Call ObDistributedTaskRunner::execute() to execute the Task
* 6. Report execution results to Scheduler
*/
int ObDistributedTaskRunner::execute(ObExecContext& ctx, ObPhysicalPlan& phy_plan, ObIArray<ObSliceEvent>& slice_events)
{
int ret = OB_SUCCESS;
int close_ret = OB_SUCCESS;
ObPhyOperator* root_op = NULL;
ObDistributedTransmitInput* trans_input = NULL;
if (OB_I(t1)(OB_ISNULL(root_op = phy_plan.get_main_query()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail execute task. no root op", K(root_op), K(ret));
} else if (OB_I(t2) OB_UNLIKELY(!IS_DIST_TRANSMIT(root_op->get_type()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("valid check fail. root op type must be ObDistributedTransmit", K(ret), K(*root_op));
} else if (OB_ISNULL(trans_input = GET_PHY_OP_INPUT(ObDistributedTransmitInput, ctx, root_op->get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fail to get op ctx", K(ret), "op_id", root_op->get_id());
} else if (FALSE_IT(trans_input->set_slice_events(&slice_events))) {
} else {
// root_op no needs to call get_next_row()
// ObTransmit will help with open/get_next_row/send_result/close
if (OB_FAIL(OB_I(t3) root_op->open(ctx))) {
LOG_WARN("fail open root op.", K(ret));
}
if (OB_SUCCESS != (close_ret = root_op->close(ctx))) {
LOG_WARN("fail close root op.", K(ret), K(close_ret));
}
ret = (OB_SUCCESS == ret) ? close_ret : ret;
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,33 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TASK_RUNNER_
#define OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TASK_RUNNER_
#include "share/ob_define.h"
#include "lib/container/ob_iarray.h"
#include "sql/executor/ob_task_event.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObPhysicalPlan;
class ObDistributedTaskRunner {
public:
ObDistributedTaskRunner();
virtual ~ObDistributedTaskRunner();
int execute(ObExecContext& ctx, ObPhysicalPlan& phy_plan, common::ObIArray<ObSliceEvent>& slice_events);
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedTaskRunner);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TASK_RUNNER_ */

View File

@ -0,0 +1,592 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "share/ob_cluster_version.h"
#include "observer/ob_server.h"
#include "sql/executor/ob_distributed_transmit.h"
#include "sql/executor/ob_interm_result_manager.h"
#include "sql/executor/ob_task_info.h"
#include "sql/executor/ob_slice_calc.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/executor/ob_range_hash_key_getter.h"
using namespace oceanbase::common;
using namespace oceanbase::share::schema;
using namespace oceanbase::observer;
namespace oceanbase {
namespace sql {
int ObDistributedTransmitInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
{
UNUSED(op);
int ret = OB_SUCCESS;
ObPhysicalPlanCtx* plan_ctx = NULL;
if (OB_ISNULL(plan_ctx = ctx.get_physical_plan_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("physical plan ctx is NULL", K(ret), K(ctx));
} else {
expire_time_ = plan_ctx->get_timeout_timestamp();
// meta data
ob_task_id_ = task_info.get_task_location().get_ob_task_id();
force_save_interm_result_ = task_info.is_force_save_interm_result();
}
return ret;
}
OB_SERIALIZE_MEMBER(
(ObDistributedTransmitInput, ObTransmitInput), expire_time_, ob_task_id_, force_save_interm_result_);
ObDistributedTransmit::ObDistributedTransmit(ObIAllocator& alloc) : ObTransmit(alloc), shuffle_func_(NULL)
{}
ObDistributedTransmit::~ObDistributedTransmit()
{}
int ObDistributedTransmit::get_part_shuffle_key(
const ObTableSchema* table_schema, int64_t part_idx, ObShuffleKey& part_shuffle_key) const
{
int ret = OB_SUCCESS;
if (NULL != table_schema && part_idx >= 0) {
if (OB_FAIL(part_shuffle_key.set_shuffle_type(*table_schema))) {
LOG_WARN("fail to set part shuffle type");
} else if (OB_SUCC(ret) && OB_FAIL(table_schema->get_part_shuffle_key(
part_idx, part_shuffle_key.get_value0(), part_shuffle_key.get_value1()))) {
LOG_WARN("fail to get part shuffle key", K(ret));
}
} else {
part_shuffle_key.set_shuffle_type(ST_NONE);
}
return ret;
}
int ObDistributedTransmit::get_subpart_shuffle_key(
const ObTableSchema* table_schema, int64_t part_idx, int64_t subpart_idx, ObShuffleKey& subpart_shuffle_key) const
{
int ret = OB_SUCCESS;
if (NULL != table_schema && subpart_idx >= 0) {
if (OB_FAIL(subpart_shuffle_key.set_sub_shuffle_type(*table_schema))) {
LOG_WARN("fail to sub part set shuffle key");
} else if (OB_FAIL(table_schema->get_subpart_shuffle_key(
part_idx, subpart_idx, subpart_shuffle_key.get_value0(), subpart_shuffle_key.get_value1()))) {
LOG_WARN("fail to get subpart shuffle key", K(ret));
}
} else {
subpart_shuffle_key.set_shuffle_type(ST_NONE);
}
return ret;
}
int ObDistributedTransmit::get_shuffle_part_key(
const ObTableSchema* table_schema, int64_t part_idx, int64_t subpart_idx, ObPartitionKey& shuffle_part_key) const
{
int ret = OB_SUCCESS;
shuffle_part_key.reset();
if (NULL != table_schema) {
uint64_t table_id = table_schema->get_table_id();
int64_t part_id = (subpart_idx < 0) ? part_idx : generate_phy_part_id(part_idx, subpart_idx);
int64_t part_num = table_schema->get_partition_cnt();
if (OB_FAIL(shuffle_part_key.init(table_id, part_id, part_num))) {
LOG_WARN("fail to init shuffle part key", K(ret));
}
}
return ret;
}
int ObDistributedTransmit::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_UNLIKELY(calc_exprs_.get_size() > 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("calc exprs should be empty", K(ret), K(calc_exprs_.get_size()));
} else if (OB_UNLIKELY(filter_exprs_.get_size() > 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("filter exprs should be empty", K(ret), K(filter_exprs_.get_size()));
} else if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObDistributedTransmitCtx, ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op ctx is NULL", K(ret));
} else if (OB_FAIL(init_cur_row(*op_ctx, false))) {
LOG_WARN("fail to int cur row", K(ret));
}
return ret;
}
bool ObDistributedTransmit::skip_empty_slice() const
{
return GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_1432;
}
int ObDistributedTransmit::prepare_interm_result(
ObIntermResultManager& interm_result_mgr, ObIntermResult*& interm_result) const
{
int ret = OB_SUCCESS;
if (OB_ISNULL(interm_result)) {
if (OB_FAIL(interm_result_mgr.alloc_result(interm_result))) {
LOG_WARN("fail alloc result", K(ret));
}
}
return ret;
}
int ObDistributedTransmit::inner_open(ObExecContext& exec_ctx) const
{
int ret = OB_SUCCESS;
const ObNewRow* row = NULL;
// int64_t last_fail_res_idx = 0;
ObSEArray<ObIntermResultInfo, 1> added_ir_info_list;
ObSEArray<ObSliceInfo, 8> slice_infos;
ObSQLSessionInfo* session = NULL;
ObDistributedTransmitInput* trans_input = NULL;
ObIntermResultManager* interm_result_mgr = ObIntermResultManager::get_instance();
ObIntermResult** interm_result = NULL;
ObPhysicalPlanCtx* plan_ctx = NULL;
ObDistributedTransmitCtx* transimt_ctx = NULL;
ObIArray<ObSliceEvent>* slice_events = NULL;
uint64_t slice_table_id = repartition_table_id_;
ObSchemaGetterGuard schema_guard;
const ObTableSchema* table_schema = NULL;
int64_t interm_result_buf_len = get_split_task_count() * sizeof(ObIntermResult*);
if (OB_ISNULL(child_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child op is NULL", K(ret));
} else if (OB_UNLIKELY(get_split_task_count() <= 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("split task count must > 0", K(ret), K(get_split_task_count()));
} else if (OB_FAIL(ObTransmit::inner_open(exec_ctx))) {
LOG_WARN("initialize operator context failed", K(ret));
} else if (OB_ISNULL(session = GET_MY_SESSION(exec_ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("session is NULL", K(ret));
} else if (OB_ISNULL(trans_input = GET_PHY_OP_INPUT(ObDistributedTransmitInput, exec_ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fail to get op ctx", K(ret), "op_id", get_id(), "op_type", get_type());
} else if (OB_ISNULL(transimt_ctx = GET_PHY_OPERATOR_CTX(ObDistributedTransmitCtx, exec_ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("get physical operator context failed", K(ret), K_(id));
} else if (OB_ISNULL(slice_events = trans_input->get_slice_events_for_update())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("slice events is NULL", K(ret), K_(id));
} else if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(exec_ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fail to get plan ctx", K(ret));
} else if (OB_ISNULL(interm_result_mgr) || OB_UNLIKELY(interm_result_buf_len <= 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("mgr is NULL or nbytes <= 0", K(ret), K(interm_result_mgr), K(interm_result_buf_len));
} else if (OB_ISNULL(interm_result =
static_cast<ObIntermResult**>(exec_ctx.get_allocator().alloc(interm_result_buf_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc intermediate result buffer", K(ret), K(interm_result_buf_len));
} else {
memset(static_cast<void*>(interm_result), 0, interm_result_buf_len);
// some meta info will be saved in interm_result[0],
// we create it no matter whether empty.
if (skip_empty_slice()) {
if (OB_FAIL(interm_result_mgr->alloc_result(interm_result[0]))) {
LOG_WARN("fail alloc result 0", K(ret));
}
} else {
for (int idx = 0; OB_SUCC(ret) && idx < get_split_task_count(); ++idx) {
if (OB_FAIL(interm_result_mgr->alloc_result(interm_result[idx]))) {
LOG_WARN("fail alloc result", K(ret), K(idx));
}
}
}
}
if (OB_SUCC(ret) && OB_INVALID_ID != slice_table_id) {
if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(session->get_effective_tenant_id(), schema_guard))) {
LOG_WARN("faile to get schema guard", K(ret));
} else if (OB_FAIL(schema_guard.get_table_schema(slice_table_id, table_schema))) {
LOG_WARN("faile to get table schema", K(ret), K(slice_table_id));
} else if (OB_ISNULL(table_schema)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table schema is null", K(ret), K(slice_table_id));
}
}
if (OB_SUCC(ret)) {
int64_t round_robin_idx = 0;
ObRangeHashKeyGetter range_hash_key_getter(repartition_table_id_, repart_columns_, repart_sub_columns_);
while (OB_SUCC(ret)) {
int64_t slice_idx = -1;
int64_t part_idx = -1;
int64_t subpart_idx = -1;
bool skip_row = false;
bool skip_get_partition_ids = false;
if (OB_ISNULL(table_schema)) {
skip_get_partition_ids = true;
slice_idx = 0;
}
if (OB_FAIL(get_next_row(exec_ctx, row))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("fail to get next row from child op", K(ret), K(child_op_->get_type()));
} else {
// iter end
// set found rows
interm_result[0]->set_found_rows(plan_ctx->get_found_rows());
interm_result[0]->set_affected_rows(plan_ctx->get_affected_rows());
interm_result[0]->set_matched_rows(plan_ctx->get_row_matched_count());
interm_result[0]->set_duplicated_rows(plan_ctx->get_row_duplicated_count());
interm_result[0]->set_last_insert_id_session(plan_ctx->calc_last_insert_id_session());
if (!plan_ctx->is_result_accurate()) {
interm_result[0]->set_is_result_accurate(plan_ctx->is_result_accurate());
}
NG_TRACE_EXT(transmit,
OB_ID(found_rows),
plan_ctx->get_found_rows(),
OB_ID(last_insert_id),
plan_ctx->calc_last_insert_id_session());
}
} else if (OB_ISNULL(row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("row is NULL", K(ret));
} else if (OB_FAIL(copy_cur_row(*transimt_ctx, row))) {
LOG_WARN("copy current row failed", K(ret));
} else if (!skip_get_partition_ids && OB_FAIL(get_slice_idx(exec_ctx,
table_schema,
row,
repart_func_,
repart_sub_func_,
repart_columns_,
repart_sub_columns_,
get_split_task_count(),
slice_idx,
skip_row))) {
LOG_WARN("fail get slice idx", K(ret), K(part_idx), K(subpart_idx));
} else if (skip_row && ObPQDistributeMethod::DROP == unmatch_row_dist_method_) {
// do nothing
} else {
if (OB_UNLIKELY(0 >= get_split_task_count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected task count", K(ret));
} else if (skip_row && ObPQDistributeMethod::RANDOM == unmatch_row_dist_method_) {
round_robin_idx++;
slice_idx = round_robin_idx % get_split_task_count();
}
if (OB_FAIL(ret)) {
/*do nothing*/
} else if (OB_UNLIKELY(slice_idx < 0 || slice_idx >= get_split_task_count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid slice idx", K(ret), K(slice_idx), K(split_task_count_));
} else if (OB_FAIL(prepare_interm_result(*interm_result_mgr, interm_result[slice_idx]))) {
LOG_WARN("fail prepare interm_result[slice_idx]", K(ret), K(slice_idx));
} else if (OB_ISNULL(interm_result[slice_idx])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("interm_result[slice_idx] is NULL", K(ret), K(slice_idx));
} else if (OB_FAIL(interm_result[slice_idx]->add_row(session->get_effective_tenant_id(), *row))) {
if (OB_UNLIKELY(OB_ITER_END == ret)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fail emit row to interm result, but ret is OB_ITER_END", K(ret));
} else {
LOG_WARN("fail emit row to interm result", K(ret));
}
} else {
// empty
}
}
}
if (OB_ITER_END == ret) {
LOG_DEBUG("all rows are fetched");
ret = OB_SUCCESS;
}
if (OB_SUCC(ret)) {
for (int64_t slice_idx = 0; OB_SUCC(ret) && slice_idx < get_split_task_count(); ++slice_idx) {
if (NULL == interm_result[slice_idx]) {
continue;
}
if (OB_FAIL(interm_result[slice_idx]->complete_add_rows(session->get_effective_tenant_id()))) {
LOG_WARN("fail to complete add rows", K(ret), K(slice_idx), K(session->get_effective_tenant_id()));
}
}
const static int64_t TOTAL_SMALL_RESULT_MEM_LIMIT = 8 * 1024; // 8k
int64_t total_interm_result_size = 0;
int64_t slice_event_count = 0;
for (int64_t slice_idx = 0; OB_SUCC(ret) && slice_idx < get_split_task_count(); ++slice_idx) {
int64_t all_data_size = 0;
if (NULL == interm_result[slice_idx]) {
continue;
}
if (OB_FAIL(interm_result[slice_idx]->get_all_data_size(all_data_size))) {
LOG_WARN("fail to get all used mem size of irm", K(ret), K(slice_idx));
} else {
total_interm_result_size += all_data_size;
slice_event_count++;
}
}
if (OB_SUCC(ret)) {
slice_events->reset();
if (OB_FAIL(slice_events->prepare_allocate(slice_event_count))) {
LOG_WARN("fail to prepare allocate small result list", K(ret), K(get_split_task_count()));
}
}
int64_t slice_event_idx = 0;
ObSliceID ob_slice_id;
ob_slice_id.set_ob_task_id(trans_input->get_ob_task_id());
int64_t part_idx = -1;
int64_t subpart_idx = -1;
if (OB_SUCC(ret) && nullptr != table_schema) {
if (OB_FAIL(slice_infos.prepare_allocate(get_split_task_count()))) {
LOG_WARN("Prepare allocate failed", K(ret));
} else if (OB_FAIL(init_slice_infos(*table_schema, slice_infos))) {
LOG_WARN("fail init slice info", K(ret));
}
}
for (int64_t slice_idx = 0; OB_SUCC(ret) && slice_idx < get_split_task_count(); ++slice_idx) {
if (NULL == interm_result[slice_idx]) {
continue;
}
bool need_save_interm_result = true;
ObSliceEvent& slice_event = slice_events->at(slice_event_idx++);
int64_t data_size = 0;
ob_slice_id.set_slice_id(slice_idx);
slice_event.set_ob_slice_id(ob_slice_id);
if (nullptr != table_schema) {
part_idx = slice_infos[slice_idx].part_idx_;
subpart_idx = slice_infos[slice_idx].subpart_idx_;
} else {
part_idx = 0;
subpart_idx = 0;
}
if (OB_FAIL(get_part_shuffle_key(table_schema, part_idx, slice_event.get_part_shuffle_key()))) {
LOG_WARN("fail to get part shuffle key", K(ret), K(slice_idx), K(part_idx), K(subpart_idx));
} else if (OB_FAIL(get_subpart_shuffle_key(
table_schema, part_idx, subpart_idx, slice_event.get_subpart_shuffle_key()))) {
LOG_WARN("fail to get subpart shuffle key", K(ret), K(slice_idx), K(part_idx), K(subpart_idx));
} else if (OB_FAIL(
get_shuffle_part_key(table_schema, part_idx, subpart_idx, slice_event.get_shuffle_part_key()))) {
LOG_WARN("fail to get shuffle partition key", K(ret), K(slice_idx), K(part_idx), K(subpart_idx));
} else if (OB_FAIL(interm_result[slice_idx]->get_all_data_size(data_size))) {
LOG_WARN("fail to get data size of irm", K(ret), K(slice_idx));
}
if (OB_SUCC(ret) && (total_interm_result_size < TOTAL_SMALL_RESULT_MEM_LIMIT || 0 == data_size)) {
if (interm_result[slice_idx]->get_scanner_count() <= 1) {
if (!trans_input->is_force_save_interm_result()) {
ObTaskSmallResult& small_result = slice_event.get_small_result_for_update();
if (OB_FAIL(interm_result[slice_idx]->try_fetch_single_scanner(small_result))) {
LOG_WARN("fail copy small result scanner", K(ret), K(slice_idx));
} else {
if (small_result.has_data()) {
need_save_interm_result = false;
}
}
}
}
}
need_save_interm_result = true;
if (OB_SUCC(ret)) {
if (need_save_interm_result) {
ObIntermResultInfo res_info;
res_info.init(ob_slice_id);
if (OB_FAIL(interm_result_mgr->add_result(
res_info, interm_result[slice_idx], trans_input->get_expire_time()))) {
LOG_WARN("fail add one result. free all result", K(slice_idx), "total", get_split_task_count(), K(ret));
// last_fail_res_idx = slice_idx;
} else if (OB_FAIL(added_ir_info_list.push_back(res_info))) {
int free_ret = OB_SUCCESS;
if (OB_SUCCESS != (free_ret = interm_result_mgr->delete_result(res_info))) {
LOG_ERROR("fail free interm result, possible memory leak!", K(free_ret), K(slice_idx));
}
interm_result[slice_idx] = NULL;
}
} else {
if (OB_FAIL(interm_result_mgr->free_result(interm_result[slice_idx]))) {
LOG_ERROR("fail free interm result, possible memory leak!", K(ret), K(slice_idx));
}
interm_result[slice_idx] = NULL;
}
}
}
}
}
// free all result
if (OB_FAIL(ret)) {
if (OB_ISNULL(trans_input) || OB_ISNULL(interm_result_mgr)) {
LOG_ERROR("trans input or mgr is NULL", K(trans_input), K(interm_result_mgr));
} else {
int64_t idx = 0;
int free_ret = OB_SUCCESS;
// (1) delete those already added result
for (idx = 0; idx < added_ir_info_list.count(); ++idx) {
ObIntermResultIterator iter;
const ObIntermResultInfo& res_info = added_ir_info_list.at(idx);
if (OB_SUCCESS != (free_ret = interm_result_mgr->get_result(res_info, iter))) {
LOG_ERROR("fail get result. possible memory leak. will try recycle later", K(idx), K(free_ret));
} else if (OB_SUCCESS != (free_ret = interm_result_mgr->delete_result(iter))) {
LOG_ERROR("fail free interm result, possible memory leak!", K(free_ret));
}
}
// (2) delete those not yet added result
if (NULL != interm_result) {
for (/*cont.*/; idx < get_split_task_count(); ++idx) {
if (NULL != interm_result[idx]) { // this cond deal with alloc_result() fail case
if (OB_SUCCESS != (free_ret = interm_result_mgr->free_result(interm_result[idx]))) {
LOG_ERROR("fail free interm result, possible memory leak!", K(idx), K(free_ret));
}
}
}
} else {
}
}
}
return ret;
}
int ObDistributedTransmit::get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ret = ObPhyOperator::get_next_row(ctx, row);
return ret;
}
int ObDistributedTransmit::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
if (OB_ISNULL(child_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child op is null");
} else if (child_op_->is_dml_without_output()) {
ret = OB_ITER_END;
} else {
ret = child_op_->get_next_row(ctx, row);
}
return ret;
}
int ObDistributedTransmit::create_operator_input(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObIPhyOperatorInput* input = NULL;
if (OB_FAIL(CREATE_PHY_OP_INPUT(ObDistributedTransmitInput, ctx, get_id(), get_type(), input))) {
LOG_WARN("fail to create phy op input", K(ret), K(get_id()), K(get_type()));
} else {
}
UNUSED(input);
return ret;
}
int ObDistributedTransmit::init_slice_infos(
const share::schema::ObTableSchema& table_schema, ObIArray<ObSliceInfo>& slice_infos) const
{
int ret = OB_SUCCESS;
ObSliceInfo slice_info;
common::ObPartitionKey pkey;
ObPartitionKeyIter iter(table_schema.get_table_id(), table_schema, false);
while (OB_SUCC(ret) && OB_SUCC(iter.next_partition_key_v2(pkey))) {
slice_info.part_idx_ = pkey.get_part_idx();
slice_info.subpart_idx_ = pkey.get_subpart_idx();
if (OB_FAIL(get_slice_idx_by_partition_ids(
slice_info.part_idx_, slice_info.subpart_idx_, table_schema, slice_info.slice_idx_))) {
LOG_WARN("Failed to get slice idx", K(ret), K(slice_info));
} else if (slice_info.slice_idx_ < 0 || slice_info.slice_idx_ >= slice_infos.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid slice idx", K(ret), K(slice_info.slice_idx_), K(slice_infos.count()));
} else {
slice_infos.at(slice_info.slice_idx_) = slice_info;
}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
return ret;
}
int ObDistributedTransmit::get_slice_idx(ObExecContext& exec_ctx, const share::schema::ObTableSchema* table_schema,
const common::ObNewRow* row, const ObSqlExpression& part_func, const ObSqlExpression& subpart_func,
const ObIArray<ObTransmitRepartColumn>& repart_columns, const ObIArray<ObTransmitRepartColumn>& repart_sub_columns,
int64_t slices_count, int64_t& slice_idx, bool& no_match_partiton) const
{
int ret = OB_SUCCESS;
ObSliceInfo slice_info;
ObShuffleService shuffle_service(exec_ctx.get_allocator());
if (OB_FAIL(shuffle_service.get_partition_ids(exec_ctx,
*table_schema,
*row,
part_func,
subpart_func,
repart_columns,
repart_sub_columns,
slice_info.part_idx_,
slice_info.subpart_idx_,
no_match_partiton))) {
LOG_WARN("Failed to get part/subpart idx", K(ret), K(slice_info));
} else if (no_match_partiton) {
// do nothing
} else if (OB_FAIL(get_slice_idx_by_partition_ids(
slice_info.part_idx_, slice_info.subpart_idx_, *table_schema, slice_info.slice_idx_))) {
LOG_WARN("Failed to get slice idx", K(ret), K(slice_info));
} else if (slice_info.slice_idx_ < 0 || slice_info.slice_idx_ >= slices_count) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Invalid slice idx", K(ret), K(slice_info.slice_idx_), K(slices_count));
} else {
slice_idx = slice_info.slice_idx_;
}
return ret;
}
OB_DEF_SERIALIZE(ObDistributedTransmit)
{
int ret = OK_;
UNF_UNUSED_SER;
BASE_SER((ObDistributedTransmit, ObTransmit));
#if 0
OB_ASSERT(shuffle_func_);
LST_DO_CODE(OB_UNIS_ENCODE, *shuffle_func_);
#endif
return ret;
}
OB_DEF_DESERIALIZE(ObDistributedTransmit)
{
int ret = OK_;
UNF_UNUSED_DES;
BASE_DESER((ObDistributedTransmit, ObTransmit));
return ret;
if (OB_SUCC(ret)) {
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, shuffle_func_))) {
LOG_WARN("make sql expression failed", K(ret));
} else {
#if 0
LST_DO_CODE(OB_UNIS_DECODE, *shuffle_func_);
#endif
}
}
return ret;
}
OB_DEF_SERIALIZE_SIZE(ObDistributedTransmit)
{
int64_t len = 0;
BASE_ADD_LEN((ObDistributedTransmit, ObTransmit));
return len;
#if 0
OB_ASSERT(shuffle_func_);
LST_DO_CODE(OB_UNIS_ADD_LEN, *shuffle_func_);
return len;
#endif
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,171 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TRANSMIT_
#define OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TRANSMIT_
#include "share/schema/ob_table_schema.h"
#include "sql/executor/ob_transmit.h"
#include "sql/executor/ob_slice_id.h"
#include "sql/executor/ob_task_event.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObDistributedTransmitInput : public ObTransmitInput {
OB_UNIS_VERSION_V(1);
public:
ObDistributedTransmitInput()
: ObTransmitInput(), expire_time_(0), ob_task_id_(), force_save_interm_result_(false), slice_events_(NULL)
{}
virtual ~ObDistributedTransmitInput()
{}
virtual void reset() override
{
ObTransmitInput::reset();
expire_time_ = 0;
ob_task_id_.reset();
force_save_interm_result_ = false;
slice_events_ = NULL;
}
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
inline virtual ObPhyOperatorType get_phy_op_type() const
{
return PHY_DISTRIBUTED_TRANSMIT;
}
inline void set_ob_task_id(const ObTaskID& id)
{
ob_task_id_ = id;
}
inline ObTaskID& get_ob_task_id()
{
return ob_task_id_;
}
inline bool is_force_save_interm_result() const
{
return force_save_interm_result_;
}
inline void set_slice_events(common::ObIArray<ObSliceEvent>* slice_events)
{
slice_events_ = slice_events;
}
inline common::ObIArray<ObSliceEvent>* get_slice_events_for_update() const
{
return slice_events_;
}
inline int64_t get_expire_time()
{
return expire_time_;
}
private:
int64_t expire_time_;
ObTaskID ob_task_id_;
bool force_save_interm_result_;
common::ObIArray<ObSliceEvent>* slice_events_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedTransmitInput);
};
class ObDistributedTransmit : public ObTransmit {
OB_UNIS_VERSION_V(1);
private:
class ObSliceInfo {
public:
ObSliceInfo()
: part_offset_(OB_INVALID_INDEX_INT64),
subpart_offset_(OB_INVALID_INDEX_INT64),
part_idx_(OB_INVALID_INDEX_INT64),
subpart_idx_(OB_INVALID_INDEX_INT64),
slice_idx_(OB_INVALID_INDEX_INT64)
{}
virtual ~ObSliceInfo() = default;
TO_STRING_KV(K(part_offset_), K(subpart_offset_), K(part_idx_), K(subpart_idx_), K(slice_idx_))
int64_t part_offset_;
int64_t subpart_offset_;
int64_t part_idx_;
int64_t subpart_idx_;
int64_t slice_idx_;
};
protected:
class ObDistributedTransmitCtx : public ObTransmitCtx {
friend class ObDistributedTransmit;
public:
explicit ObDistributedTransmitCtx(ObExecContext& ctx) : ObTransmitCtx(ctx)
{}
virtual ~ObDistributedTransmitCtx()
{}
virtual void destroy()
{
ObTransmitCtx::destroy();
}
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedTransmitCtx);
};
public:
explicit ObDistributedTransmit(common::ObIAllocator& alloc);
virtual ~ObDistributedTransmit();
virtual int create_operator_input(ObExecContext& ctx) const;
inline void set_shuffle_func(ObSqlExpression* shuffle_func);
int get_part_shuffle_key(
const share::schema::ObTableSchema* table_schema, int64_t part_idx, ObShuffleKey& part_shuffle_key) const;
int get_subpart_shuffle_key(const share::schema::ObTableSchema* table_schema, int64_t part_idx, int64_t subpart_idx,
ObShuffleKey& subpart_shuffle_key) const;
int get_shuffle_part_key(const share::schema::ObTableSchema* table_schema, int64_t part_idx, int64_t subpart_idx,
common::ObPartitionKey& shuffle_part_key) const;
private:
int init_slice_infos(
const share::schema::ObTableSchema& table_schema, common::ObIArray<ObSliceInfo>& slices_info) const;
int get_slice_idx(ObExecContext& exec_ctx, const share::schema::ObTableSchema* table_schema,
const common::ObNewRow* row, const ObSqlExpression& part_partition_func,
const ObSqlExpression& subpart_partition_func, const ObIArray<ObTransmitRepartColumn>& repart_columns,
const ObIArray<ObTransmitRepartColumn>& repart_sub_columns, int64_t slices_count, int64_t& slice_idx,
bool& no_match_partiton) const;
protected:
virtual int inner_open(ObExecContext& exec_ctx) const;
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
bool skip_empty_slice() const;
int prepare_interm_result(ObIntermResultManager& interm_result_mgr, ObIntermResult*& interm_result) const;
int get_next_row(ObExecContext& ctx, const ObNewRow*& row) const override;
int inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const;
private:
const static int64_t NO_MATCH_PARTITION = -2;
ObSqlExpression* shuffle_func_;
DISALLOW_COPY_AND_ASSIGN(ObDistributedTransmit);
};
inline void ObDistributedTransmit::set_shuffle_func(ObSqlExpression* shuffle_func)
{
shuffle_func_ = shuffle_func;
}
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_DISTRIBUTED_TRANSMIT_ */

View File

@ -0,0 +1,266 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_execute_result.h"
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_operator.h"
#include "sql/engine/ob_exec_context.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObExecuteResult::ObExecuteResult() : err_code_(OB_ERR_UNEXPECTED), root_op_(NULL), static_engine_root_(NULL)
{}
int ObExecuteResult::open(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
// TODO : temporary code.
// We should invoke open(void) directly.
if (NULL != static_engine_root_) {
ret = open();
} else {
if (OB_ISNULL(root_op_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(root_op_->open(ctx))) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("root op fail to open", K(ret));
}
}
}
return ret;
}
int ObExecuteResult::get_next_row(ObExecContext& ctx, const common::ObNewRow*& row)
{
int ret = OB_SUCCESS;
// TODO : temporary code.
// We should invoke get_next_row(void) directly.
if (NULL != static_engine_root_) {
bool got_row = false;
ret = get_next_row();
// convert datum to obj
if (OB_SUCC(ret)) {
row = &row_;
const ObOpSpec& spec = static_engine_root_->get_spec();
if (spec.output_.count() > 0 && NULL == row_.cells_) {
if (OB_ISNULL(
row_.cells_ = static_cast<ObObj*>(ctx.get_allocator().alloc(sizeof(ObObj) * spec.output_.count())))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
for (int64_t i = 0; i < spec.output_.count(); i++) {
new (&row_.cells_[i]) ObObj();
}
row_.count_ = spec.output_.count();
row_.projector_size_ = 0;
row_.projector_ = NULL;
}
}
if (OB_SUCC(ret)) {
for (int64_t i = 0; OB_SUCC(ret) && i < spec.output_.count(); i++) {
ObDatum* datum = NULL;
ObExpr* expr = spec.output_.at(i);
if (OB_FAIL(expr->eval(static_engine_root_->get_eval_ctx(), datum))) {
LOG_WARN("expr evaluate failed", K(ret));
} else if (OB_FAIL(datum->to_obj(row_.cells_[i], expr->obj_meta_, expr->obj_datum_map_))) {
LOG_WARN("convert datum to obj failed", K(ret));
}
}
}
}
} else {
bool got_row = false;
ObPhysicalPlanCtx* plan_ctx = NULL;
if (OB_ISNULL(root_op_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_ISNULL(plan_ctx = ctx.get_physical_plan_ctx())) {
ret = OB_NOT_INIT;
LOG_WARN("physical plan ctx is null", K(ret));
}
// swtich bind array iterator in DML returning plan
while (OB_SUCC(ret) && !got_row) {
if (OB_FAIL(root_op_->get_next_row(ctx, row))) {
if (OB_ITER_END == ret) {
if (plan_ctx->get_bind_array_count() <= 0) {
// not contain bind array, do nothing
} else if (OB_FAIL(root_op_->switch_iterator(ctx))) {
if (OB_ITER_END != ret) {
LOG_WARN("switch op iterator failed", K(ret), "op_type", ob_phy_operator_type_str(root_op_->get_type()));
}
}
} else if (OB_TRY_LOCK_ROW_CONFLICT != ret) {
LOG_WARN("get next row from operator failed", K(ret));
}
} else {
got_row = true;
}
}
}
return ret;
}
int ObExecuteResult::close(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
// TODO : temporary code.
// We should invoke get_next_row(void) directly.
if (NULL != static_engine_root_) {
ret = close();
} else {
if (OB_ISNULL(root_op_)) {
// ret = OB_NOT_INIT;
ret = OB_SUCCESS;
} else if (OB_FAIL(root_op_->close(ctx))) {
LOG_WARN("root op fail to close", K(ret));
} else {
}
}
return ret;
}
int ObExecuteResult::open() const
{
int ret = OB_SUCCESS;
if (OB_ISNULL(static_engine_root_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(static_engine_root_->open())) {
if (OB_TRY_LOCK_ROW_CONFLICT != ret && OB_TRANSACTION_SET_VIOLATION != ret) {
LOG_WARN("open operator failed", K(ret));
}
}
return ret;
}
int ObExecuteResult::get_next_row() const
{
int ret = OB_SUCCESS;
bool got_row = false;
if (OB_ISNULL(static_engine_root_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret), KP(static_engine_root_));
}
// switch bind array iterator in DML returning plan
while (OB_SUCC(ret) && !got_row) {
if (OB_FAIL(static_engine_root_->get_next_row())) {
if (OB_ITER_END == ret) {
ObPhysicalPlanCtx* plan_ctx = static_engine_root_->get_exec_ctx().get_physical_plan_ctx();
if (plan_ctx->get_bind_array_count() <= 0 ||
plan_ctx->get_bind_array_idx() >= plan_ctx->get_bind_array_count()) {
// no bind array or reach binding array end, do nothing
} else {
plan_ctx->inc_bind_array_idx();
if (OB_FAIL(static_engine_root_->switch_iterator())) {
if (OB_ITER_END != ret) {
LOG_WARN("switch op iterator failed", K(ret), "op_type", static_engine_root_->op_name());
}
}
}
} else if (OB_TRY_LOCK_ROW_CONFLICT != ret) {
LOG_WARN("get next row from operator failed", K(ret));
}
} else {
got_row = true;
}
}
return ret;
}
int ObExecuteResult::close() const
{
int ret = OB_SUCCESS;
if (NULL != static_engine_root_) {
if (OB_FAIL(static_engine_root_->close())) {
LOG_WARN("close failed", K(ret));
}
}
return ret;
}
ObAsyncExecuteResult::ObAsyncExecuteResult() : field_count_(0), scanner_(nullptr), cur_row_(nullptr), spec_(nullptr)
{}
int ObAsyncExecuteResult::open(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
ObPhysicalPlanCtx* plan_ctx = ctx.get_physical_plan_ctx();
ObSQLSessionInfo* session = ctx.get_my_session();
if (OB_ISNULL(scanner_) || OB_ISNULL(plan_ctx) || OB_ISNULL(session)) {
ret = OB_NOT_INIT;
LOG_WARN("scanner is invalid", K(ret), K(scanner_), K(plan_ctx), K(session));
} else if (ObTaskExecutorCtxUtil::merge_task_result_meta(*plan_ctx, *scanner_)) {
LOG_WARN("merge task result meta failed", K(ret), KPC_(scanner));
} else if (OB_FAIL(session->replace_user_variables(ctx, scanner_->get_session_var_map()))) {
LOG_WARN("replace user variables failed", K(ret));
} else if (field_count_ <= 0) {
// no date from remote, so don't need to create row buffer.
} else if (OB_FAIL(ob_create_row(ctx.get_allocator(), field_count_, cur_row_))) {
LOG_WARN("create current row failed", K(ret), K(field_count_));
} else {
if (nullptr == spec_) {
row_iter_ = scanner_->begin();
} else {
if (OB_FAIL(scanner_->get_datum_store().begin(datum_iter_))) {
LOG_WARN("fail to init datum iter", K(ret));
}
}
}
return ret;
}
int ObAsyncExecuteResult::get_next_row(ObExecContext& ctx, const ObNewRow*& row)
{
UNUSED(ctx);
int ret = OB_SUCCESS;
if (OB_ISNULL(cur_row_)) {
ret = OB_NOT_INIT;
LOG_WARN("scanner is invalid", K(ret));
} else if (nullptr == spec_) {
if (OB_FAIL(row_iter_.get_next_row(*cur_row_))) {
if (OB_ITER_END != ret) {
LOG_WARN("get next row from row iterator failed", K(ret));
}
}
} else {
// Static engine.
// For async execute result, ObExecContext::eval_ctx_ is destroyed, can not be used.
const ObChunkDatumStore::StoredRow* sr = NULL;
if (OB_FAIL(datum_iter_.get_next_row(sr))) {
if (OB_ITER_END != ret) {
LOG_WARN("get next row from datum iterator failed", K(ret));
}
} else if (OB_ISNULL(sr) || spec_->output_.count() != sr->cnt_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("store row is NULL or datum count mismatch", K(ret), KP(sr), K(spec_->output_.count()));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < spec_->output_.count(); i++) {
const sql::ObExpr* e = spec_->output_.at(i);
if (OB_FAIL(sr->cells()[i].to_obj(cur_row_->cells_[i], e->obj_meta_, e->obj_datum_map_))) {
LOG_WARN("convert datum to obj failed", K(ret));
}
}
}
}
if (OB_SUCC(ret)) {
row = cur_row_;
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,127 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_EXECUTE_RESULT_
#define OCEANBASE_SQL_EXECUTOR_OB_EXECUTE_RESULT_
#include "common/row/ob_row.h"
#include "share/ob_scanner.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObPhyOperator;
class ObOperator;
class ObOpSpec;
class ObIExecuteResult {
public:
virtual ~ObIExecuteResult()
{}
virtual int open(ObExecContext& ctx) = 0;
virtual int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) = 0;
virtual int close(ObExecContext& ctx) = 0;
};
class ObExecuteResult : public ObIExecuteResult {
friend class ObLocalTaskExecutor;
friend class ObExecutor;
public:
ObExecuteResult();
virtual ~ObExecuteResult()
{}
virtual int open(ObExecContext& ctx) override;
virtual int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) override;
virtual int close(ObExecContext& ctx) override;
inline const ObPhyOperator* get_root_op() const
{
return root_op_;
}
inline int get_err_code()
{
return err_code_;
}
// interface for static typing engine
int open() const;
int get_next_row() const;
int close() const;
const ObOperator* get_static_engine_root() const
{
return static_engine_root_;
}
void set_static_engine_root(ObOperator* op)
{
static_engine_root_ = op;
}
inline void set_root_op(ObPhyOperator* root_op)
{
root_op_ = root_op;
}
private:
int err_code_;
ObPhyOperator* root_op_;
ObOperator* static_engine_root_;
// TODO : temporary code.
// row used to adapt old get_next_row interface.
mutable common::ObNewRow row_;
private:
DISALLOW_COPY_AND_ASSIGN(ObExecuteResult);
};
class ObAsyncExecuteResult : public ObIExecuteResult {
public:
ObAsyncExecuteResult();
virtual ~ObAsyncExecuteResult()
{}
void set_result_stream(common::ObScanner* scanner, int64_t field_count)
{
scanner_ = scanner;
field_count_ = field_count;
}
virtual int open(ObExecContext& ctx) override;
virtual int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) override;
virtual int close(ObExecContext& ctx) override
{
UNUSED(ctx);
return common::OB_SUCCESS;
}
// interface for static typing engine
const ObOpSpec* get_static_engine_spec() const
{
return spec_;
}
void set_static_engine_spec(const ObOpSpec* spec)
{
spec_ = spec;
}
private:
int64_t field_count_;
common::ObScanner* scanner_;
common::ObNewRow* cur_row_;
common::ObScanner::Iterator row_iter_;
// used for static engine
const ObOpSpec* spec_;
ObChunkDatumStore::Iterator datum_iter_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_EXECUTE_RESULT_ */

View File

@ -0,0 +1,40 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include "sql/executor/ob_execution_id.h"
#include "lib/json/ob_yson.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
const common::ObAddr& ObExecutionID::global_id_addr()
{
static ObAddr global_addr(1, 1); // 1.0.0.0:1
return global_addr;
}
DEFINE_TO_YSON_KV(ObExecutionID, OB_ID(addr), server_, OB_ID(execution_id), execution_id_);
OB_SERIALIZE_MEMBER(ObExecutionID, server_, execution_id_, execution_flag_);
int ObExecutionID::compare(const ObExecutionID& other) const
{
int cmp_ret = 0;
if (execution_id_ > other.execution_id_) {
cmp_ret = 1;
} else if (execution_id_ < other.execution_id_) {
cmp_ret = -1;
}
return cmp_ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,122 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_EXECUTION_ID_
#define OCEANBASE_SQL_EXECUTOR_OB_EXECUTION_ID_
#include "lib/net/ob_addr.h"
namespace oceanbase {
namespace sql {
// execution_type, no more than 255.
static const uint64_t ET_DIST_TASK = 0; // dist task must be 0 for compatibility.
static const uint64_t ET_MINI_TASK = 1;
class ObExecutionID final {
OB_UNIS_VERSION(1);
public:
public:
ObExecutionID(const common::ObAddr& server, uint64_t execution_id)
: server_(server), execution_id_(execution_id), task_type_(ET_DIST_TASK)
{}
ObExecutionID() : server_(), execution_id_(common::OB_INVALID_ID), task_type_(ET_DIST_TASK)
{}
inline void set_server(const common::ObAddr& server)
{
server_ = server;
}
inline void set_execution_id(const uint64_t execution_id)
{
execution_id_ = execution_id;
}
inline void set_task_type(uint64_t task_type)
{
task_type_ = task_type;
}
inline void set_dist_task_type()
{
task_type_ = ET_DIST_TASK;
}
inline void set_mini_task_type()
{
task_type_ = ET_MINI_TASK;
}
inline const common::ObAddr& get_server() const
{
return server_;
}
inline uint64_t get_execution_id() const
{
return execution_id_;
}
inline uint64_t get_task_type() const
{
return task_type_;
}
inline bool is_dist_task_type() const
{
return task_type_ == ET_DIST_TASK;
}
inline bool is_mini_task_type() const
{
return task_type_ == ET_MINI_TASK;
}
inline bool equal(const ObExecutionID& id) const
{
return id.server_ == server_ && id.execution_id_ == execution_id_ && id.task_type_ == task_type_;
}
inline int64_t hash() const
{
// ignore server_ because servers are same in most cases.
return common::murmurhash(&execution_id_, sizeof(execution_id_), 0);
}
int compare(const ObExecutionID& other) const;
inline bool operator==(const ObExecutionID& id) const
{
return equal(id);
}
inline bool is_valid() const
{
return server_.is_valid() && common::OB_INVALID_ID != execution_id_;
}
inline void reset()
{
server_.reset();
execution_id_ = common::OB_INVALID_ID;
// ET_DIST_TASK is initial value in constructor.
task_type_ = ET_DIST_TASK;
}
// print hash value for debug.
TO_STRING_KV(
N_SERVER, server_, N_EXECUTION_ID, execution_id_, N_TASK_TYPE, task_type_, "hash", static_cast<uint64_t>(hash()));
DECLARE_TO_YSON_KV;
// fake control server address for global execution_id
static const common::ObAddr& global_id_addr();
private:
common::ObAddr server_;
uint64_t execution_id_;
union {
uint64_t execution_flag_;
struct {
uint64_t task_type_ : 8;
uint64_t reserved_ : 56;
};
};
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_EXECUTION_ID_ */

View File

@ -0,0 +1,327 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_executor.h"
#include "lib/stat/ob_diagnose_info.h"
#include "sql/executor/ob_distributed_scheduler.h"
#include "sql/executor/ob_remote_scheduler.h"
#include "sql/executor/ob_local_scheduler.h"
#include "sql/executor/ob_task_executor_ctx.h"
#include "sql/executor/ob_execute_result.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/ob_operator.h"
#include "sql/engine/table/ob_table_scan.h"
#include "lib/profile/ob_perf_event.h"
#include "sql/executor/ob_transmit.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
ObExecutor::ObExecutor() : inited_(false), phy_plan_(NULL), execution_id_(OB_INVALID_ID)
{
/* add your code here */
}
int ObExecutor::init(ObPhysicalPlan* plan)
{
int ret = OB_SUCCESS;
if (true == inited_) {
ret = OB_INIT_TWICE;
LOG_WARN("executor is inited twice", K(ret));
} else if (OB_ISNULL(plan)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("plan is NULL", K(ret));
} else {
phy_plan_ = plan;
inited_ = true;
}
return ret;
}
void ObExecutor::reset()
{
inited_ = false;
phy_plan_ = NULL;
execution_id_ = OB_INVALID_ID;
}
int ObExecutor::execute_plan(ObExecContext& ctx)
{
NG_TRACE(exec_plan_begin);
int ret = OB_SUCCESS;
ObTaskExecutorCtx& task_exec_ctx = ctx.get_task_exec_ctx();
ObExecuteResult& exec_result = task_exec_ctx.get_execute_result();
ObSQLSessionInfo* session_info = ctx.get_my_session();
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not inited", K(ret));
} else if (OB_ISNULL(session_info)) {
ret = OB_NOT_INIT;
LOG_WARN("session info is NULL", K(ret));
} else if (OB_ISNULL(phy_plan_)) {
ret = OB_NOT_INIT;
LOG_WARN("phy_plan_ is NULL", K(ret));
} else if (OB_FAIL(session_info->set_cur_phy_plan(phy_plan_))) {
LOG_WARN("set extra serialize vars", K(ret));
} else if (session_info->use_static_typing_engine() &&
OB_FAIL(phy_plan_->get_expr_frame_info().pre_alloc_exec_memory(ctx))) {
LOG_WARN("fail to pre allocate memory", K(ret), K(phy_plan_->get_expr_frame_info()));
} else if (session_info->use_static_typing_engine() && OB_FAIL(ctx.init_eval_ctx())) {
LOG_WARN("init eval ctx failed", K(ret));
} else {
ObPhyPlanType execute_type = phy_plan_->get_plan_type();
if (phy_plan_->get_need_serial_exec()) {
session_info->set_need_serial_exec(phy_plan_->get_need_serial_exec());
}
// consider this case:
// MULTI PART INSERT (remote)
// SELECT (local)
// the plan type is OB_PHY_PLAN_DISTRIBUTED, but need schedule as LOCAL plan.
if (execute_type != OB_PHY_PLAN_LOCAL && phy_plan_->is_require_local_execution()) {
execute_type = OB_PHY_PLAN_LOCAL;
LOG_TRACE("change the plan execution type", "fact", execute_type, K(phy_plan_->get_plan_type()));
}
switch (execute_type) {
case OB_PHY_PLAN_LOCAL: {
EVENT_INC(SQL_LOCAL_COUNT);
if (NULL == phy_plan_->get_root_op_spec()) {
// root operator spec is NULL, old plan
exec_result.set_root_op(phy_plan_->get_main_query());
} else {
ObOperator* op = NULL;
if (OB_FAIL(phy_plan_->get_root_op_spec()->create_operator(ctx, op))) {
LOG_WARN("create operator from spec failed", K(ret));
} else if (OB_ISNULL(op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("created operator is NULL", K(ret));
} else {
exec_result.set_static_engine_root(op);
}
}
break;
}
case OB_PHY_PLAN_REMOTE:
EVENT_INC(SQL_REMOTE_COUNT);
ret = execute_remote_single_partition_plan(ctx);
break;
case OB_PHY_PLAN_DISTRIBUTED:
EVENT_INC(SQL_DISTRIBUTED_COUNT);
if (phy_plan_->is_use_px()) {
// ObPxCoord will do schedule job.
if (NULL != phy_plan_->get_root_op_spec()) {
ret = execute_static_cg_px_plan(ctx);
} else {
ret = execute_old_px_plan(ctx);
}
} else {
if (OB_FAIL(task_exec_ctx.reset_and_init_stream_handler())) {
LOG_WARN("init stream handler failed", K(ret));
} else {
// user var & distributed => not supported
if (phy_plan_->is_contains_assignment()) {
ret = OB_ERR_DISTRIBUTED_NOT_SUPPORTED;
LOG_USER_ERROR(OB_ERR_DISTRIBUTED_NOT_SUPPORTED, "user variable assignment in distributed plan");
} else {
ret = execute_distributed_plan(ctx);
}
}
}
break;
default:
ret = OB_ERR_UNEXPECTED;
break;
}
}
NG_TRACE(exec_plan_end);
return ret;
}
int ObExecutor::execute_local_single_partition_plan(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
ObLocalScheduler scheduler;
if (OB_ISNULL(phy_plan_)) {
ret = OB_ERR_UNEXPECTED;
} else {
ret = scheduler.schedule(ctx, phy_plan_);
}
return ret;
}
int ObExecutor::execute_remote_single_partition_plan(ObExecContext& ctx)
{
ObRemoteScheduler scheduler;
return scheduler.schedule(ctx, phy_plan_);
}
int ObExecutor::execute_distributed_plan(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
OB_ASSERT(NULL != phy_plan_);
ObDistributedSchedulerManager* scheduler_manager = NULL;
ObPhysicalPlanCtx* phy_plan_ctx = ctx.get_physical_plan_ctx();
int64_t remain_time_us = 0;
int64_t now = ::oceanbase::common::ObTimeUtility::current_time();
if (OB_ISNULL(phy_plan_) || OB_ISNULL(phy_plan_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("plan or context is NULL", K_(phy_plan), K(phy_plan_ctx));
} else if (OB_UNLIKELY((remain_time_us = phy_plan_ctx->get_timeout_timestamp() - now) <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("timeout", K(ret), K(remain_time_us), K(now), "timeout_timestamp", phy_plan_ctx->get_timeout_timestamp());
} else if (OB_ISNULL(scheduler_manager = ObDistributedSchedulerManager::get_instance())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get ObDistributedSchedulerManager instance", K(ret));
} else if (OB_FAIL(scheduler_manager->alloc_scheduler(ctx, execution_id_))) {
execution_id_ = OB_INVALID_ID;
LOG_WARN("fail to alloc scheduler", K(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == execution_id_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succeed to alloc but execution id is invalid", K(execution_id_), K(ret));
} else if (OB_FAIL(scheduler_manager->parse_jobs_and_start_sche_thread(
execution_id_, ctx, phy_plan_, phy_plan_ctx->get_timeout_timestamp()))) {
LOG_WARN("fail to schedule", K(ret));
} else {
ctx.set_execution_id(execution_id_);
}
return ret;
}
int ObExecutor::execute_static_cg_px_plan(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
ObOperator* op = NULL;
if (OB_FAIL(phy_plan_->get_root_op_spec()->create_op_input(ctx))) {
LOG_WARN("create input from spec failed", K(ret));
} else if (OB_FAIL(phy_plan_->get_root_op_spec()->create_operator(ctx, op))) {
LOG_WARN("create operator from spec failed", K(ret));
} else if (OB_ISNULL(op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("created operator is NULL", K(ret));
} else {
/**
* FIXME
* these codes are ugly
*/
ObSEArray<const ObTableScanSpec*, 8> scan_ops;
// pre query range and init scan input (for compatible)
if (OB_FAIL(ObTaskSpliter::find_scan_ops(scan_ops, *phy_plan_->get_root_op_spec()))) {
LOG_WARN("fail get scan ops", K(ret));
} else {
ARRAY_FOREACH_X(scan_ops, idx, cnt, OB_SUCC(ret))
{
if (OB_ISNULL(scan_ops.at(idx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL scan op ptr unexpected", K(ret));
} else {
ObOperatorKit* kit = ctx.get_operator_kit(scan_ops.at(idx)->get_id());
if (OB_ISNULL(kit) || OB_ISNULL(kit->input_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("operator is NULL", K(ret), KP(kit));
} else {
ObTableScanOpInput* scan_input = static_cast<ObTableScanOpInput*>(kit->input_);
// hard code idx to 0
scan_input->set_location_idx(0);
}
}
}
}
}
if (OB_SUCC(ret)) {
ctx.get_task_executor_ctx()->get_execute_result().set_static_engine_root(op);
}
return ret;
}
int ObExecutor::execute_old_px_plan(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObJobControl::alloc_phy_op_input(ctx, phy_plan_->get_main_query()))) {
LOG_WARN("fail alloc all op input", K(ret));
} else {
ObSEArray<const ObTableScan*, 8> scan_ops;
// pre query range and init scan input (for compatible)
if (OB_FAIL(ObTaskSpliter::find_scan_ops(scan_ops, *phy_plan_->get_main_query()))) {
LOG_WARN("fail get scan ops", K(ret));
} else {
ARRAY_FOREACH_X(scan_ops, idx, cnt, OB_SUCC(ret))
{
ObTableScanInput* tsc_input = NULL;
if (OB_ISNULL(scan_ops.at(idx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL scan op ptr unexpected", K(ret));
} else if (OB_ISNULL(tsc_input = GET_PHY_OP_INPUT(ObTableScanInput, ctx, scan_ops.at(idx)->get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("can't get tsc op input", K(ret));
} else {
// hard code idx to 0
tsc_input->set_location_idx(0);
}
}
}
}
if (OB_SUCC(ret)) {
ctx.get_task_exec_ctx().get_execute_result().set_root_op(phy_plan_->get_main_query());
}
return ret;
}
int ObExecutor::close(ObExecContext& ctx)
{
// close() may be called anytime, so ignore inited_.
int ret = OB_SUCCESS;
ObSQLSessionInfo* session_info = ctx.get_my_session();
if (OB_LIKELY(NULL != session_info)) {
session_info->reset_cur_phy_plan_to_null();
}
if (OB_LIKELY(NULL != phy_plan_)) {
ObPhyPlanType execute_type = phy_plan_->get_plan_type();
switch (execute_type) {
case OB_PHY_PLAN_LOCAL:
case OB_PHY_PLAN_REMOTE:
break;
case OB_PHY_PLAN_DISTRIBUTED: {
int free_ret = OB_SUCCESS;
ObDistributedSchedulerManager* scheduler_manager = NULL;
if (phy_plan_->is_use_px()) {
// do nothing
} else if (OB_UNLIKELY(OB_INVALID_ID == execution_id_)) {
// fail to alloc distributed scheduler, do nothing
LOG_WARN("fail to alloc distributed scheduler, do nothing", K(ret), K(execution_id_));
} else if (OB_UNLIKELY(NULL == (scheduler_manager = ObDistributedSchedulerManager::get_instance()))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fail to get ObDistributedSchedulerManager instance", K(ret));
} else {
if (OB_FAIL(scheduler_manager->close_scheduler(ctx, execution_id_))) {
LOG_WARN("fail to close scheduler", K(ret), K(execution_id_));
}
if (OB_SUCCESS != (free_ret = scheduler_manager->free_scheduler(execution_id_))) {
ret = (OB_SUCCESS == ret) ? free_ret : ret;
LOG_ERROR("fail to free scheduler", K(ret), K(free_ret));
}
}
break;
}
default:
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid execute_type", K(ret), K(execute_type));
break;
}
}
return ret;
}

View File

@ -0,0 +1,52 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_
#define OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_
#include "share/ob_define.h"
namespace oceanbase {
namespace sql {
class ObPhysicalPlan;
class ObExecContext;
class ObPhyOperator;
class ObExecutor {
public:
ObExecutor();
~ObExecutor(){};
int init(ObPhysicalPlan* plan);
void reset();
int execute_plan(ObExecContext& ctx);
int close(ObExecContext& ctx);
private:
// disallow copy
ObExecutor(const ObExecutor& other);
ObExecutor& operator=(const ObExecutor& ohter);
private:
int execute_local_single_partition_plan(ObExecContext& ctx);
int execute_remote_single_partition_plan(ObExecContext& ctx);
int execute_distributed_plan(ObExecContext& ctx);
int execute_old_px_plan(ObExecContext& ctx);
int execute_static_cg_px_plan(ObExecContext& ctx);
private:
bool inited_;
ObPhysicalPlan* phy_plan_;
uint64_t execution_id_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_ */

View File

@ -0,0 +1,661 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "ob_executor_rpc_impl.h"
#include "share/ob_worker.h"
#include "share/ob_cluster_version.h"
#include "sql/ob_sql_context.h"
#include "sql/executor/ob_executor_rpc_processor.h"
#include "sql/executor/ob_remote_executor_processor.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
int ObExecutorRpcImpl::init(obrpc::ObExecutorRpcProxy* rpc_proxy, obrpc::ObBatchRpc* batch_rpc)
{
int ret = OB_SUCCESS;
proxy_ = rpc_proxy;
batch_rpc_ = batch_rpc;
return ret;
}
int ObExecutorRpcImpl::mini_task_execute(ObExecutorRpcCtx& rpc_ctx, ObMiniTask& task, ObMiniTaskResult& result)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
const ObAddr& svr = task.get_runner_server();
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
obrpc::ObExecutorRpcProxy to_proxy = proxy_->to(svr);
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (OB_FAIL(to_proxy.by(tenant_id).as(OB_SYS_TENANT_ID).timeout(timeout).mini_task_execute(task, result))) {
LOG_WARN("rpc task_execute fail", K(ret), K(tenant_id), K(svr), K(timeout), K(timeout_timestamp));
const obrpc::ObRpcResultCode& rcode = to_proxy.get_result_code();
if (OB_LIKELY(OB_SUCCESS != rcode.rcode_)) {
FORWARD_USER_ERROR(rcode.rcode_, rcode.msg_);
}
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
return ret;
}
int ObExecutorRpcImpl::mini_task_submit(ObExecutorRpcCtx& rpc_ctx, ObMiniTask& task)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
ObCurTraceId::TraceId* cur_trace_id = NULL;
const ObAddr& svr = task.get_runner_server();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("proxy_ is NULL", K(ret), K(proxy_));
} else if (OB_UNLIKELY(!rpc_ctx.min_cluster_version_is_valid())) { // only local execution
ret = OB_ERR_UNEXPECTED;
LOG_WARN("min cluster version is invalid", K(ret), K(rpc_ctx.get_min_cluster_version()));
} else if (OB_ISNULL(cur_trace_id = ObCurTraceId::get_trace_id())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("current trace id is NULL", K(ret));
} else if (OB_ISNULL(rpc_ctx.get_ap_mini_task_mgr())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("ap mini task mgr is null", K(ret), K(rpc_ctx.get_ap_mini_task_mgr()));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_submit timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else {
ObRpcAPMiniDistExecuteCB task_submit_cb(
rpc_ctx.get_ap_mini_task_mgr(), task.get_ob_task_id(), *cur_trace_id, svr, timeout_timestamp);
if (OB_FAIL(proxy_->to(svr)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.ap_mini_task_submit(task, &task_submit_cb))) {
LOG_WARN("rpc ap mini task_submit fail", K(ret), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
}
if (OB_FAIL(ret)) {
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
}
return ret;
}
int ObExecutorRpcImpl::ping_sql_task(ObExecutorPingRpcCtx& ping_ctx, ObPingSqlTask& ping_task)
{
int ret = OB_SUCCESS;
const ObAddr& exec_svr = ping_task.exec_svr_;
uint64_t tenant_id = THIS_WORKER.get_rpc_tenant() > 0 ? THIS_WORKER.get_rpc_tenant() : ping_ctx.get_rpc_tenant_id();
int64_t wait_timeout = ping_ctx.get_wait_timeout();
ObRpcAPPingSqlTaskCB task_cb(ping_task.task_id_);
OV(OB_NOT_NULL(proxy_));
switch (ping_task.task_id_.get_task_type()) {
case ET_DIST_TASK:
OZ(task_cb.set_dist_task_mgr(ping_ctx.get_dist_task_mgr()));
break;
case ET_MINI_TASK:
OZ(task_cb.set_mini_task_mgr(ping_ctx.get_mini_task_mgr()));
break;
default:
break;
}
OZ(proxy_->to(exec_svr)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(wait_timeout)
.ap_ping_sql_task(ping_task, &task_cb),
exec_svr,
tenant_id,
wait_timeout,
ping_task);
return ret;
}
int ObExecutorRpcImpl::task_execute(ObExecutorRpcCtx& rpc_ctx, ObTask& task, const common::ObAddr& svr,
RemoteExecuteStreamHandle& handler, bool& has_sent_task, bool& has_transfer_err)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
has_sent_task = false;
has_transfer_err = false;
handler.set_task_id(task.get_ob_task_id());
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
RemoteStreamHandle& real_handler = handler.get_remote_stream_handle();
RemoteStreamHandle::MyHandle& h = real_handler.get_handle();
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_) || OB_ISNULL(real_handler.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("not init", K(ret), K_(proxy), "result", real_handler.get_result());
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
obrpc::ObExecutorRpcProxy to_proxy = proxy_->to(svr);
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (FALSE_IT(has_sent_task = true)) {
} else if (OB_FAIL(to_proxy.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.task_execute(task, *real_handler.get_result(), h))) {
LOG_WARN("rpc task_execute fail", K(ret), K(tenant_id), K(svr), K(timeout), K(timeout_timestamp));
// rcode.rcode_ will be set in ObRpcProcessor<T>::part_response() of remote server,
// and return to local server from remote server. so:
// 1. if we get OB_SUCCESS from rcode.rcode_ here, transfer process must has error,
// such as network error or crash of remote server.
// 2. if we get some error from rcode.rcode_ here, transfer process must has no error,
// otherwise we can not get rcode.rcode_ from remote server.
const obrpc::ObRpcResultCode& rcode = to_proxy.get_result_code();
if (OB_LIKELY(OB_SUCCESS != rcode.rcode_)) {
FORWARD_USER_ERROR(rcode.rcode_, rcode.msg_);
} else {
has_transfer_err = true;
}
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
handler.set_result_code(ret);
return ret;
}
int ObExecutorRpcImpl::task_execute_v2(ObExecutorRpcCtx& rpc_ctx, ObRemoteTask& task, const common::ObAddr& svr,
RemoteExecuteStreamHandle& handler, bool& has_sent_task, bool& has_transfer_err)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
has_sent_task = false;
has_transfer_err = false;
handler.set_task_id(task.get_task_id());
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
RemoteStreamHandleV2& real_handler = handler.get_remote_stream_handle_v2();
RemoteStreamHandleV2::MyHandle& h = real_handler.get_handle();
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_) || OB_ISNULL(real_handler.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("not init", K(ret), K_(proxy), "result", real_handler.get_result());
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
obrpc::ObExecutorRpcProxy to_proxy = proxy_->to(svr);
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (FALSE_IT(has_sent_task = true)) {
} else if (OB_FAIL(to_proxy.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.remote_task_execute(task, *real_handler.get_result(), h))) {
LOG_WARN("rpc task_execute fail", K(ret), K(tenant_id), K(svr), K(timeout), K(timeout_timestamp));
// rcode.rcode_ will be set in ObRpcProcessor<T>::part_response() of remote server,
// and return to local server from remote server. so:
// 1. if we get OB_SUCCESS from rcode.rcode_ here, transfer process must has error,
// such as network error or crash of remote server.
// 2. if we get some error from rcode.rcode_ here, transfer process must has no error,
// otherwise we can not get rcode.rcode_ from remote server.
const obrpc::ObRpcResultCode& rcode = to_proxy.get_result_code();
if (OB_LIKELY(OB_SUCCESS != rcode.rcode_)) {
FORWARD_USER_ERROR(rcode.rcode_, rcode.msg_);
} else {
has_transfer_err = true;
}
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
handler.set_result_code(ret);
return ret;
}
int ObExecutorRpcImpl::remote_task_submit(
ObExecutorRpcCtx& rpc_ctx, ObRemoteTask& task, const ObAddr& svr, bool& has_sent_task)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
ObCurTraceId::TraceId* cur_trace_id = NULL;
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
has_sent_task = false;
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret), K(proxy_));
} else if (OB_ISNULL(cur_trace_id = ObCurTraceId::get_trace_id())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("current trace id is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else {
has_sent_task = true;
if (OB_FAIL(
proxy_->to(svr).by(tenant_id).as(OB_SYS_TENANT_ID).timeout(timeout).remote_task_submit(task, nullptr))) {
LOG_WARN("rpc task_submit fail", K(ret), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
}
if (OB_FAIL(ret)) {
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
}
return ret;
}
int ObExecutorRpcImpl::remote_post_result_async(
ObExecutorRpcCtx& rpc_ctx, ObRemoteResult& remote_result, const ObAddr& svr, bool& has_sent_result)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
ObCurTraceId::TraceId* cur_trace_id = NULL;
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
has_sent_result = false;
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret), K(proxy_));
} else if (OB_ISNULL(cur_trace_id = ObCurTraceId::get_trace_id())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("current trace id is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else {
has_sent_result = true;
if (OB_FAIL(proxy_->to(svr)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.remote_post_result(remote_result, nullptr))) {
LOG_WARN("rpc task_submit fail", K(ret), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
}
}
}
return ret;
}
int ObExecutorRpcImpl::task_submit(
ObExecutorRpcCtx& rpc_ctx, ObTask& task, const common::ObAddr& svr, const TransResult* trans_result) const
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
ObCurTraceId::TraceId* cur_trace_id = NULL;
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret), K(proxy_));
} else if (OB_ISNULL(trans_result)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("trans_result is NULL", K(ret));
} else if (OB_UNLIKELY(!rpc_ctx.min_cluster_version_is_valid())) { // only local execution
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("min cluster version is invalid", K(ret), K(rpc_ctx.get_min_cluster_version()));
} else if (OB_ISNULL(cur_trace_id = ObCurTraceId::get_trace_id())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("current trace id is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else {
ObRpcAPDistExecuteCB task_submit_cb(
task.get_runner_server(), task.get_ob_task_id(), *cur_trace_id, timeout_timestamp);
task.set_max_sql_no(trans_result->get_max_sql_no());
if (OB_FAIL(proxy_->to(svr)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.ap_task_submit(task, &task_submit_cb))) {
LOG_WARN("rpc task_submit fail", K(ret), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
}
if (OB_FAIL(ret)) {
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
}
return ret;
}
/*
* kill task and wait
* */
int ObExecutorRpcImpl::task_kill(ObExecutorRpcCtx& rpc_ctx, const ObTaskID& task_id, const common::ObAddr& svr)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (OB_FAIL(proxy_->to(svr).by(tenant_id).as(OB_SYS_TENANT_ID).timeout(timeout).task_kill(task_id))) {
LOG_WARN("rpc task_kill fail", K(ret), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
return ret;
}
/*
* task complete, wakeup scheduler
* */
int ObExecutorRpcImpl::task_complete(ObExecutorRpcCtx& rpc_ctx, ObTaskCompleteEvent& task, const common::ObAddr& svr)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (OB_FAIL(proxy_->to(svr).by(tenant_id).as(OB_SYS_TENANT_ID).timeout(timeout).task_complete(task))) {
LOG_WARN("rpc task_complete fail", K(ret), K(tenant_id), K(svr), K(timeout), K(timeout_timestamp));
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
return ret;
}
int ObExecutorRpcImpl::task_notify_fetch(ObExecutorRpcCtx& rpc_ctx, ObTaskEvent& task_event, const common::ObAddr& svr)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
proxy_->set_server(svr);
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (OB_FAIL(
proxy_->to(svr).by(tenant_id).as(OB_SYS_TENANT_ID).timeout(timeout).task_notify_fetch(task_event))) {
LOG_WARN("rpc task_notify_fetch fail", K(ret), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
return ret;
}
int ObExecutorRpcImpl::task_fetch_result(ObExecutorRpcCtx& rpc_ctx, const ObSliceID& ob_slice_id,
const common::ObAddr& svr, FetchResultStreamHandle& handler)
{
handler.set_task_id(ob_slice_id.get_ob_task_id());
OB_ASSERT(NULL != handler.get_result());
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (OB_FAIL(proxy_->to(svr)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.task_fetch_result(ob_slice_id, *handler.get_result(), handler.get_handle()))) {
LOG_WARN(
"rpc task_notify_fetch fail", K(ret), K(ob_slice_id), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
handler.set_result_code(ret);
return ret;
}
int ObExecutorRpcImpl::task_fetch_interm_result(ObExecutorRpcCtx& rpc_ctx, const ObSliceID& ob_slice_id,
const common::ObAddr& svr, FetchIntermResultStreamHandle& handler)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_) || OB_ISNULL(handler.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy or handler result is null", K(ret), K(proxy_));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (OB_FAIL(proxy_->to(svr)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.task_fetch_interm_result(ob_slice_id, *handler.get_result(), handler.get_handle()))) {
LOG_WARN(
"rpc task_notify_fetch fail", K(ret), K(ob_slice_id), K(svr), K(tenant_id), K(timeout), K(timeout_timestamp));
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
handler.set_result_code(ret);
return ret;
}
int ObExecutorRpcImpl::fetch_interm_result_item(ObExecutorRpcCtx& rpc_ctx, const common::ObAddr& dst,
const ObSliceID& slice_id, const int64_t fetch_index, ObFetchIntermResultItemRes& res)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
ObFetchIntermResultItemArg arg;
arg.slice_id_ = slice_id;
arg.index_ = fetch_index;
const int64_t timeout = rpc_ctx.get_timeout_timestamp() - ObTimeUtility::current_time();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy is null", K(ret), K(proxy_));
} else if (timeout < 0) {
ret = OB_TIMEOUT;
LOG_WARN("already timeout", K(ret), K(dst), K(rpc_ctx.get_timeout_timestamp()));
} else if (OB_FAIL(proxy_->to(dst)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.fetch_interm_result_item(arg, res))) {
LOG_WARN("fetch interm result interm failed", K(ret), K(dst), K(arg));
}
return ret;
}
int ObExecutorRpcImpl::close_result(ObExecutorRpcCtx& rpc_ctx, ObSliceID& ob_slice_id, const common::ObAddr& svr)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = rpc_ctx.get_rpc_tenant_id();
if (THIS_WORKER.get_rpc_tenant() > 0) {
tenant_id = THIS_WORKER.get_rpc_tenant();
}
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
if (OB_ISNULL(proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("proxy_ is NULL", K(ret));
} else {
int64_t timeout = timeout_timestamp - ::oceanbase::common::ObTimeUtility::current_time();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("task_execute timeout before rpc", K(ret), K(svr), K(timeout), K(timeout_timestamp));
} else if (OB_FAIL(proxy_->to(svr)
.by(tenant_id)
.as(OB_SYS_TENANT_ID)
.timeout(timeout)
.close_result(ob_slice_id, NULL))) {
LOG_WARN("rpc close_result fail", K(ret), K(svr), K(tenant_id), K(ob_slice_id), K(timeout), K(timeout_timestamp));
deal_with_rpc_timeout_err(rpc_ctx, ret, svr);
}
}
return ret;
}
void ObExecutorRpcImpl::deal_with_rpc_timeout_err(ObExecutorRpcCtx& rpc_ctx, int& err, const ObAddr& dist_server) const
{
if (OB_TIMEOUT == err) {
int64_t timeout_timestamp = rpc_ctx.get_timeout_timestamp();
int64_t cur_timestamp = ::oceanbase::common::ObTimeUtility::current_time();
if (timeout_timestamp - cur_timestamp > 0) {
LOG_DEBUG("rpc return OB_TIMEOUT, but it is actually not timeout, "
"change error code to OB_CONNECT_ERROR",
K(err),
K(timeout_timestamp),
K(cur_timestamp));
ObQueryRetryInfo* retry_info = rpc_ctx.get_retry_info_for_update();
if (NULL != retry_info) {
int a_ret = OB_SUCCESS;
if (OB_UNLIKELY(OB_SUCCESS != (a_ret = retry_info->add_invalid_server_distinctly(dist_server)))) {
LOG_WARN("fail to add invalid server distinctly", K(a_ret), K(dist_server));
} else {
// LOG_INFO("YZFDEBUG add invalid server distinctly", K(a_ret), K(dist_server), "p",
// &retry_info->get_invalid_servers());
}
}
err = OB_RPC_CONNECT_ERROR;
} else {
LOG_DEBUG("rpc return OB_TIMEOUT, and it is actually timeout, "
"do not change error code",
K(err),
K(timeout_timestamp),
K(cur_timestamp));
ObQueryRetryInfo* retry_info = rpc_ctx.get_retry_info_for_update();
if (NULL != retry_info) {
retry_info->set_is_rpc_timeout(true);
}
}
}
}
int ObExecutorRpcCtx::check_status() const
{
int ret = OB_SUCCESS;
int64_t cur_timestamp = ::oceanbase::common::ObTimeUtility::current_time();
if (cur_timestamp > timeout_timestamp_) {
ret = OB_TIMEOUT;
LOG_WARN("query is timeout", K(cur_timestamp), K(timeout_timestamp_), K(ret));
} else if (OB_ISNULL(session_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is NULL", K(ret));
} else if (session_->is_terminate(ret)) {
LOG_WARN("execution was terminated", K(ret));
}
return ret;
}
int ObExecutorRpcImpl::get_sql_batch_req_type(int64_t execution_id) const
{
int type = 0;
if (execution_id & 0x1) {
type = obrpc::SQL_BATCH_REQ_NODELAY1;
} else {
type = obrpc::SQL_BATCH_REQ_NODELAY2;
}
return type;
}
int ObExecutorRpcImpl::remote_task_batch_submit(const uint64_t tenant_id, const ObAddr& server,
const int64_t cluster_id, const ObRemoteTask& task, bool& has_sent_task)
{
int ret = OB_SUCCESS;
const ObPartitionKey fake_pkey;
int batch_req_type = get_sql_batch_req_type(task.get_task_id().get_execution_id());
if (OB_ISNULL(batch_rpc_)) {
ret = OB_NOT_INIT;
LOG_WARN("executor rpc not init", K(ret));
} else if (OB_FAIL(batch_rpc_->post(
tenant_id, server, cluster_id, batch_req_type, OB_SQL_REMOTE_TASK_TYPE, fake_pkey, task))) {
LOG_WARN("post batch rpc failed", K(ret));
} else {
has_sent_task = true;
}
return ret;
}
int ObExecutorRpcImpl::remote_batch_post_result(const uint64_t tenant_id, const common::ObAddr& server,
const int64_t cluster_id, const ObRemoteResult& result, bool& has_sent_result)
{
int ret = OB_SUCCESS;
const ObPartitionKey fake_pkey;
int batch_req_type = get_sql_batch_req_type(result.get_task_id().get_execution_id());
if (OB_ISNULL(batch_rpc_)) {
ret = OB_NOT_INIT;
LOG_WARN("executor rpc not init", K(ret));
} else if (OB_FAIL(batch_rpc_->post(
tenant_id, server, cluster_id, batch_req_type, OB_SQL_REMOTE_RESULT_TYPE, fake_pkey, result))) {
LOG_WARN("post batch rpc failed", K(ret));
} else {
has_sent_result = true;
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,506 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_RPC_IMPL_
#define OCEANBASE_SQL_EXECUTOR_RPC_IMPL_
#include "share/ob_define.h"
#include "lib/container/ob_array.h"
#include "lib/allocator/ob_allocator.h"
#include "rpc/obrpc/ob_rpc_proxy.h"
#include "share/ob_scanner.h"
#include "share/rpc/ob_batch_rpc.h"
#include "sql/executor/ob_task.h"
#include "sql/executor/ob_task_info.h"
#include "sql/executor/ob_slice_id.h"
#include "sql/executor/ob_executor_rpc_proxy.h"
namespace oceanbase {
namespace obrpc {
class ObExecutorRpcProxy;
}
namespace sql {
class ObQueryRetryInfo;
class ObAPMiniTaskMgr;
/* The life cycle of MySteamHandler is the entire SQL statement,
* the same as ObResultSet, essentially a variable on the stack. reference:obmp_query.cpp */
template <obrpc::ObRpcPacketCode pcode>
class MyStreamHandle {
public:
typedef typename obrpc::ObExecutorRpcProxy::SSHandle<pcode> MyHandle;
typedef common::ObScanner MyResult;
explicit MyStreamHandle(const char* label) : result_(label), rc_(common::OB_SUCCESS)
{}
virtual ~MyStreamHandle()
{}
void reset()
{
result_.reset();
rc_ = common::OB_SUCCESS;
}
const common::ObAddr& get_dst_addr() const
{
return handle_.get_dst_addr();
}
MyHandle& get_handle()
{
return handle_;
}
MyResult* get_result()
{
MyResult* ret_result = NULL;
if (!result_.is_inited()) {
SQL_EXE_LOG(ERROR, "result_ is not inited");
} else {
ret_result = &result_;
}
return ret_result;
}
int reset_and_init_result()
{
int ret = common::OB_SUCCESS;
result_.reset();
if (!result_.is_inited() && OB_FAIL(result_.init())) {
SQL_EXE_LOG(WARN, "fail to init result", K(ret));
}
return ret;
}
void set_result_code(int code)
{
rc_ = code;
}
int get_result_code()
{
return rc_;
}
void set_task_id(const ObTaskID& task_id)
{
task_id_ = task_id;
}
const ObTaskID& get_task_id() const
{
return task_id_;
}
private:
ObTaskID task_id_;
MyHandle handle_;
MyResult result_;
int rc_;
};
template <obrpc::ObRpcPacketCode pcode>
class MySSHandle {
public:
typedef typename obrpc::ObExecutorRpcProxy::SSHandle<pcode> MyHandle;
typedef ObIntermResultItem MyResult;
explicit MySSHandle(const char* label) : result_(label), rc_(common::OB_SUCCESS)
{}
virtual ~MySSHandle()
{}
void reset()
{
result_.reset();
rc_ = common::OB_SUCCESS;
}
MyHandle& get_handle()
{
return handle_;
}
MyResult* get_result()
{
MyResult* ret_result = NULL;
if (!result_.is_valid()) {
SQL_EXE_LOG(ERROR, "result_ is not valid", K(result_));
} else {
ret_result = &result_;
}
return ret_result;
}
int reset_result()
{
int ret = common::OB_SUCCESS;
result_.reset();
if (OB_UNLIKELY(!result_.is_valid())) {
ret = OB_ERR_UNEXPECTED;
SQL_EXE_LOG(ERROR, "after reset, result_ is invalid", K(ret), K(result_));
}
return ret;
}
void set_result_code(int code)
{
rc_ = code;
}
int get_result_code()
{
return rc_;
}
private:
MyHandle handle_;
MyResult result_;
int rc_;
};
typedef MyStreamHandle<obrpc::OB_REMOTE_EXECUTE> RemoteStreamHandle;
typedef MyStreamHandle<obrpc::OB_REMOTE_SYNC_EXECUTE> RemoteStreamHandleV2;
typedef MyStreamHandle<obrpc::OB_TASK_FETCH_RESULT> FetchResultStreamHandle;
typedef MySSHandle<obrpc::OB_TASK_FETCH_INTERM_RESULT> FetchIntermResultStreamHandle;
class RemoteExecuteStreamHandle {
public:
RemoteExecuteStreamHandle(const char* label)
: use_remote_protocol_v2_(false), sync_stream_handle_(label), sync_stream_handle_v2_(label)
{}
~RemoteExecuteStreamHandle() = default;
void set_use_remote_protocol_v2()
{
use_remote_protocol_v2_ = true;
}
void reset()
{
if (use_remote_protocol_v2_) {
sync_stream_handle_v2_.reset();
} else {
sync_stream_handle_.reset();
}
}
const common::ObAddr& get_dst_addr() const
{
const common::ObAddr* dst_addr = nullptr;
if (use_remote_protocol_v2_) {
dst_addr = &sync_stream_handle_v2_.get_dst_addr();
} else {
dst_addr = &sync_stream_handle_.get_dst_addr();
}
return *dst_addr;
}
int reset_and_init_result()
{
int ret = common::OB_SUCCESS;
if (use_remote_protocol_v2_) {
ret = sync_stream_handle_v2_.reset_and_init_result();
} else {
ret = sync_stream_handle_.reset_and_init_result();
}
return ret;
}
void set_result_code(int code)
{
if (use_remote_protocol_v2_) {
sync_stream_handle_v2_.set_result_code(code);
} else {
sync_stream_handle_.set_result_code(code);
}
}
int get_result_code()
{
int ret = common::OB_SUCCESS;
if (use_remote_protocol_v2_) {
ret = sync_stream_handle_v2_.get_result_code();
} else {
ret = sync_stream_handle_.get_result_code();
}
return ret;
}
void set_task_id(const ObTaskID& task_id)
{
if (use_remote_protocol_v2_) {
sync_stream_handle_v2_.set_task_id(task_id);
} else {
sync_stream_handle_.set_task_id(task_id);
}
}
const ObTaskID& get_task_id() const
{
const ObTaskID* task_id = nullptr;
if (use_remote_protocol_v2_) {
task_id = &(sync_stream_handle_v2_.get_task_id());
} else {
task_id = &(sync_stream_handle_.get_task_id());
}
return *task_id;
}
common::ObScanner* get_result()
{
common::ObScanner* result = nullptr;
if (use_remote_protocol_v2_) {
result = sync_stream_handle_v2_.get_result();
} else {
result = sync_stream_handle_.get_result();
}
return result;
}
bool has_more()
{
bool bret = false;
if (use_remote_protocol_v2_) {
bret = sync_stream_handle_v2_.get_handle().has_more();
} else {
bret = sync_stream_handle_.get_handle().has_more();
}
return bret;
}
int abort()
{
int ret = common::OB_SUCCESS;
if (use_remote_protocol_v2_) {
ret = sync_stream_handle_v2_.get_handle().abort();
} else {
ret = sync_stream_handle_.get_handle().abort();
}
return ret;
}
int get_more(common::ObScanner& result)
{
int ret = common::OB_SUCCESS;
if (use_remote_protocol_v2_) {
ret = sync_stream_handle_v2_.get_handle().get_more(result);
} else {
ret = sync_stream_handle_.get_handle().get_more(result);
}
return ret;
}
RemoteStreamHandle& get_remote_stream_handle()
{
return sync_stream_handle_;
}
RemoteStreamHandleV2& get_remote_stream_handle_v2()
{
return sync_stream_handle_v2_;
}
private:
bool use_remote_protocol_v2_;
RemoteStreamHandle sync_stream_handle_;
RemoteStreamHandleV2 sync_stream_handle_v2_;
};
class ObExecutorRpcCtx {
public:
static const uint64_t INVALID_CLUSTER_VERSION = 0;
public:
ObExecutorRpcCtx(uint64_t rpc_tenant_id, int64_t timeout_timestamp, uint64_t min_cluster_version,
ObQueryRetryInfo* retry_info, ObSQLSessionInfo* session, bool is_plain_select,
ObAPMiniTaskMgr* ap_mini_task_mgr = NULL)
: rpc_tenant_id_(rpc_tenant_id),
timeout_timestamp_(timeout_timestamp),
min_cluster_version_(min_cluster_version),
retry_info_(retry_info),
session_(session),
is_plain_select_(is_plain_select),
ap_mini_task_mgr_(ap_mini_task_mgr)
{}
~ObExecutorRpcCtx()
{}
uint64_t get_rpc_tenant_id() const
{
return rpc_tenant_id_;
}
inline int64_t get_timeout_timestamp() const
{
return timeout_timestamp_;
}
// equal to INVALID_CLUSTER_VERSION, means this description is serialized from the old observer on the remote
inline bool min_cluster_version_is_valid() const
{
return INVALID_CLUSTER_VERSION != min_cluster_version_;
}
inline ObAPMiniTaskMgr* get_ap_mini_task_mgr()
{
return ap_mini_task_mgr_;
}
inline uint64_t get_min_cluster_version() const
{
return min_cluster_version_;
}
inline const ObQueryRetryInfo* get_retry_info() const
{
return retry_info_;
}
inline ObQueryRetryInfo* get_retry_info_for_update() const
{
return retry_info_;
}
bool is_retry_for_rpc_timeout() const
{
return is_plain_select_;
}
int check_status() const;
TO_STRING_KV(K_(rpc_tenant_id), K_(timeout_timestamp), K_(min_cluster_version), K_(retry_info), K_(is_plain_select));
private:
uint64_t rpc_tenant_id_;
int64_t timeout_timestamp_;
uint64_t min_cluster_version_;
// retry_info_ == NULL means that this rpc does not need to give feedback to the retry module
ObQueryRetryInfo* retry_info_;
const ObSQLSessionInfo* session_;
bool is_plain_select_; // stmt_type == T_SELECT && not select...for update
ObAPMiniTaskMgr* ap_mini_task_mgr_;
private:
DISALLOW_COPY_AND_ASSIGN(ObExecutorRpcCtx);
};
class ObDistributedSchedulerManager;
class ObExecutorPingRpcCtx {
public:
ObExecutorPingRpcCtx(uint64_t rpc_tenant_id, int64_t wait_timeout, ObDistributedSchedulerManager* dist_task_mgr,
ObAPMiniTaskMgr* mini_task_mgr)
: rpc_tenant_id_(rpc_tenant_id),
wait_timeout_(wait_timeout),
dist_task_mgr_(dist_task_mgr),
mini_task_mgr_(mini_task_mgr)
{}
~ObExecutorPingRpcCtx()
{}
uint64_t get_rpc_tenant_id() const
{
return rpc_tenant_id_;
}
int64_t get_wait_timeout() const
{
return wait_timeout_;
}
ObDistributedSchedulerManager* get_dist_task_mgr()
{
return dist_task_mgr_;
}
ObAPMiniTaskMgr* get_mini_task_mgr()
{
return mini_task_mgr_;
}
private:
uint64_t rpc_tenant_id_;
int64_t wait_timeout_;
ObDistributedSchedulerManager* dist_task_mgr_;
ObAPMiniTaskMgr* mini_task_mgr_;
private:
DISALLOW_COPY_AND_ASSIGN(ObExecutorPingRpcCtx);
};
#define OB_SQL_REMOTE_TASK_TYPE 1
#define OB_SQL_REMOTE_RESULT_TYPE 2
// All calls to rpc must use the to function to support concurrent calls
class ObExecutorRpcImpl {
public:
ObExecutorRpcImpl() : proxy_(NULL), batch_rpc_(nullptr)
{}
virtual ~ObExecutorRpcImpl()
{}
/*
* set rpc proxy
* */
int init(obrpc::ObExecutorRpcProxy* rpc_proxy, obrpc::ObBatchRpc* batch_rpc);
/*
* Submit an asynchronous task execution request,
* and receive the result data driven by the OB_TASK_NOTIFY_FETCH message
*/
virtual int task_submit(
ObExecutorRpcCtx& rpc_ctx, ObTask& task, const common::ObAddr& svr, const TransResult* trans_result) const;
/*
* Send a task and block and wait until the peer returns to the execution state.
* Save the execution handle in the handler, and then receive data through the handler
* */
virtual int task_execute(ObExecutorRpcCtx& rpc_ctx, ObTask& task, const common::ObAddr& svr,
RemoteExecuteStreamHandle& handler, bool& has_sent_task, bool& has_transfer_err);
virtual int task_execute_v2(ObExecutorRpcCtx& rpc_ctx, ObRemoteTask& task, const common::ObAddr& svr,
RemoteExecuteStreamHandle& handler, bool& has_sent_task, bool& has_transfer_err);
virtual int remote_task_submit(
ObExecutorRpcCtx& rpc_ctx, ObRemoteTask& task, const common::ObAddr& svr, bool& has_sent_task);
virtual int remote_post_result_async(
ObExecutorRpcCtx& rpc_ctx, ObRemoteResult& remote_result, const common::ObAddr& svr, bool& has_sent_result);
int remote_task_batch_submit(const uint64_t tenant_id, const common::ObAddr& server, const int64_t cluster_id,
const ObRemoteTask& task, bool& has_sent_task);
int remote_batch_post_result(const uint64_t tenant_id, const common::ObAddr& server, const int64_t cluster_id,
const ObRemoteResult& result, bool& has_sent_result);
virtual int mini_task_execute(ObExecutorRpcCtx& rpc_ctx, ObMiniTask& task, ObMiniTaskResult& result);
virtual int mini_task_submit(ObExecutorRpcCtx& rpc_ctx, ObMiniTask& task);
virtual int ping_sql_task(ObExecutorPingRpcCtx& ping_rpc_ctx, ObPingSqlTask& task);
/*
* Send a command to kill a task and block waiting for the peer to return to the execution state
* */
virtual int task_kill(ObExecutorRpcCtx& rpc_ctx, const ObTaskID& task_id, const common::ObAddr& svr);
/*
* Task is executed on the Worker side,
* and the Scheduler is notified to start the Task to read the result
* */
virtual int task_complete(ObExecutorRpcCtx& rpc_ctx, ObTaskCompleteEvent& task_event, const common::ObAddr& svr);
/*
* Send the execution result of a task without waiting for return
* */
virtual int task_notify_fetch(ObExecutorRpcCtx& rpc_ctx, ObTaskEvent& task_event, const common::ObAddr& svr);
/*
* Get all the scanners of the intermediate result of a task,
* block and wait until all the scanners return
* */
virtual int task_fetch_result(ObExecutorRpcCtx& rpc_ctx, const ObSliceID& ob_slice_id, const common::ObAddr& svr,
FetchResultStreamHandle& handler);
/*
* Get all interm result items of the intermediate result of a task,
* block and wait until all items are returned
* */
virtual int task_fetch_interm_result(ObExecutorRpcCtx& rpc_ctx, const ObSliceID& ob_slice_id,
const common::ObAddr& svr, FetchIntermResultStreamHandle& handler);
/*
* Send a command to remove the intermediate result corresponding to an ObSliceID and
* block waiting for the peer to return to the execution state
* */
virtual int close_result(ObExecutorRpcCtx& rpc_ctx, ObSliceID& ob_slice_id, const common::ObAddr& svr);
// Fetch interm result by slice_id and interm result item index,
// return interm result item and total interm result item count of this slice.
virtual int fetch_interm_result_item(ObExecutorRpcCtx& rpc_ctx, const common::ObAddr& dst,
const ObSliceID& ob_slice_id, const int64_t fetch_index, ObFetchIntermResultItemRes& res);
obrpc::ObExecutorRpcProxy* get_proxy()
{
return proxy_;
}
private:
void deal_with_rpc_timeout_err(ObExecutorRpcCtx& rpc_ctx, int& err, const common::ObAddr& dist_server) const;
int get_sql_batch_req_type(int64_t execution_id) const;
private:
/* functions */
/* variables */
obrpc::ObExecutorRpcProxy* proxy_;
obrpc::ObBatchRpc* batch_rpc_;
/* other */
DISALLOW_COPY_AND_ASSIGN(ObExecutorRpcImpl);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_RPC_IMPL_ */
//// end of header file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,549 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_RPC_PROCESSOR_
#define OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_RPC_PROCESSOR_
#include "observer/virtual_table/ob_virtual_table_iterator_factory.h"
#include "observer/ob_server_struct.h"
#include "rpc/obrpc/ob_rpc_proxy.h"
#include "rpc/obrpc/ob_rpc_processor.h"
#include "sql/executor/ob_executor_rpc_proxy.h"
#include "sql/monitor/ob_phy_plan_monitor_info.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_des_exec_context.h"
#include "sql/ob_sql_trans_control.h"
#include "share/schema/ob_schema_getter_guard.h"
#define OB_DEFINE_SQL_PROCESSOR(cls, pcode, pname) \
class pname : public obrpc::ObRpcProcessor<obrpc::Ob##cls##RpcProxy::ObRpc<pcode>>
#define OB_DEFINE_SQL_TASK_PROCESSOR(cls, pcode, pname) \
OB_DEFINE_SQL_PROCESSOR(cls, obrpc::pcode, pname) \
{ \
public: \
pname(const observer::ObGlobalContext& gctx) : gctx_(gctx) exec_ctx_(), phy_plan_() \
{} \
virturl ~pname() \
{} \
virtual int init(); \
\
protected: \
virtual int process(); \
\
private: \
const observer::ObGlobalContext& gctx_; \
sql::ObExecContext& exec_ctx_; \
sql::ObPhysicalPlan& phy_plan_; \
}
#define OB_DEFINE_SQL_CMD_PROCESSOR(cls, pcode, pname) \
OB_DEFINE_SQL_PROCESSOR(cls, obrpc::pcode, pname) \
{ \
public: \
pname(const observer::ObGlobalContext& gctx) : gctx_(gctx) \
{} \
virtual ~pname() \
{} \
\
protected: \
int process(); \
int preprocess_arg(); \
\
private: \
const observer::ObGlobalContext& gctx_; \
common::ObArenaAllocator alloc_; \
}
namespace oceanbase {
namespace observer {
class ObGlobalContext;
}
namespace share {
namespace schema {}
} // namespace share
namespace sql {
class ObExecContext;
class ObPhysicalPlan;
class ObIntermResultManager;
class ObIntermResultIterator;
class ObWorkerSessionGuard {
public:
ObWorkerSessionGuard(ObSQLSessionInfo* session);
~ObWorkerSessionGuard();
};
class ObDistExecuteBaseP {
public:
ObDistExecuteBaseP(const observer::ObGlobalContext& gctx, bool sync)
: gctx_(gctx),
exec_ctx_(gctx.session_mgr_),
vt_iter_factory_(*gctx_.vt_iter_creator_),
phy_plan_(),
sql_ctx_(),
trans_state_(),
exec_record_(),
process_timestamp_(0),
exec_start_timestamp_(0),
exec_end_timestamp_(0),
sync_(sync),
partition_location_cache_()
{}
virtual ~ObDistExecuteBaseP()
{}
virtual int init(ObTask& task);
int64_t get_exec_start_timestamp() const
{
return exec_start_timestamp_;
}
int64_t get_exec_end_timestamp() const
{
return exec_end_timestamp_;
}
int64_t get_process_timestamp() const
{
return process_timestamp_;
}
int64_t get_single_process_timestamp() const
{
return exec_start_timestamp_;
}
ObDesExecContext& get_exec_ctx()
{
return exec_ctx_;
}
protected:
virtual int param_preprocess(ObTask& task);
virtual int execute_dist_plan(ObTask& task, ObTaskCompleteEvent& task_event);
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp) = 0;
private:
int get_participants(common::ObPartitionIArray& participants, const ObTask& task);
private:
const observer::ObGlobalContext& gctx_;
sql::ObDesExecContext exec_ctx_;
observer::ObVirtualTableIteratorFactory vt_iter_factory_;
sql::ObPhysicalPlan phy_plan_;
sql::ObSqlCtx sql_ctx_;
share::schema::ObSchemaGetterGuard schema_guard_;
TransState trans_state_;
ObExecRecord exec_record_;
int64_t process_timestamp_;
int64_t exec_start_timestamp_;
int64_t exec_end_timestamp_;
bool sync_;
/* partition cache for global index lookup*/
ObSqlPartitionLocationCache partition_location_cache_;
};
// This would not used after cluster version upgrade to 1.3.0
// Remain this class, only for compatibility
class ObRpcDistExecuteP : public ObDistExecuteBaseP,
public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_DIST_EXECUTE>> {
public:
ObRpcDistExecuteP(const observer::ObGlobalContext& gctx) : ObDistExecuteBaseP(gctx, true)
{
set_preserve_recv_data();
}
virtual ~ObRpcDistExecuteP()
{}
virtual int init();
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp)
{
ObExecStatUtils::record_exec_timestamp(*this, is_first, exec_timestamp);
}
protected:
virtual int before_process();
virtual int process();
virtual int after_process();
virtual void cleanup();
};
class ObRpcAPDistExecuteP : public ObDistExecuteBaseP,
public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_AP_DIST_EXECUTE>> {
public:
ObRpcAPDistExecuteP(const observer::ObGlobalContext& gctx) : ObDistExecuteBaseP(gctx, false)
{
set_preserve_recv_data();
}
virtual ~ObRpcAPDistExecuteP()
{}
virtual int init();
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp)
{
ObExecStatUtils::record_exec_timestamp(*this, is_first, exec_timestamp);
}
protected:
virtual int before_process();
virtual int process();
virtual int after_process();
virtual int before_response();
virtual void cleanup();
};
class ObRpcAPDistExecuteCB : public obrpc::ObExecutorRpcProxy::AsyncCB<obrpc::OB_AP_DIST_EXECUTE> {
public:
ObRpcAPDistExecuteCB(const common::ObAddr& server, const ObTaskID& ob_task_id, const ObCurTraceId::TraceId& trace_id,
int64_t timeout_ts)
: task_loc_(server, ob_task_id), timeout_ts_(timeout_ts)
{
trace_id_.set(trace_id);
}
virtual ~ObRpcAPDistExecuteCB()
{}
public:
virtual int process();
virtual void on_invalid()
{
free_my_memory();
}
virtual void on_timeout();
rpc::frame::ObReqTransport::AsyncCB* clone(const rpc::frame::SPAlloc& alloc) const
{
void* buf = alloc(sizeof(*this));
rpc::frame::ObReqTransport::AsyncCB* newcb = NULL;
if (NULL != buf) {
newcb =
new (buf) ObRpcAPDistExecuteCB(task_loc_.get_server(), task_loc_.get_ob_task_id(), trace_id_, timeout_ts_);
}
return newcb;
}
void set_args(const ObTask& arg)
{
UNUSED(arg);
}
private:
void free_my_memory()
{
result_.reset();
}
private:
ObTaskLocation task_loc_;
common::ObCurTraceId::TraceId trace_id_;
int64_t timeout_ts_;
DISALLOW_COPY_AND_ASSIGN(ObRpcAPDistExecuteCB);
};
class ObMiniTaskBaseP {
public:
ObMiniTaskBaseP(const observer::ObGlobalContext& gctx)
: gctx_(gctx),
exec_ctx_(gctx.session_mgr_),
vt_iter_factory_(*gctx_.vt_iter_creator_),
phy_plan_(),
sql_ctx_(),
trans_state_(),
is_rollback_(false),
process_timestamp_(0),
exec_start_timestamp_(0),
exec_end_timestamp_(0)
{}
virtual ~ObMiniTaskBaseP()
{}
int init_task(ObMiniTask& task);
int64_t get_exec_start_timestamp() const
{
return exec_start_timestamp_;
}
int64_t get_exec_end_timestamp() const
{
return exec_end_timestamp_;
}
int64_t get_process_timestamp() const
{
return process_timestamp_;
}
int64_t get_single_process_timestamp() const
{
return exec_start_timestamp_;
}
protected:
int prepare_task_env(ObMiniTask& task);
int execute_subplan(const ObOpSpec& root_spec, ObScanner& scanner);
int execute_mini_plan(ObMiniTask& task, ObMiniTaskResult& result);
int sync_send_result(ObExecContext& exec_ctx, const ObPhyOperator& op, common::ObScanner& scanner);
int sync_send_result(ObExecContext& exec_ctx, ObOperator& op, ObScanner& scanner);
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp) = 0;
protected:
const observer::ObGlobalContext& gctx_;
sql::ObDesExecContext exec_ctx_;
observer::ObVirtualTableIteratorFactory vt_iter_factory_;
sql::ObPhysicalPlan phy_plan_;
sql::ObSqlCtx sql_ctx_;
share::schema::ObSchemaGetterGuard schema_guard_;
TransState trans_state_;
bool is_rollback_;
int64_t process_timestamp_;
int64_t exec_start_timestamp_;
int64_t exec_end_timestamp_;
};
class ObRpcMiniTaskExecuteP
: public ObMiniTaskBaseP,
public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_MINI_TASK_EXECUTE>> {
public:
ObRpcMiniTaskExecuteP(const observer::ObGlobalContext& gctx) : ObMiniTaskBaseP(gctx)
{
set_preserve_recv_data();
}
virtual int init();
protected:
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp)
{
ObExecStatUtils::record_exec_timestamp(*this, is_first, exec_timestamp);
}
protected:
virtual int before_process();
virtual int process();
virtual int before_response();
virtual int after_process()
{
return common::OB_SUCCESS;
}
virtual void cleanup();
};
class ObRpcAPMiniDistExecuteP
: public ObMiniTaskBaseP,
public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_AP_MINI_DIST_EXECUTE>> {
public:
ObRpcAPMiniDistExecuteP(const observer::ObGlobalContext& gctx) : ObMiniTaskBaseP(gctx)
{
set_preserve_recv_data();
}
virtual int init();
protected:
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp)
{
ObExecStatUtils::record_exec_timestamp(*this, is_first, exec_timestamp);
}
protected:
virtual int before_process();
virtual int process();
virtual int before_response();
virtual int after_process()
{
return common::OB_SUCCESS;
}
virtual void cleanup();
};
class ObRpcAPMiniDistExecuteCB : public obrpc::ObExecutorRpcProxy::AsyncCB<obrpc::OB_AP_MINI_DIST_EXECUTE> {
public:
ObRpcAPMiniDistExecuteCB(ObAPMiniTaskMgr* ap_mini_task_mgr, const ObTaskID& task_id,
const ObCurTraceId::TraceId& trace_id, const ObAddr& dist_server_, int64_t timeout_ts);
virtual ~ObRpcAPMiniDistExecuteCB()
{
free_my_memory();
}
public:
virtual int process();
virtual void on_invalid()
{
free_my_memory();
}
virtual void on_timeout();
rpc::frame::ObReqTransport::AsyncCB* clone(const rpc::frame::SPAlloc& alloc) const
{
void* buf = alloc(sizeof(*this));
rpc::frame::ObReqTransport::AsyncCB* newcb = NULL;
if (NULL != buf) {
newcb = new (buf) ObRpcAPMiniDistExecuteCB(ap_mini_task_mgr_, task_id_, trace_id_, dist_server_, timeout_ts_);
}
return newcb;
}
void set_args(const ObMiniTask& arg)
{
UNUSED(arg);
}
public:
static void deal_with_rpc_timeout_err(const int64_t timeout_ts, int& err);
private:
void free_my_memory();
private:
ObAPMiniTaskMgr* ap_mini_task_mgr_;
ObTaskID task_id_;
common::ObCurTraceId::TraceId trace_id_;
const ObAddr dist_server_;
int64_t timeout_ts_;
DISALLOW_COPY_AND_ASSIGN(ObRpcAPMiniDistExecuteCB);
};
class ObPingSqlTaskBaseP {
public:
ObPingSqlTaskBaseP(const observer::ObGlobalContext& gctx) : gctx_(gctx)
{}
virtual ~ObPingSqlTaskBaseP()
{}
protected:
int try_forbid_task(const ObPingSqlTask& ping_task, bool& forbid_succ);
int try_kill_task(const ObPingSqlTask& ping_task, bool& is_running);
protected:
const observer::ObGlobalContext& gctx_;
};
class ObRpcAPPingSqlTaskP : public ObPingSqlTaskBaseP,
public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_AP_PING_SQL_TASK>> {
public:
ObRpcAPPingSqlTaskP(const observer::ObGlobalContext& gctx) : ObPingSqlTaskBaseP(gctx)
{}
virtual ~ObRpcAPPingSqlTaskP()
{}
protected:
virtual int process();
};
class ObDistributedSchedulerManager;
class ObRpcAPPingSqlTaskCB : public obrpc::ObExecutorRpcProxy::AsyncCB<obrpc::OB_AP_PING_SQL_TASK> {
public:
ObRpcAPPingSqlTaskCB(const ObTaskID& task_id);
virtual ~ObRpcAPPingSqlTaskCB()
{
free_my_memory();
}
public:
int set_dist_task_mgr(ObDistributedSchedulerManager* dist_task_mgr);
int set_mini_task_mgr(ObAPMiniTaskMgr* mini_task_mgr);
virtual int process();
virtual void on_invalid()
{
free_my_memory();
}
virtual void on_timeout()
{
free_my_memory();
}
rpc::frame::ObReqTransport::AsyncCB* clone(const rpc::frame::SPAlloc& alloc) const
{
void* buf = alloc(sizeof(*this));
ObRpcAPPingSqlTaskCB* newcb = NULL;
if (NULL != buf) {
newcb = new (buf) ObRpcAPPingSqlTaskCB(task_id_);
}
if (NULL != newcb) {
switch (task_id_.get_task_type()) {
case ET_DIST_TASK:
newcb->set_dist_task_mgr(dist_task_mgr_);
break;
case ET_MINI_TASK:
newcb->set_mini_task_mgr(mini_task_mgr_);
break;
default:
break;
}
}
return newcb;
}
void set_args(const ObPingSqlTask& arg)
{
UNUSED(arg);
}
protected:
void free_my_memory();
protected:
ObTaskID task_id_;
ObDistributedSchedulerManager* dist_task_mgr_;
ObAPMiniTaskMgr* mini_task_mgr_;
DISALLOW_COPY_AND_ASSIGN(ObRpcAPPingSqlTaskCB);
};
class ObRpcTaskFetchResultP
: public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_TASK_FETCH_RESULT>> {
public:
ObRpcTaskFetchResultP(const observer::ObGlobalContext& gctx) : gctx_(gctx)
{
set_preserve_recv_data();
}
virtual ~ObRpcTaskFetchResultP()
{}
virtual int init();
protected:
virtual int process();
private:
int sync_send_result(ObIntermResultIterator& iter);
private:
const observer::ObGlobalContext& gctx_;
};
class ObRpcTaskFetchIntermResultP
: public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_TASK_FETCH_INTERM_RESULT>> {
public:
ObRpcTaskFetchIntermResultP(const observer::ObGlobalContext& gctx) : gctx_(gctx)
{
set_preserve_recv_data();
}
virtual ~ObRpcTaskFetchIntermResultP()
{}
// virtual int init();
protected:
virtual int process();
private:
int sync_send_result(ObIntermResultIterator& iter);
private:
const observer::ObGlobalContext& gctx_;
};
class ObRpcBKGDTaskCompleteP
: public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_BKGD_TASK_COMPLETE>> {
public:
ObRpcBKGDTaskCompleteP(const observer::ObGlobalContext&)
{
set_preserve_recv_data();
}
virtual int process();
static int notify_error(const ObTaskID& task_id, const uint64_t scheduler_id, const int return_code);
};
} // namespace sql
namespace sql {
OB_DEFINE_SQL_CMD_PROCESSOR(Executor, OB_TASK_COMPLETE, ObRpcTaskCompleteP);
OB_DEFINE_SQL_CMD_PROCESSOR(Executor, OB_TASK_NOTIFY_FETCH, ObRpcTaskNotifyFetchP);
OB_DEFINE_SQL_CMD_PROCESSOR(Executor, OB_TASK_KILL, ObRpcTaskKillP);
OB_DEFINE_SQL_CMD_PROCESSOR(Executor, OB_CLOSE_RESULT, ObRpcCloseResultP);
OB_DEFINE_SQL_CMD_PROCESSOR(Executor, OB_BKGD_DIST_EXECUTE, ObRpcBKGDDistExecuteP);
OB_DEFINE_SQL_CMD_PROCESSOR(Executor, OB_FETCH_INTERM_RESULT_ITEM, ObFetchIntermResultItemP);
OB_DEFINE_SQL_CMD_PROCESSOR(Executor, OB_CHECK_BUILD_INDEX_TASK_EXIST, ObCheckBuildIndexTaskExistP);
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_RPC_PROCESSOR_ */

View File

@ -0,0 +1,29 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include "sql/executor/ob_executor_rpc_proxy.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
OB_SERIALIZE_MEMBER(ObBKGDDistExecuteArg, return_addr_, tenant_id_, task_id_, scheduler_id_, serialized_task_);
OB_SERIALIZE_MEMBER(ObBKGDTaskCompleteArg, task_id_, scheduler_id_, return_code_, event_);
OB_SERIALIZE_MEMBER(ObFetchIntermResultItemArg, slice_id_, index_);
OB_SERIALIZE_MEMBER(ObFetchIntermResultItemRes, result_item_, total_item_cnt_);
} // namespace sql
namespace obrpc {} // namespace obrpc
} // namespace oceanbase

View File

@ -0,0 +1,127 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_RPC_PROXY_
#define OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_RPC_PROXY_
#include "rpc/obrpc/ob_rpc_proxy.h"
#include "sql/executor/ob_task.h"
#include "sql/executor/ob_task_event.h"
#include "share/config/ob_server_config.h"
#include "observer/ob_server_struct.h"
namespace oceanbase {
namespace sql {
class ObMiniTask;
class ObMiniTaskResult;
} // namespace sql
namespace sql {
struct ObBKGDDistExecuteArg {
OB_UNIS_VERSION(1);
public:
ObBKGDDistExecuteArg() : tenant_id_(OB_INVALID_ID), scheduler_id_(0)
{}
TO_STRING_KV(K_(tenant_id), K_(task_id), K_(scheduler_id), K_(return_addr), K(serialized_task_.length()));
bool is_valid() const
{
return OB_INVALID_ID != tenant_id_ && task_id_.is_valid() && scheduler_id_ > 0 && return_addr_.is_valid() &&
!serialized_task_.empty();
}
uint64_t tenant_id_;
ObTaskID task_id_;
uint64_t scheduler_id_;
common::ObAddr return_addr_;
common::ObString serialized_task_;
};
struct ObBKGDTaskCompleteArg {
OB_UNIS_VERSION(1);
public:
ObBKGDTaskCompleteArg() : scheduler_id_(0), return_code_(common::OB_SUCCESS)
{}
TO_STRING_KV(K_(task_id), K_(scheduler_id), K_(return_code), K_(event));
ObTaskID task_id_;
uint64_t scheduler_id_;
int return_code_;
ObTaskCompleteEvent event_;
};
struct ObFetchIntermResultItemArg {
OB_UNIS_VERSION(1);
public:
ObFetchIntermResultItemArg() : index_(OB_INVALID_INDEX)
{}
ObSliceID slice_id_;
int64_t index_;
TO_STRING_KV(K_(slice_id), K_(index));
};
struct ObFetchIntermResultItemRes {
OB_UNIS_VERSION(1);
public:
ObFetchIntermResultItemRes() : total_item_cnt_(-1)
{}
ObIntermResultItem result_item_;
int64_t total_item_cnt_;
TO_STRING_KV(K_(result_item), K_(total_item_cnt));
};
} // namespace sql
namespace obrpc {
class ObExecutorRpcProxy : public obrpc::ObRpcProxy {
public:
DEFINE_TO(ObExecutorRpcProxy);
RPC_SS(@PR5 task_execute, obrpc::OB_REMOTE_EXECUTE, (sql::ObTask), common::ObScanner);
RPC_SS(@PR5 remote_task_execute, obrpc::OB_REMOTE_SYNC_EXECUTE, (sql::ObRemoteTask), common::ObScanner);
RPC_SS(@PR5 task_fetch_result, obrpc::OB_TASK_FETCH_RESULT, (sql::ObSliceID), common::ObScanner);
RPC_SS(@PR5 task_fetch_interm_result, obrpc::OB_TASK_FETCH_INTERM_RESULT, (sql::ObSliceID), sql::ObIntermResultItem);
RPC_S(@PR4 fetch_interm_result_item, obrpc::OB_FETCH_INTERM_RESULT_ITEM, (sql::ObFetchIntermResultItemArg),
sql::ObFetchIntermResultItemRes);
// task_submit not used after cluser version upgrade to 1.3.0
// Remain this for compatibility
RPC_S(@PR5 task_submit, obrpc::OB_DIST_EXECUTE, (sql::ObTask));
RPC_S(@PR5 task_kill, obrpc::OB_TASK_KILL, (sql::ObTaskID));
RPC_S(@PR5 task_notify_fetch, obrpc::OB_TASK_NOTIFY_FETCH, (sql::ObTaskEvent));
RPC_S(@PR5 task_complete, obrpc::OB_TASK_COMPLETE, (sql::ObTaskCompleteEvent));
RPC_S(@PR5 mini_task_execute, obrpc::OB_MINI_TASK_EXECUTE, (sql::ObMiniTask), sql::ObMiniTaskResult);
RPC_S(@PR5 bkgd_task_submit, obrpc::OB_BKGD_DIST_EXECUTE, (sql::ObBKGDDistExecuteArg));
RPC_S(@PR5 bkgd_task_complete, obrpc::OB_BKGD_TASK_COMPLETE, (sql::ObBKGDTaskCompleteArg));
RPC_S(
@PR5 check_build_index_task_exist, OB_CHECK_BUILD_INDEX_TASK_EXIST, (ObCheckBuildIndexTaskExistArg), obrpc::Bool);
RPC_AP(@PR5 close_result, obrpc::OB_CLOSE_RESULT, (sql::ObSliceID));
// ap_task_submit async process of task submit.
// The task complete event would process in IO thread.
RPC_AP(@PR5 ap_task_submit, obrpc::OB_AP_DIST_EXECUTE, (sql::ObTask), sql::ObTaskCompleteEvent);
RPC_AP(@PR5 ap_mini_task_submit, obrpc::OB_AP_MINI_DIST_EXECUTE, (sql::ObMiniTask), sql::ObMiniTaskResult);
RPC_AP(@PR5 ap_ping_sql_task, obrpc::OB_AP_PING_SQL_TASK, (sql::ObPingSqlTask), sql::ObPingSqlTaskResult);
RPC_AP(@PR5 remote_task_submit, obrpc::OB_REMOTE_ASYNC_EXECUTE, (sql::ObRemoteTask));
RPC_AP(@PR5 remote_post_result, obrpc::OB_REMOTE_POST_RESULT, (sql::ObRemoteResult));
};
} // namespace obrpc
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_EXECUTOR_RPC_PROXY_ */
//// end of header file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,510 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_FIFO_RECEIVE_
#define OCEANBASE_SQL_EXECUTOR_FIFO_RECEIVE_
#include "sql/engine/sort/ob_base_sort.h"
#include "sql/executor/ob_receive.h"
#include "sql/executor/ob_executor_rpc_impl.h"
#include "sql/executor/ob_distributed_scheduler.h"
namespace oceanbase {
namespace sql {
class ObExecContext;
class ObTaskInfo;
class ObPhyOperator;
class ObTaskResultIter {
public:
enum IterType {
IT_ROOT,
IT_DISTRIBUTED,
};
public:
explicit ObTaskResultIter(IterType iter_tyep);
virtual ~ObTaskResultIter();
virtual int get_next_task_result(ObTaskResult& task_result) = 0;
virtual int check_status() = 0;
inline IterType get_iter_type()
{
return iter_type_;
}
protected:
IterType iter_type_;
};
class ObRootTaskResultIter : public ObTaskResultIter {
public:
ObRootTaskResultIter(ObExecContext& exec_ctx, uint64_t exec_id, uint64_t child_op_id, int64_t ts_timeout);
virtual ~ObRootTaskResultIter();
virtual int get_next_task_result(ObTaskResult& task_result);
virtual int check_status();
int init();
private:
ObExecContext& exec_ctx_;
uint64_t exec_id_;
ObDistributedSchedulerManager::ObDistributedSchedulerHolder scheduler_holder_;
ObDistributedScheduler* scheduler_;
uint64_t child_op_id_;
int64_t ts_timeout_;
};
class ObDistributedTaskResultIter : public ObTaskResultIter {
public:
explicit ObDistributedTaskResultIter(const ObIArray<ObTaskResultBuf>& task_results);
virtual ~ObDistributedTaskResultIter();
virtual int get_next_task_result(ObTaskResult& task_result);
virtual int check_status();
private:
const ObIArray<ObTaskResultBuf>& task_results_;
int64_t cur_idx_;
};
// class ObRootReceiveInput : public ObIPhyOperatorInput
//{
// OB_UNIS_VERSION_V(1);
// public:
// ObRootReceiveInput();
// virtual ~ObRootReceiveInput();
// virtual ObPhyOperatorType get_phy_op_type() const { return PHY_ROOT_RECEIVE; }
// virtual int init(ObExecContext &exec_ctx, ObTaskInfo &task_info, ObPhyOperator &phy_op);
// inline uint64_t get_child_op_id() const { return child_op_id_; }
// private:
// uint64_t child_op_id_;
//};
class ObDistributedReceiveInput : public ObIPhyOperatorInput {
OB_UNIS_VERSION_V(1);
public:
ObDistributedReceiveInput();
virtual ~ObDistributedReceiveInput();
virtual void reset() override;
virtual ObPhyOperatorType get_phy_op_type() const
{
return PHY_DISTRIBUTED_RECEIVE;
}
virtual int init(ObExecContext& exec_ctx, ObTaskInfo& task_info, const ObPhyOperator& phy_op);
inline const common::ObIArray<ObTaskResultBuf>& get_child_task_results() const
{
return child_task_results_;
}
inline void set_child_job_id(uint64_t child_job_id)
{
child_job_id_ = child_job_id;
}
private:
common::ObSEArray<ObTaskResultBuf, 8> child_task_results_;
uint64_t child_job_id_; // need serialized, but only used for init child_task_results.
};
class ObIDataSource {
public:
ObIDataSource();
virtual ~ObIDataSource();
int get_next_row(ObNewRow& row);
virtual int open(ObExecContext& exec_ctx, const ObTaskResult& task_result);
virtual int close() = 0;
TO_STRING_KV(K_(inited), K_(cur_scanner));
protected:
virtual int fetch_next_scanner() = 0;
common::ObScanner cur_scanner_;
common::ObRowStore::Iterator row_iter_;
ObExecContext* exec_ctx_;
ObSliceID slice_id_;
common::ObAddr peer_;
bool use_small_result_;
bool inited_;
};
// Fetch interm result using stream RPC (occupy one peer thread).
// Only for compatibility, can be removed after all cluster upgrade to 2.1.0
class ObStreamDataSource : public ObIDataSource {
public:
ObStreamDataSource();
~ObStreamDataSource();
virtual int close() override;
private:
virtual int fetch_next_scanner() override;
/*
* ObExecutorRpcImpl::task_fetch_interm_result() => FetchIntermResultStreamHandle
* FetchIntermResultStreamHandle::get_result() => ObIntermResultItem
* ObIntermResultItem::assign_to_scanner() => ObScanner
* ObScanner::begin() => ObRowStore::Iterator
* ObRowStore::Iterator::get_next_row() => ObNewRow
*/
FetchIntermResultStreamHandle stream_handler_;
bool stream_opened_;
};
// Fetch interm result using normal RPC, specify interm item index every time.
class ObSpecifyDataSource : public ObIDataSource {
public:
ObSpecifyDataSource();
~ObSpecifyDataSource();
virtual int close() override;
private:
virtual int fetch_next_scanner() override;
int64_t fetch_index_;
ObFetchIntermResultItemRes interm_result_item_;
};
class ObAsyncReceive : public ObReceive {
protected:
class ObAsyncReceiveCtx : public ObReceiveCtx {
friend class ObAsyncReceive;
friend class ObFifoReceiveV2;
public:
explicit ObAsyncReceiveCtx(ObExecContext& exec_ctx);
virtual ~ObAsyncReceiveCtx();
virtual void destroy();
int init_root_iter(uint64_t child_op_id);
int init_distributed_iter(const ObIArray<ObTaskResultBuf>& task_results);
protected:
ObTaskResultIter* task_result_iter_;
int64_t found_rows_;
int64_t affected_rows_;
int64_t matched_rows_;
int64_t duplicated_rows_;
bool iter_end_;
};
public:
explicit ObAsyncReceive(common::ObIAllocator& alloc);
virtual ~ObAsyncReceive();
void set_in_root_job(bool in_root_job)
{
in_root_job_ = in_root_job;
}
bool is_in_root_job() const
{
return in_root_job_;
}
protected:
virtual int create_operator_input(ObExecContext& exec_ctx) const;
virtual int create_op_ctx(ObExecContext& exec_ctx, ObAsyncReceiveCtx*& op_ctx) const = 0;
virtual int init_op_ctx(ObExecContext& exec_ctx) const;
virtual int inner_close(ObExecContext& exec_ctx) const;
virtual int rescan(ObExecContext& ctx) const;
virtual int get_next_task_result(ObAsyncReceiveCtx& op_ctx, ObTaskResult& task_result) const;
int create_data_source(ObExecContext& exec_ctx, ObIDataSource*& data_source) const;
protected:
bool in_root_job_;
};
class ObSerialReceive : public ObAsyncReceive {
protected:
class ObSerialReceiveCtx : public ObAsyncReceiveCtx {
friend class ObSerialReceive;
public:
explicit ObSerialReceiveCtx(ObExecContext& exec_ctx);
virtual ~ObSerialReceiveCtx();
virtual void destroy();
protected:
ObIDataSource* data_source_;
};
public:
explicit ObSerialReceive(common::ObIAllocator& alloc);
virtual ~ObSerialReceive();
protected:
virtual int inner_open(ObExecContext& exec_ctx) const;
virtual int inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const;
virtual int inner_close(ObExecContext& exec_ctx) const;
};
class ObParallelReceive : public ObAsyncReceive {
protected:
class ObParallelReceiveCtx : public ObAsyncReceiveCtx {
friend class ObParallelReceive;
friend class ObMergeSortReceive;
public:
explicit ObParallelReceiveCtx(ObExecContext& exec_ctx);
virtual ~ObParallelReceiveCtx();
virtual void destroy();
protected:
common::ObSEArray<ObIDataSource*, 8> data_sources_;
common::ObSEArray<common::ObNewRow, 8> child_rows_;
common::ObSEArray<int64_t, 8> row_idxs_;
};
public:
explicit ObParallelReceive(common::ObIAllocator& alloc);
virtual ~ObParallelReceive();
protected:
virtual int inner_open(ObExecContext& exec_ctx) const;
virtual int inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const = 0;
virtual int inner_close(ObExecContext& exec_ctx) const;
int create_new_row(ObExecContext& exec_ctx, const ObNewRow& cur_row, ObNewRow& row) const;
};
class ObFifoReceiveV2 : public ObSerialReceive {
typedef ObSerialReceiveCtx ObFifoReceiveCtx;
public:
explicit ObFifoReceiveV2(common::ObIAllocator& alloc);
virtual ~ObFifoReceiveV2();
protected:
virtual int create_op_ctx(ObExecContext& exec_ctx, ObAsyncReceiveCtx*& op_ctx) const;
};
class ObTaskOrderReceive : public ObSerialReceive {
struct ObTaskComparer {
public:
ObTaskComparer();
virtual ~ObTaskComparer();
bool operator()(const ObTaskResult& task1, const ObTaskResult& task2);
};
class ObTaskOrderReceiveCtx : public ObSerialReceiveCtx {
friend class ObTaskOrderReceive;
public:
explicit ObTaskOrderReceiveCtx(ObExecContext& exec_ctx);
virtual ~ObTaskOrderReceiveCtx();
virtual void destroy();
protected:
common::ObSEArray<ObTaskResult, 8> task_results_;
ObTaskComparer task_comparer_;
uint64_t cur_task_id_;
};
public:
explicit ObTaskOrderReceive(common::ObIAllocator& alloc);
virtual ~ObTaskOrderReceive();
protected:
virtual int create_op_ctx(ObExecContext& exec_ctx, ObAsyncReceiveCtx*& op_ctx) const;
virtual int get_next_task_result(ObAsyncReceiveCtx& op_ctx, ObTaskResult& task_result) const;
private:
int get_next_task_result_root(ObAsyncReceiveCtx& op_ctx, ObTaskResult& task_result) const;
int get_next_task_result_distributed(ObAsyncReceiveCtx& op_ctx, ObTaskResult& task_result) const;
};
class ObMergeSortReceive : public ObParallelReceive, public ObSortableTrait {
OB_UNIS_VERSION_V(1);
private:
struct ObRowComparer {
public:
ObRowComparer();
virtual ~ObRowComparer();
void init(const common::ObIArray<ObSortColumn>& columns, const common::ObIArray<ObNewRow>& rows);
bool operator()(int64_t row_idx1, int64_t row_idx2);
int get_ret() const
{
return ret_;
}
private:
const common::ObIArray<ObSortColumn>* columns_;
const common::ObIArray<ObNewRow>* rows_;
int ret_;
};
class ObMergeSortReceiveCtx : public ObParallelReceiveCtx {
friend class ObMergeSortReceive;
public:
explicit ObMergeSortReceiveCtx(ObExecContext& exec_ctx);
virtual ~ObMergeSortReceiveCtx();
private:
ObRowComparer row_comparer_;
int64_t last_row_idx_;
};
public:
explicit ObMergeSortReceive(common::ObIAllocator& alloc);
virtual ~ObMergeSortReceive();
protected:
virtual int create_op_ctx(ObExecContext& exec_ctx, ObAsyncReceiveCtx*& op_ctx) const;
virtual int inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const;
};
class ObFifoReceiveInput : public ObReceiveInput {
friend class ObFifoReceive;
OB_UNIS_VERSION_V(1);
public:
ObFifoReceiveInput(){};
virtual ~ObFifoReceiveInput()
{}
virtual void reset() override
{}
virtual ObPhyOperatorType get_phy_op_type() const;
};
class ObDistributedScheduler;
class ObFifoReceive : public ObReceive, public ObSortableTrait {
private:
struct MergeRowComparer;
struct MergeSortHandle {
MergeSortHandle()
: stream_handler_(common::ObModIds::OB_SQL_EXECUTOR),
merge_id_(0),
row_iter_(),
curhandler_scanner_(common::ObModIds::OB_SQL_EXECUTOR_MERGE_SORT_SCANNER)
{}
~MergeSortHandle()
{}
virtual void destroy()
{
stream_handler_.~FetchIntermResultStreamHandle();
curhandler_scanner_.~ObScanner();
}
void reset()
{
stream_handler_.reset();
merge_id_ = 0;
curhandler_scanner_.reset();
}
TO_STRING_KV("merge id", merge_id_);
FetchIntermResultStreamHandle stream_handler_;
int64_t merge_id_; // pos in merge_handles
common::ObRowStore::Iterator row_iter_;
common::ObScanner curhandler_scanner_;
};
struct ObSortRow {
ObSortRow() : row_(NULL), pos_(0)
{}
TO_STRING_KV("pos", pos_);
common::ObNewRow* row_;
int64_t pos_;
};
class ObFifoReceiveCtx : public ObReceiveCtx {
friend class ObFifoReceive;
public:
explicit ObFifoReceiveCtx(ObExecContext& ctx);
virtual ~ObFifoReceiveCtx();
virtual void destroy()
{
stream_handler_.~FetchIntermResultStreamHandle();
old_stream_handler_.~FetchResultStreamHandle();
cur_scanner_.~ObScanner();
for (int64_t i = 0; i < merge_handles_.count(); ++i) {
if (NULL != merge_handles_.at(i)) {
merge_handles_.at(i)->~MergeSortHandle();
}
}
merge_handles_.destroy();
heap_sort_rows_.destroy();
ObReceiveCtx::destroy();
}
private:
int64_t found_rows_;
FetchIntermResultStreamHandle stream_handler_;
FetchResultStreamHandle old_stream_handler_;
common::ObScanner cur_scanner_;
common::ObRowStore::Iterator cur_scanner_iter_;
ObDistributedSchedulerManager::ObDistributedSchedulerHolder scheduler_holder_;
bool iter_has_started_;
bool iter_end_;
common::ObArenaAllocator allocator_;
common::ObList<ObTaskResult, common::ObArenaAllocator> waiting_finish_tasks_;
uint64_t last_pull_task_id_;
int64_t affected_row_;
common::ObNewRow* row_i_; // cur row from merge sort
common::ObNewRow child_row_;
MergeSortHandle* cur_merge_handle_;
common::ObSEArray<MergeSortHandle*, 8> merge_handles_;
common::ObSEArray<ObSortRow, 8> heap_sort_rows_;
};
public:
explicit ObFifoReceive(common::ObIAllocator& aloc);
virtual ~ObFifoReceive();
virtual int create_operator_input(ObExecContext& ctx) const;
virtual int rescan(ObExecContext& ctx) const;
private:
/* functions */
virtual int inner_open(ObExecContext& ctx) const;
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
virtual int inner_close(ObExecContext& ctx) const;
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
int fetch_more_result(
ObExecContext& ctx, ObFifoReceiveCtx* fifo_receive_ctx, const ObFifoReceiveInput* fifo_receive_input) const;
int new_fetch_more_result(
ObExecContext& ctx, ObFifoReceiveCtx* fifo_receive_ctx, const ObFifoReceiveInput* fifo_receive_input) const;
int old_fetch_more_result(
ObExecContext& ctx, ObFifoReceiveCtx* fifo_receive_ctx, const ObFifoReceiveInput* fifo_receive_input) const;
int check_schedule_status(ObDistributedScheduler* scheduler) const;
int deal_with_insert(ObExecContext& ctx) const;
// for merge sort
int merge_sort_result(
ObExecContext& ctx, ObFifoReceiveCtx* fifo_receive_ctx, const ObFifoReceiveInput* fifo_receive_input) const;
int deal_with_task(ObExecContext& ctx, ObFifoReceiveCtx& fifo_receive_ctx, MergeSortHandle& sort_handler,
ObTaskResult& task_result, bool& result_scanner_is_empty, const ObFifoReceiveInput& fifo_receive_input) const;
int fetch_a_new_scanner(ObExecContext& ctx, ObFifoReceiveCtx& fifo_receive_ctx, MergeSortHandle& sort_handler,
const ObFifoReceiveInput& fifo_receive_input) const;
int partition_order_fetch_task(ObExecContext& ctx, ObFifoReceiveCtx& fifo_receive_ctx,
ObDistributedScheduler* scheduler, int64_t timeout_timestamp, ObTaskLocation& task_loc,
const ObFifoReceiveInput& fifo_receive_input) const;
int create_new_cur_row(ObExecContext& ctx, const common::ObNewRow& cur_row, common::ObNewRow*& row_i) const;
int first_get_row(ObExecContext& ctx, ObFifoReceiveCtx& fifo_receive_ctx) const;
int get_row_from_heap(ObFifoReceiveCtx& fifo_receive_ctx) const;
int more_get_scanner(ObExecContext& ctx, ObFifoReceiveCtx& fifo_receive_ctx, MergeSortHandle& sort_handler,
const ObFifoReceiveInput& fifo_receive_input) const;
int get_row_from_scanner(ObExecContext& ctx, ObFifoReceiveCtx& fifo_receive_ctx, MergeSortHandle& sort_handler,
const ObFifoReceiveInput& fifo_receive_input) const;
int init_row_heap(ObExecContext& ctx, ObFifoReceiveCtx& fifo_receive_ctx) const;
int pop_a_row_from_heap(ObFifoReceiveCtx& fifo_receive_ctx, ObSortRow& row) const;
int push_a_row_into_heap(ObFifoReceiveCtx& fifo_receive_ctx, ObSortRow& row) const;
TO_STRING_KV(N_ORDER_BY, sort_columns_);
private:
DISALLOW_COPY_AND_ASSIGN(ObFifoReceive);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_FIFO_RECEIVE_ */
//// end of header file

View File

@ -0,0 +1,738 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_interm_result.h"
#include "sql/executor/ob_task_event.h"
#include "lib/utility/ob_tracepoint.h"
#include "lib/alloc/ob_malloc_allocator.h"
#include "share/config/ob_server_config.h"
#include "sql/executor/ob_interm_result_pool.h"
namespace oceanbase {
namespace sql {
using namespace common;
ObIntermResult::ObIntermResult()
: cur_scanner_(NULL),
data_(),
rows_is_completed_(false),
expire_time_(-1),
found_rows_(0),
last_insert_id_session_(0),
is_result_accurate_(true),
matched_rows_(0),
duplicated_rows_(0),
fd_(-1),
dir_id_(-1),
offset_(0),
row_reclaim_func_(NULL)
{
cnt_and_state_.cnt_ = 0;
cnt_and_state_.state_ = STATE_NORMAL;
ir_item_pool_ = ObIntermResultItemPool::get_instance();
if (OB_ISNULL(ir_item_pool_)) {
LOG_ERROR("unexpected, global interm result item pool is NULL");
}
affected_rows_ = 0;
}
ObIntermResult::~ObIntermResult()
{
reset();
}
void ObIntermResult::reset()
{
int ret = OB_SUCCESS;
reclaim_rows();
rows_is_completed_ = false;
expire_time_ = -1;
found_rows_ = 0;
last_insert_id_session_ = 0;
is_result_accurate_ = true;
matched_rows_ = 0;
duplicated_rows_ = 0;
ObIIntermResultItem* cur_ir_item = NULL;
// ir_item_pool_ need not set to NULL
if (OB_I(t1) OB_ISNULL(ir_item_pool_)) {
LOG_ERROR("unexpected, interm result item pool is NULL");
} else {
while (OB_SUCCESS == data_.pop_back(cur_ir_item)) {
if (OB_ISNULL(cur_ir_item)) {
LOG_ERROR("interm result item in data_ is NULL");
} else {
cur_ir_item->reset();
ir_item_pool_->free(cur_ir_item);
}
}
}
data_.reset();
if (is_disk_store_opened()) {
if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.remove(fd_))) {
LOG_WARN("close disk store file failed", K(ret), K_(fd));
} else {
LOG_INFO("close disk store file success", K_(fd));
}
fd_ = -1;
dir_id_ = -1;
}
offset_ = 0;
free_scanner();
if (OB_FAIL(try_end_recycle())) {
LOG_DEBUG("fail to end recycle, maybe it has not begin recycling", K(ret));
}
affected_rows_ = 0;
}
int ObIntermResult::try_inc_cnt()
{
int ret = OB_SUCCESS;
AtomicCntAndState atomic_old = {0};
AtomicCntAndState atomic_new = {0};
AtomicCntAndState atomic_cmp = {0};
bool cas_succ = false;
while (OB_SUCC(ret) && false == cas_succ) {
atomic_old.atomic_ = cnt_and_state_.atomic_;
atomic_cmp.atomic_ = atomic_old.atomic_;
atomic_new.atomic_ = atomic_old.atomic_;
if (OB_I(t1)(STATE_NORMAL != atomic_cmp.state_)) {
ret = OB_STATE_NOT_MATCH;
LOG_DEBUG("the interm result is being recycled, fail to increase it's reference count", K(ret));
} else {
atomic_new.cnt_ += 1;
if (atomic_old.atomic_ == ATOMIC_VCAS(&(cnt_and_state_.atomic_), atomic_cmp.atomic_, atomic_new.atomic_)) {
cas_succ = true;
}
}
}
return ret;
}
int ObIntermResult::try_dec_cnt()
{
int ret = OB_SUCCESS;
AtomicCntAndState atomic_old = {0};
AtomicCntAndState atomic_new = {0};
AtomicCntAndState atomic_cmp = {0};
bool cas_succ = false;
while (OB_SUCC(ret) && false == cas_succ) {
atomic_old.atomic_ = cnt_and_state_.atomic_;
atomic_cmp.atomic_ = atomic_old.atomic_;
atomic_new.atomic_ = atomic_old.atomic_;
if (OB_I(t1)(STATE_NORMAL != atomic_cmp.state_)) {
ret = OB_STATE_NOT_MATCH;
LOG_DEBUG("the interm result is being recycled, fail to decrease it's reference count", K(ret));
} else {
atomic_new.cnt_ -= 1;
if (atomic_old.atomic_ == ATOMIC_VCAS(&(cnt_and_state_.atomic_), atomic_cmp.atomic_, atomic_new.atomic_)) {
cas_succ = true;
}
}
}
return ret;
}
int ObIntermResult::try_begin_recycle()
{
int ret = OB_SUCCESS;
AtomicCntAndState atomic_old = {0};
AtomicCntAndState atomic_new = {0};
AtomicCntAndState atomic_cmp = {0};
bool cas_succ = false;
while (OB_SUCC(ret) && false == cas_succ) {
atomic_old.atomic_ = cnt_and_state_.atomic_;
atomic_cmp.atomic_ = atomic_old.atomic_;
atomic_new.atomic_ = atomic_old.atomic_;
atomic_new.state_ = STATE_RECYCLE;
if (OB_I(t1)(STATE_NORMAL != atomic_cmp.state_)) {
ret = OB_STATE_NOT_MATCH;
LOG_DEBUG("the iterm result is already in recycling, fail to set state to STATE_RECYCLE", K(ret));
} else if (0 != atomic_cmp.cnt_) {
ret = OB_STATE_NOT_MATCH;
LOG_DEBUG("the reference count is not 0, fail to set state to STATE_RECYCLE", K(ret));
} else if (atomic_old.atomic_ == ATOMIC_VCAS(&(cnt_and_state_.atomic_), atomic_cmp.atomic_, atomic_new.atomic_)) {
cas_succ = true;
}
}
return ret;
}
int ObIntermResult::try_end_recycle()
{
int ret = OB_SUCCESS;
AtomicCntAndState atomic_old = {0};
AtomicCntAndState atomic_new = {0};
AtomicCntAndState atomic_cmp = {0};
bool cas_succ = false;
while (OB_SUCC(ret) && false == cas_succ) {
atomic_old.atomic_ = cnt_and_state_.atomic_;
atomic_cmp.atomic_ = atomic_old.atomic_;
atomic_new.atomic_ = atomic_old.atomic_;
atomic_new.state_ = STATE_NORMAL;
if (OB_I(t1)(STATE_RECYCLE != atomic_cmp.state_)) {
ret = OB_ERR_UNEXPECTED;
LOG_DEBUG("the iterm result has not begin recycling, fail to set state to STATE_NORMAL", K(ret));
} else if (OB_I(t2)(0 != atomic_cmp.cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("the reference count is not 0, fail to set state to STATE_NORMAL", K(ret));
} else if (atomic_old.atomic_ == ATOMIC_VCAS(&(cnt_and_state_.atomic_), atomic_cmp.atomic_, atomic_new.atomic_)) {
cas_succ = true;
}
}
return ret;
}
int ObIntermResult::choose_store(bool& disk, const uint64_t tenant_id, const bool force_disk_store) const
{
int ret = OB_SUCCESS;
disk = false; // default memory store
if (force_disk_store || is_disk_store_opened()) {
disk = true;
} else if (!GCONF.is_sql_operator_dump_enabled()) {
} else {
const int64_t mem_ctx_pct_trigger = 70;
lib::ObMallocAllocator* instance = lib::ObMallocAllocator::get_instance();
lib::ObTenantCtxAllocator* allocator = NULL;
if (NULL == instance) {
ret = common::OB_ERR_SYS;
LOG_ERROR("NULL malloc allocator", K(ret));
} else if (OB_ISNULL(allocator = instance->get_tenant_ctx_allocator(tenant_id, common::ObCtxIds::WORK_AREA))) {
// no tenant allocator, do nothing
} else {
const int64_t limit = allocator->get_limit();
const int64_t hold = allocator->get_hold();
if (limit / 100 * mem_ctx_pct_trigger <= hold) {
disk = true;
}
LOG_TRACE("choose store for interm result", K(tenant_id), K(limit), K(hold), K(disk));
}
}
return ret;
}
int ObIntermResult::alloc_ir_item(ObIIntermResultItem*& item, const uint64_t tenant_id, const bool force_disk_store)
{
int ret = OB_SUCCESS;
bool use_disk_store = false;
if (OB_ISNULL(ir_item_pool_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("item pool is NULL", K(ret));
} else if (OB_FAIL(choose_store(use_disk_store, tenant_id, force_disk_store))) {
LOG_WARN("choose store failed", K(ret), K(tenant_id), K(force_disk_store));
} else {
item = NULL;
if (!use_disk_store) {
ObIntermResultItem* mem_item = NULL;
if (OB_FAIL(ir_item_pool_->alloc_mem_item(mem_item, tenant_id))) {
LOG_WARN("alloc memory interm result item failed", K(ret), K(tenant_id));
} else {
item = mem_item;
}
} else {
if (!is_disk_store_opened()) {
if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.alloc_dir(dir_id_))) {
LOG_WARN("allocate disk store file directory failed", K(ret));
} else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.open(fd_, dir_id_))) {
LOG_WARN("open disk store file failed", K(ret));
} else {
offset_ = 0;
LOG_INFO("open disk store file success", K_(fd), K_(dir_id));
}
}
ObDiskIntermResultItem* disk_item = NULL;
if (OB_FAIL(ret)) {
} else if (OB_FAIL(ir_item_pool_->alloc_disk_item(disk_item, tenant_id, fd_, dir_id_, offset_))) {
LOG_WARN("alloc disk interm result item failed", K(ret), K(tenant_id), K_(fd), K_(dir_id));
} else {
item = disk_item;
}
}
}
return ret;
}
int ObIntermResult::save_cur_scanner(uint64_t tenant_id)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(ir_item_pool_) || OB_ISNULL(cur_scanner_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("interm result item pool or scanner is NULL", K(ret));
} else {
cur_scanner_->set_is_result_accurate(is_result_accurate_);
bool in_memory = false;
ObIIntermResultItem* item = NULL;
do {
// try to save on disk since the second scanner or fail to save in memory.
bool force_disk_store = (!data_.empty() || in_memory) && GCONF.is_sql_operator_dump_enabled();
in_memory = false;
if (OB_FAIL(alloc_ir_item(item, tenant_id, force_disk_store))) {
LOG_WARN("fail to alloc interm result item", K(ret), K(tenant_id));
} else if (OB_ISNULL(item)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("succeed to alloc interm result item, but item is NULL", K(ret), K(tenant_id));
} else {
in_memory = item->in_memory();
if (OB_FAIL(item->from_scanner(*cur_scanner_))) {
LOG_WARN("fail to assign interm result item from current scanner", K(ret));
item->reset();
ir_item_pool_->free(item);
item = NULL;
} else {
if (!in_memory) {
offset_ += item->get_data_len();
}
}
}
} while (GCONF.is_sql_operator_dump_enabled() && in_memory &&
(OB_ALLOCATE_MEMORY_FAILED == ret || OB_TENANT_OUT_OF_MEM == ret));
if (OB_FAIL(ret)) {
} else if (OB_FAIL(data_.push_back(item))) {
LOG_WARN("fail to push back interm result item", K(ret), KP(item));
item->reset();
ir_item_pool_->free(item);
item = NULL;
}
}
return ret;
}
int ObIntermResult::add_row(uint64_t tenant_id, const common::ObNewRow& row)
{
int ret = OB_SUCCESS;
if (OB_I(t2)(STATE_NORMAL != cnt_and_state_.state_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("the state of the interm result is not STATE_NORMAL, it is not writable.", K(ret));
} else if (OB_UNLIKELY(rows_is_completed())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("rows is completed, can not add row", K(ret), K(row));
} else if (OB_FAIL(check_and_init_cur_scanner())) {
LOG_WARN("fail to check and init current scanner", K(ret));
} else if (OB_UNLIKELY(OB_SIZE_OVERFLOW == (ret = cur_scanner_->add_row(row)))) {
if (OB_FAIL(save_cur_scanner(tenant_id))) {
LOG_WARN("fail to save current scanner", K(ret), K(row), K(tenant_id));
} else if (OB_FAIL(reset_and_init_cur_scanner())) {
LOG_WARN("fail to reset and init current scanner", K(ret));
} else if (OB_FAIL(OB_I(t6) cur_scanner_->add_row(row))) {
// give lob row second chance and reset mem size limit
if (OB_SIZE_OVERFLOW == ret) {
cur_scanner_->set_mem_size_limit(common::ObScanner::DEFAULT_MAX_SERIALIZE_SIZE);
if (OB_FAIL(reset_and_init_cur_scanner())) {
LOG_WARN("fail to reset and init current scanner", K(ret));
} else if (OB_FAIL(cur_scanner_->add_row(row))) {
LOG_WARN("fail to add big row to new cur scanner", K(ret), K(row));
}
} else {
LOG_WARN("fail to add row to new cur scanner", K(ret), K(row));
}
} else {
// empty
}
} else if (OB_FAIL(ret)) {
LOG_WARN("fail to add row to cur scanner", K(ret), K(row));
} else {
// empty
}
return ret;
}
int ObIntermResult::complete_add_rows(uint64_t tenant_id)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(data_.count() < 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("data count less than 0", K(ret), K(data_.count()));
} else {
if (NULL != cur_scanner_ && !cur_scanner_->is_empty()) {
cur_scanner_->set_found_rows(get_found_rows());
cur_scanner_->set_last_insert_id_session(get_last_insert_id_session());
cur_scanner_->set_row_matched_count(get_matched_rows());
cur_scanner_->set_row_duplicated_count(get_duplicated_rows());
if (!is_result_accurate()) {
cur_scanner_->set_is_result_accurate(is_result_accurate());
}
NG_TRACE_EXT(found_rows, OB_ID(found_rows), get_found_rows());
NG_TRACE_EXT(last_insert_id, OB_ID(last_insert_id), cur_scanner_->get_last_insert_id_session());
if (OB_FAIL(save_cur_scanner(tenant_id))) {
LOG_WARN("fail to save last current scanner", K(ret), K(tenant_id));
} else {
// all rows have been added, need not reset.
cur_scanner_->reset();
}
}
if (OB_SUCC(ret) && is_disk_store_opened()) {
int64_t timeout_ms = 0;
if (OB_FAIL(ObDiskIntermResultItem::get_timeout(timeout_ms))) {
LOG_WARN("get timeout failed", K(ret));
} else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.sync(fd_, timeout_ms))) {
LOG_WARN("sync interm result disk store file failed", K(ret), K_(fd), K(timeout_ms));
}
}
}
if (OB_SUCC(ret)) {
rows_is_completed_ = true;
}
// When add row complete, free scanner
free_scanner();
return ret;
}
int ObIntermResult::try_fetch_single_scanner(ObTaskSmallResult& small_result) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!rows_is_completed())) {
LOG_ERROR("rows is not completed", K(ret));
} else if (OB_UNLIKELY(data_.count() > 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("scanner count > 1", K(ret), K(data_.count()));
} else if (0 == data_.count()) {
small_result.set_affected_rows(get_affected_rows());
small_result.set_duplicated_rows(get_duplicated_rows());
small_result.set_matched_rows(get_matched_rows());
small_result.set_found_rows(get_found_rows());
small_result.set_last_insert_id(get_last_insert_id_session());
small_result.set_has_data(true);
small_result.set_data_len(0);
} else { // 1 == data_.count()
ObIIntermResultItem* single_ir_item = data_.at(0);
if (OB_ISNULL(single_ir_item)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("interm result item ptr is NULL", K(ret));
} else if (OB_UNLIKELY(single_ir_item->get_data_len() > ObTaskSmallResult::MAX_DATA_BUF_LEN)) {
// do nothing.
} else if (OB_FAIL(small_result.assign_from_ir_item(*single_ir_item))) {
LOG_WARN("fail assign single interm result item to small result", K(ret), K(*single_ir_item));
}
}
return ret;
}
int ObIntermResult::get_all_row_count(int64_t& all_row_count)
{
int ret = OB_SUCCESS;
int64_t all_count = 0;
ObIIntermResultItem* ir_item = NULL;
for (int64_t i = 0; OB_SUCC(ret) && i < data_.count(); ++i) {
if (OB_FAIL(data_.at(i, ir_item))) {
LOG_WARN("fail to get interm result item from data", K(ret), K(i));
} else if (OB_ISNULL(ir_item)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("interm result item is NULL", K(ret), K(i));
} else {
all_count += ir_item->get_row_count();
}
}
if (OB_SUCC(ret)) {
all_row_count = all_count;
}
return ret;
}
int ObIntermResult::get_all_data_size(int64_t& size) const
{
int64_t mem = 0;
int64_t disk = 0;
int ret = get_data_size_detail(mem, disk);
if (OB_FAIL(ret)) {
LOG_WARN("get data size detail failed", K(ret));
} else {
size = mem + disk;
}
return ret;
}
int ObIntermResult::get_data_size_detail(int64_t& mem, int64_t& disk) const
{
int ret = OB_SUCCESS;
mem = 0;
disk = 0;
ObIIntermResultItem* ir_item = NULL;
for (int64_t i = 0; OB_SUCC(ret) && i < data_.count(); ++i) {
if (OB_FAIL(data_.at(i, ir_item))) {
LOG_WARN("fail to get interm result item from data", K(ret), K(i));
} else if (OB_ISNULL(ir_item)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("interm result item is NULL", K(ret), K(i));
} else {
if (ir_item->in_memory()) {
mem += ir_item->get_data_len();
} else {
disk += ir_item->get_data_len();
}
}
}
return ret;
}
int ObIntermResult::alloc_scanner()
{
int ret = OB_SUCCESS;
if (NULL == cur_scanner_) {
ObIntermResultPool* pool = ObIntermResultPool::get_instance();
if (NULL == pool) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("interm result pool is NULL", K(ret));
} else if (OB_FAIL(pool->alloc_scanner(cur_scanner_))) {
LOG_WARN("alloc scanner from interm result pool failed", K(ret));
}
}
return ret;
}
void ObIntermResult::free_scanner()
{
ObIntermResultPool* pool = ObIntermResultPool::get_instance();
if (NULL != cur_scanner_ && NULL != pool) {
pool->free_scanner(cur_scanner_);
cur_scanner_ = NULL;
}
}
int ObIntermResult::check_and_init_cur_scanner()
{
int ret = OB_SUCCESS;
if (NULL == cur_scanner_) {
if (OB_FAIL(alloc_scanner())) {
LOG_WARN("alloc scanner failed", K(ret));
} else if (OB_ISNULL(cur_scanner_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("allocated scanner is NULL", K(ret));
} else if (OB_FAIL(cur_scanner_->init())) {
LOG_WARN("scanner init failed", K(ret));
}
}
return ret;
}
int ObIntermResult::reset_and_init_cur_scanner()
{
int ret = OB_SUCCESS;
if (NULL == cur_scanner_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cur scanner is NULL", K(ret));
} else {
cur_scanner_->reset();
if (!cur_scanner_->is_inited() && OB_FAIL(cur_scanner_->init())) {
LOG_WARN("scanner init failed", K(ret));
}
}
return ret;
}
void ObIntermResult::reclaim_rows()
{
// reclaim rows are description for associated resources, 128 cells is enough.
const static int64_t max_row_cells_for_reclaim = 128;
int ret = OB_SUCCESS;
if (NULL != row_reclaim_func_) {
FOREACH_CNT(item, data_)
{ // continue reclaim when error happen, no need to check ret.
ObIntermResultItem* mem_item = NULL;
ObIntermResultItem disk2mem_item;
if (NULL != *item) {
if ((*item)->in_memory()) {
mem_item = static_cast<ObIntermResultItem*>(*item);
} else if (OB_FAIL(disk2mem_item.from_disk_ir_item(*static_cast<ObDiskIntermResultItem*>(*item)))) {
LOG_WARN("convert disk interm result item to memory interm item failed", K(ret), "item", *item);
} else {
mem_item = &disk2mem_item;
}
}
if (NULL != mem_item) {
ObScanner scanner;
const ObRowStore& rs = scanner.get_row_store();
int64_t pos = 0;
if (OB_FAIL(scanner.deserialize(mem_item->get_data_buf(), mem_item->get_data_len(), pos))) {
LOG_WARN("fail to deserialize scanner", K(ret), K(pos));
} else if (rs.get_col_count() <= 0 || rs.get_col_count() >= max_row_cells_for_reclaim) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("col count is invalid or exceed max cells for reclaim",
K(ret),
K(rs.get_col_count()),
LITERAL_K(max_row_cells_for_reclaim));
} else {
ObObj objs[rs.get_col_count()];
ObNewRow row;
row.cells_ = objs;
row.count_ = rs.get_col_count();
ObRowStore::Iterator it = rs.begin();
while (OB_SUCC(it.get_next_row(row))) {
row_reclaim_func_(row);
}
if (OB_ITER_END != ret) {
LOG_WARN("get row from row store iterator failed", K(ret));
} else {
ret = OB_SUCCESS;
}
}
}
}
// after row reclaimed, set reclaim function to NULL
row_reclaim_func_ = NULL;
}
}
int ObIntermResult::get_item(const int64_t index, ObIIntermResultItem*& item)
{
int ret = OB_SUCCESS;
if (index < 0 || index >= data_.count()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("index out of range", K(index));
} else {
item = data_.at(index);
}
return ret;
}
ObIntermResultIterator::ObIntermResultIterator()
: cur_pos_(0), cur_scanner_(NULL), row_store_it_(), ir_(NULL), ir_info_(), has_inc_cnt_(false)
{}
ObIntermResultIterator::~ObIntermResultIterator()
{
reset();
}
void ObIntermResultIterator::reset()
{
int ret = OB_SUCCESS;
if (NULL != ir_) {
if (has_inc_cnt_) {
if (OB_FAIL(ir_->try_dec_cnt())) {
LOG_ERROR("fail to decrease the reference count", K(ret));
} else {
has_inc_cnt_ = false;
}
} else {
LOG_ERROR("has not increase reference count");
}
}
ir_info_.reset();
ir_ = NULL;
row_store_it_.reset();
cur_scanner_ = NULL;
cur_pos_ = 0;
}
int ObIntermResultIterator::set_interm_result(const ObIntermResultInfo& ir_info, ObIntermResult* ir, bool has_inc_cnt)
{
int ret = OB_SUCCESS;
if (OB_I(t1) OB_ISNULL(ir)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("ir is NULL", K(ret));
} else if (OB_I(t2)(!ir_info.is_init())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("ir info is not init", K(ret), K(ir_info));
} else {
reset();
ir_info_ = ir_info;
ir_ = ir;
has_inc_cnt_ = has_inc_cnt;
}
return ret;
}
int ObIntermResultIterator::get_interm_result_info(ObIntermResultInfo& ir_info)
{
int ret = OB_SUCCESS;
if (OB_I(t1)(!ir_info_.is_init())) {
ret = OB_NOT_INIT;
LOG_WARN("interm result info is not init", K(ret), K(ir_info));
} else {
ir_info = ir_info_;
}
return ret;
}
int64_t ObIntermResultIterator::get_scanner_count()
{
int64_t count = 0;
if (OB_I(t1) OB_ISNULL(ir_)) {
LOG_WARN("The interm result iterator has not been initialized!");
} else {
count = ir_->data_.count();
}
return count;
}
int ObIntermResultIterator::get_next_scanner(ObScanner& scanner)
{
int ret = OB_SUCCESS;
ObIIntermResultItem* item = NULL;
int64_t pos = 0;
if (OB_I(t1) OB_ISNULL(ir_)) {
ret = OB_NOT_INIT;
LOG_WARN("The interm result iterator has not been initialized!", K(ret));
} else if (OB_I(t2)(ObIntermResult::STATE_NORMAL != ir_->cnt_and_state_.state_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the state of the interm result is not STATE_NORMAL, it is not readable.", K(ret));
} else if (OB_FAIL(ir_->data_.at(cur_pos_++, item))) {
if (OB_ARRAY_OUT_OF_RANGE == ret) {
ret = OB_ITER_END;
} else {
LOG_WARN("fail to get interm result item", K(ret), K(cur_pos_));
}
} else if (OB_ISNULL(item)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("interm result item in data is NULL", K(ret), K(cur_pos_));
} else {
scanner.reset();
ObIntermResultItem* mem_item = NULL;
ObIntermResultItem disk2mem_item;
if (item->in_memory()) {
mem_item = static_cast<ObIntermResultItem*>(item);
} else {
if (OB_FAIL(disk2mem_item.from_disk_ir_item(*static_cast<ObDiskIntermResultItem*>(item)))) {
LOG_WARN("convert disk interm result item to memory interm item failed", K(ret), "item", *item);
} else {
mem_item = &disk2mem_item;
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(scanner.deserialize(mem_item->get_data_buf(), mem_item->get_data_len(), pos))) {
LOG_WARN("fail to deserialize scanner", K(ret), K(cur_pos_), K(pos));
}
}
return ret;
}
int ObIntermResultIterator::get_next_interm_result_item(ObIIntermResultItem*& ir_item)
{
int ret = OB_SUCCESS;
if (OB_I(t1) OB_ISNULL(ir_)) {
ret = OB_NOT_INIT;
LOG_WARN("The interm result iterator has not been initialized!", K(ret));
} else if (OB_I(t2)(ObIntermResult::STATE_NORMAL != ir_->cnt_and_state_.state_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the state of the interm result is not STATE_NORMAL, it is not readable.", K(ret));
} else if (OB_FAIL(ir_->data_.at(cur_pos_++, ir_item))) {
if (OB_ARRAY_OUT_OF_RANGE == ret) {
ret = OB_ITER_END;
} else {
LOG_WARN("fail to get interm result item", K(ret), K(cur_pos_));
}
} else if (OB_ISNULL(ir_item)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("interm result item in data is NULL", K(ret), K(cur_pos_));
}
return ret;
}
} /* namespace sql */
} /* namespace oceanbase */

View File

@ -0,0 +1,240 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_
#define OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_
#include "sql/executor/ob_interm_result_item_pool.h"
#include "sql/executor/ob_slice_id.h"
#include "share/ob_scanner.h"
#include "lib/list/ob_list.h"
#include "share/ob_define.h"
#include "sql/engine/ob_phy_operator.h"
#include "storage/blocksstable/ob_tmp_file.h"
namespace oceanbase {
namespace sql {
class ObIntermResultItem;
class ObTaskSmallResult;
union AtomicCntAndState {
volatile uint64_t atomic_;
struct {
int32_t cnt_;
int32_t state_;
};
};
class ObIntermResultInfo {
public:
ObIntermResultInfo() : slice_id_()
{}
virtual ~ObIntermResultInfo()
{}
ObSliceID slice_id_;
inline void reset()
{
slice_id_.reset();
}
inline void init(const ObSliceID& slice_id)
{
slice_id_ = slice_id;
}
inline bool is_init() const
{
return slice_id_.is_valid();
}
inline int64_t hash() const
{
return slice_id_.hash();
}
inline bool operator==(const ObIntermResultInfo& other) const
{
return slice_id_.equal(other.slice_id_);
}
TO_STRING_KV(K_(slice_id));
};
class ObIntermResult {
public:
friend class ObIntermResultIterator;
static const int32_t STATE_NORMAL = 0;
static const int32_t STATE_RECYCLE = 1;
ObIntermResult();
virtual ~ObIntermResult();
void reset();
int try_inc_cnt();
int try_dec_cnt();
int try_begin_recycle();
int try_end_recycle();
int add_row(uint64_t tenant_id, const common::ObNewRow& row);
int get_all_row_count(int64_t& all_row_count);
int get_all_data_size(int64_t& size) const;
int get_data_size_detail(int64_t& mem, int64_t& disk) const;
bool rows_is_completed() const
{
return rows_is_completed_;
}
int complete_add_rows(uint64_t tenant_id);
inline void set_expire_time(int64_t expire_time)
{
expire_time_ = expire_time;
}
inline int64_t get_expire_time() const
{
return expire_time_;
}
inline int32_t get_state() const
{
return cnt_and_state_.state_;
}
void set_found_rows(const int64_t count)
{
found_rows_ = count;
}
int64_t get_found_rows() const
{
return found_rows_;
}
void set_affected_rows(const int64_t count)
{
affected_rows_ = count;
}
int64_t get_affected_rows() const
{
return affected_rows_;
}
int64_t get_scanner_count() const
{
return data_.count();
}
void set_last_insert_id_session(const int64_t last_insert_id)
{
last_insert_id_session_ = last_insert_id;
}
int64_t get_last_insert_id_session() const
{
return last_insert_id_session_;
}
int try_fetch_single_scanner(ObTaskSmallResult& small_result) const;
inline void set_is_result_accurate(bool is_accurate)
{
is_result_accurate_ = is_accurate;
}
inline bool is_result_accurate() const
{
return is_result_accurate_;
}
inline void set_matched_rows(int64_t matched_rows)
{
matched_rows_ = matched_rows;
}
inline int64_t get_matched_rows() const
{
return matched_rows_;
}
inline void set_duplicated_rows(int64_t duplicated_rows)
{
duplicated_rows_ = duplicated_rows;
}
inline int64_t get_duplicated_rows() const
{
return duplicated_rows_;
}
TO_STRING_EMPTY();
inline bool is_disk_store_opened(void) const
{
return fd_ >= 0;
}
int choose_store(bool& disk, const uint64_t tenant_id, const bool force_disk_store) const;
void set_row_reclaim_func(ObPhyOperator::reclaim_row_t func)
{
row_reclaim_func_ = func;
}
int get_item(const int64_t index, ObIIntermResultItem*& item);
private:
int alloc_scanner();
void free_scanner();
// reclaim with %row_reclaim_func_ when free interm result
void reclaim_rows();
int save_cur_scanner(uint64_t tenant_id);
int alloc_ir_item(ObIIntermResultItem*& item, const uint64_t tenant_id, const bool force_disk_store);
int reset_and_init_cur_scanner();
int check_and_init_cur_scanner();
private:
static const int64_t DEFAULT_INTERM_RESULT_ITEM_NUM = 2;
common::ObScanner* cur_scanner_;
ObIntermResultItemPool* ir_item_pool_;
common::ObSEArray<ObIIntermResultItem*, DEFAULT_INTERM_RESULT_ITEM_NUM> data_;
bool rows_is_completed_;
int64_t expire_time_;
int64_t found_rows_;
uint64_t last_insert_id_session_;
bool is_result_accurate_;
AtomicCntAndState cnt_and_state_;
int64_t affected_rows_;
int64_t matched_rows_;
int64_t duplicated_rows_;
int64_t fd_;
int64_t dir_id_;
int64_t offset_;
ObPhyOperator::reclaim_row_t row_reclaim_func_;
private:
DISALLOW_COPY_AND_ASSIGN(ObIntermResult);
};
class ObIntermResultIterator {
public:
friend class ObIntermResultManager;
ObIntermResultIterator();
virtual ~ObIntermResultIterator();
void reset();
// get scanner directly is used in old rpc, will be removed after upgrade to 1.4
// @deprecated
int get_next_scanner(common::ObScanner& scanner);
int get_next_interm_result_item(ObIIntermResultItem*& ir_item);
int64_t get_scanner_count();
ObIntermResult* get_interm_result()
{
return ir_;
}
// int64_t get_col_count();
private:
int set_interm_result(const ObIntermResultInfo& ir_info, ObIntermResult* ir, bool has_inc_cnt);
int get_interm_result_info(ObIntermResultInfo& ir_info);
int32_t cur_pos_;
// current iter scanner
common::ObScanner* cur_scanner_;
common::ObRowStore::Iterator row_store_it_;
ObIntermResult* ir_;
ObIntermResultInfo ir_info_;
bool has_inc_cnt_;
DISALLOW_COPY_AND_ASSIGN(
ObIntermResultIterator); // Copy must be prohibited, otherwise the reference count will go wrong
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_ */

View File

@ -0,0 +1,262 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "ob_interm_result_item.h"
#include "share/ob_scanner.h"
#include "storage/blocksstable/ob_tmp_file.h"
#include "share/ob_worker.h"
namespace oceanbase {
using namespace common;
namespace sql {
void ObIIntermResultItem::reset()
{
row_count_ = 0;
data_len_ = 0;
}
ObIntermResultItem::ObIntermResultItem(const char* label, uint64_t tenant_id)
: ObIIntermResultItem(),
allocator_(label, common::OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id, common::ObCtxIds::WORK_AREA),
data_buf_(NULL)
{}
ObIntermResultItem::~ObIntermResultItem()
{
reset();
}
void ObIntermResultItem::reset()
{
ObIIntermResultItem::reset();
data_buf_ = NULL;
allocator_.reset();
}
int ObIntermResultItem::assign(const ObIntermResultItem& other)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!other.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("other interm result interm is invalid", K(ret), K(other));
} else {
reset();
row_count_ = other.row_count_;
data_len_ = other.data_len_;
if (other.data_len_ > 0) {
if (OB_ISNULL(data_buf_ = static_cast<char*>(allocator_.alloc(other.data_len_)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc memory", K(ret), K(other.data_len_));
} else {
MEMCPY(data_buf_, other.data_buf_, other.data_len_);
}
}
}
return ret;
}
int ObIntermResultItem::from_scanner(const ObScanner& scanner)
{
int ret = OB_SUCCESS;
reset();
row_count_ = scanner.get_row_count();
data_len_ = scanner.get_serialize_size();
if (data_len_ > 0) {
int64_t pos = 0;
if (OB_ISNULL(data_buf_ = static_cast<char*>(allocator_.alloc(data_len_)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc memory", K(ret), K(data_len_));
} else if (OB_FAIL(scanner.serialize(data_buf_, data_len_, pos))) {
LOG_WARN("fail to serialize scanner", K(ret), K(data_len_), K(pos));
}
}
return ret;
}
int ObIntermResultItem::from_disk_ir_item(ObDiskIntermResultItem& disk_item)
{
int ret = OB_SUCCESS;
if (!disk_item.is_inited()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid disk interm result", K(ret), K(disk_item));
} else {
reset();
row_count_ = disk_item.get_row_count();
data_len_ = disk_item.get_data_len();
if (data_len_ > 0) {
if (OB_ISNULL(data_buf_ = static_cast<char*>(allocator_.alloc(data_len_)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc memory failed", K(ret), K_(data_len));
} else if (OB_FAIL(disk_item.copy_data(data_buf_, data_len_))) {
LOG_WARN("copy disk interm result item data to memory failed", K(ret));
}
}
}
return ret;
}
int ObIntermResultItem::to_scanner(ObScanner& scanner)
{
int ret = OB_SUCCESS;
scanner.reset();
if (data_len_ > 0) {
int64_t pos = 0;
if (OB_FAIL(scanner.deserialize(data_buf_, data_len_, pos))) {
LOG_WARN("fail to deserialize scanner", K(ret), K(pos), K(data_len_));
}
}
return ret;
}
int ObIntermResultItem::copy_data(char* buf, const int64_t size) const
{
int ret = OB_SUCCESS;
if (OB_ISNULL(buf) || size < data_len_) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalud argument", KP(buf), K(size), K_(data_len));
} else {
if (data_len_ > 0) {
MEMCPY(buf, data_buf_, data_len_);
}
}
return ret;
}
ObDiskIntermResultItem::ObDiskIntermResultItem()
: ObIIntermResultItem(), tenant_id_(0), fd_(-1), dir_id_(-1), offset_(0)
{}
ObDiskIntermResultItem::~ObDiskIntermResultItem()
{
reset();
}
int ObDiskIntermResultItem::init(uint64_t tenant_id, const int64_t fd, const int64_t dir_id, const int64_t offset)
{
int ret = OB_SUCCESS;
if (is_inited()) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", K(ret));
} else if (fd < 0 || offset < 0) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(fd), K(offset));
} else {
tenant_id_ = tenant_id;
fd_ = fd;
dir_id_ = dir_id;
offset_ = offset;
}
return ret;
}
void ObDiskIntermResultItem::reset()
{
ObIIntermResultItem::reset();
tenant_id_ = 0;
fd_ = -1;
dir_id_ = -1;
offset_ = 0;
}
int ObDiskIntermResultItem::from_scanner(const ObScanner& scanner)
{
int ret = OB_SUCCESS;
if (!is_inited()) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else {
row_count_ = scanner.get_row_count();
data_len_ = scanner.get_serialize_size();
blocksstable::ObTmpFileIOInfo io;
io.fd_ = fd_;
io.dir_id_ = dir_id_;
io.size_ = data_len_;
io.tenant_id_ = tenant_id_;
io.io_desc_.category_ = GCONF._large_query_io_percentage.get_value() > 0 ? common::LARGE_QUERY_IO : common::USER_IO;
io.io_desc_.wait_event_no_ = ObWaitEventIds::INTERM_RESULT_DISK_WRITE;
int64_t timeout_ms = 0;
if (data_len_ > 0) {
ObArenaAllocator allocator(
ObModIds::OB_SQL_EXECUTOR_INTERM_RESULT_ITEM, OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id_, ObCtxIds::WORK_AREA);
int64_t size = scanner.get_serialize_size();
char* buf = static_cast<char*>(allocator.alloc(size));
io.buf_ = buf;
int64_t pos = 0;
if (NULL == buf) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc memory failed", K(ret), K(size));
} else if (OB_FAIL(get_timeout(timeout_ms))) {
LOG_WARN("get timeout failed", K(ret));
} else if (OB_FAIL(scanner.serialize(buf, size, pos))) {
LOG_WARN("serialize scanner failed", K(ret), KP(buf), K(size));
} else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.write(io, timeout_ms))) {
LOG_WARN("write to disk failed", K(ret), K(io), K(timeout_ms));
}
}
}
return ret;
}
int ObDiskIntermResultItem::copy_data(char* buf, const int64_t size) const
{
int ret = OB_SUCCESS;
if (!is_inited()) {
ret = OB_NOT_INIT;
LOG_WARN("not init");
} else if (data_len_ > 0) {
if (NULL == buf || size < data_len_) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), KP(buf), K(size), K_(data_len));
} else {
int64_t timeout_ms = 0;
blocksstable::ObTmpFileIOInfo io;
io.fd_ = fd_;
io.dir_id_ = dir_id_;
io.buf_ = buf;
io.size_ = data_len_;
io.tenant_id_ = tenant_id_;
io.io_desc_.category_ =
GCONF._large_query_io_percentage.get_value() > 0 ? common::LARGE_QUERY_IO : common::USER_IO;
io.io_desc_.wait_event_no_ = ObWaitEventIds::INTERM_RESULT_DISK_READ;
blocksstable::ObTmpFileIOHandle handle;
if (OB_FAIL(get_timeout(timeout_ms))) {
LOG_WARN("get timeout failed", K(ret));
} else if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.pread(io, offset_, timeout_ms, handle))) {
LOG_WARN("read from disk failed", K(ret), K(io), K(timeout_ms));
} else if (handle.get_data_size() != data_len_) {
ret = OB_INNER_STAT_ERROR;
LOG_WARN("read data less than expected", K(ret), K(io), "read_size", handle.get_data_size());
}
}
}
return ret;
}
int ObDiskIntermResultItem::get_timeout(int64_t& timeout_ms)
{
int ret = OB_SUCCESS;
const int64_t timeout_us = THIS_WORKER.get_timeout_remain();
if (timeout_us / 1000 <= 0) {
ret = OB_TIMEOUT;
LOG_WARN("query is timeout", K(ret), K(timeout_us));
} else {
timeout_ms = timeout_us / 1000;
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,132 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_EXECUTOR_OB_INTERM_RESULT_ITEM_H_
#define OCEANBASE_EXECUTOR_OB_INTERM_RESULT_ITEM_H_
#include "share/ob_define.h"
#include "lib/allocator/page_arena.h"
#include "lib/utility/ob_print_utils.h"
namespace oceanbase {
namespace common {
class ObScanner;
}
namespace sql {
class ObIIntermResultItem {
public:
ObIIntermResultItem() : row_count_(0), data_len_(0)
{}
virtual ~ObIIntermResultItem()
{}
virtual bool in_memory() const = 0;
virtual void reset() = 0;
inline int64_t get_row_count() const
{
return row_count_;
}
inline int64_t get_data_len() const
{
return data_len_;
}
virtual int from_scanner(const common::ObScanner& scanner) = 0;
// copy data to buffer
virtual int copy_data(char* buf, const int64_t size) const = 0;
VIRTUAL_TO_STRING_KV(K_(row_count), K_(data_len));
protected:
int64_t row_count_;
int64_t data_len_;
DISALLOW_COPY_AND_ASSIGN(ObIIntermResultItem);
};
class ObDiskIntermResultItem;
class ObIntermResultItem : public ObIIntermResultItem {
OB_UNIS_VERSION(1);
public:
ObIntermResultItem(const char* label = common::ObModIds::OB_SQL_EXECUTOR_INTERM_RESULT_ITEM,
uint64_t tenant_id = common::OB_SERVER_TENANT_ID);
virtual ~ObIntermResultItem();
virtual bool in_memory() const override
{
return true;
}
virtual void reset() override;
int assign(const ObIntermResultItem& other);
virtual int from_scanner(const common::ObScanner& scanner) override;
int from_disk_ir_item(ObDiskIntermResultItem& disk_item);
int to_scanner(common::ObScanner& scanner);
inline bool is_valid() const
{
return data_len_ <= 0 || NULL != data_buf_;
}
inline const char* get_data_buf() const
{
return data_buf_;
}
virtual int copy_data(char* buf, const int64_t size) const override;
INHERIT_TO_STRING_KV("iinterm_result", ObIIntermResultItem, KP_(data_buf));
private:
common::ObArenaAllocator allocator_;
char* data_buf_;
private:
DISALLOW_COPY_AND_ASSIGN(ObIntermResultItem);
};
class ObDiskIntermResultItem : public ObIIntermResultItem {
public:
ObDiskIntermResultItem();
virtual ~ObDiskIntermResultItem();
virtual bool in_memory() const override
{
return false;
}
int init(const uint64_t tenant_id, const int64_t fd, const int64_t dir_id_, const int64_t offset);
bool is_inited() const
{
return fd_ >= 0;
}
virtual void reset() override;
virtual int from_scanner(const common::ObScanner& scanner) override;
virtual int copy_data(char* buf, const int64_t size) const override;
static int get_timeout(int64_t& timeout_ms);
INHERIT_TO_STRING_KV("iinterm_result", ObIIntermResultItem, K_(fd), K_(offset), K_(tenant_id));
private:
uint64_t tenant_id_;
int64_t fd_;
int64_t dir_id_;
int64_t offset_;
DISALLOW_COPY_AND_ASSIGN(ObDiskIntermResultItem);
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_EXECUTOR_OB_INTERM_RESULT_ITEM_H_

View File

@ -0,0 +1,161 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_interm_result_item_pool.h"
#include "sql/executor/ob_task_event.h"
#include "lib/alloc/alloc_func.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObIntermResultItemPool* ObIntermResultItemPool::instance_ = NULL;
ObIntermResultItemPool::ObIntermResultItemPool() : inited_(false), mem_item_allocator_(), disk_item_allocator_()
{}
ObIntermResultItemPool::~ObIntermResultItemPool()
{
reset();
}
void ObIntermResultItemPool::reset()
{
int ret = OB_SUCCESS;
inited_ = false;
if (OB_FAIL(mem_item_allocator_.destroy())) {
LOG_ERROR("fail to destroy allocator", K(ret));
} else if (OB_FAIL(disk_item_allocator_.destroy())) {
LOG_ERROR("fail to destroy allocator", K(ret));
}
}
int ObIntermResultItemPool::build_instance()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(NULL != instance_)) {
ret = OB_INIT_TWICE;
LOG_ERROR("instance is not NULL, build twice", K(ret));
} else if (OB_ISNULL(instance_ = OB_NEW(ObIntermResultItemPool, ObModIds::OB_SQL_EXECUTOR))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("instance is NULL, unexpected", K(ret));
} else if (OB_FAIL(instance_->init())) {
instance_->reset();
OB_DELETE(ObIntermResultItemPool, ObModIds::OB_SQL_EXECUTOR, instance_);
instance_ = NULL;
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to init scanner pool", K(ret));
} else {
}
return ret;
}
ObIntermResultItemPool* ObIntermResultItemPool::get_instance()
{
ObIntermResultItemPool* instance = NULL;
if (OB_ISNULL(instance_) || OB_UNLIKELY(!instance_->inited_)) {
LOG_ERROR("instance is NULL or not inited", K(instance_));
} else {
instance = instance_;
}
return instance;
}
int ObIntermResultItemPool::init()
{
const static int64_t block_size = OB_MALLOC_NORMAL_BLOCK_SIZE;
int ret = OB_SUCCESS;
if (OB_UNLIKELY(inited_)) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", K(ret));
} else if (OB_FAIL(mem_item_allocator_.init(sizeof(ObIntermResultItem),
ObModIds::OB_SQL_EXECUTOR,
OB_SERVER_TENANT_ID,
block_size,
1,
get_capacity()))) {
LOG_WARN("fail to init allocator", K(ret), "capacity", get_capacity());
} else if (OB_FAIL(disk_item_allocator_.init(sizeof(ObDiskIntermResultItem),
ObModIds::OB_SQL_EXECUTOR,
OB_SERVER_TENANT_ID,
block_size,
1,
get_capacity()))) {
LOG_WARN("fail to init allocator", K(ret), "capacity", get_capacity());
} else {
inited_ = true;
LOG_INFO("initialize scanner pool", "size", get_capacity());
}
return ret;
}
int ObIntermResultItemPool::alloc_mem_item(ObIntermResultItem*& ir_item, const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
void* ir_item_ptr = NULL;
if (OB_ISNULL(ir_item_ptr = mem_item_allocator_.alloc())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc scanner from obj pool", K(ret));
} else if (OB_ISNULL(ir_item = new (ir_item_ptr)
ObIntermResultItem(ObModIds::OB_SQL_EXECUTOR_INTERM_RESULT_ITEM, tenant_id))) {
LOG_WARN("fail to new ObIntermResultItem", K(ret), K(tenant_id));
}
return ret;
}
int ObIntermResultItemPool::alloc_disk_item(ObDiskIntermResultItem*& item, const uint64_t tenant_id, const int64_t fd,
const int64_t dir_id, const int64_t offset)
{
int ret = OB_SUCCESS;
if (!inited_) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else {
void* mem = disk_item_allocator_.alloc();
if (OB_ISNULL(mem)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("failed to alloc disk interim result item", K(ret));
} else {
ObDiskIntermResultItem* it = new (mem) ObDiskIntermResultItem();
if (OB_FAIL(it->init(tenant_id, fd, dir_id, offset))) {
LOG_WARN("init disk interim result item failed", K(ret), K(tenant_id), K(fd), K(offset));
it->~ObDiskIntermResultItem();
it = NULL;
disk_item_allocator_.free(mem);
mem = NULL;
} else {
item = it;
}
}
}
return ret;
}
int ObIntermResultItemPool::free(ObIIntermResultItem* item)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(item)) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("interm result item is NULL", K(ret));
} else {
ObSmallAllocator& allocator = get_allocator(item->in_memory());
item->~ObIIntermResultItem();
allocator.free(item);
item = NULL;
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,67 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_ITEM_POOL_
#define OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_ITEM_POOL_
#include "lib/allocator/ob_small_allocator.h"
#include "sql/executor/ob_interm_result_pool.h"
namespace oceanbase {
namespace sql {
class ObIIntermResultItem;
class ObIntermResultItem;
class ObDiskIntermResultItem;
class ObIntermResultItemPool {
public:
static const int64_t MAX_INTERM_RESULT_ITEM_POOL_CAPACITY = ObIntermResultPool::INTERM_RESULT_CAPACITY;
ObIntermResultItemPool();
virtual ~ObIntermResultItemPool();
static int build_instance();
static ObIntermResultItemPool* get_instance();
void reset();
int alloc_mem_item(ObIntermResultItem*& item, const uint64_t tenant_id);
int alloc_disk_item(ObDiskIntermResultItem*& item, const uint64_t tenant_id, const int64_t fd, const int64_t dir_id,
const int64_t offset);
int free(ObIIntermResultItem* item);
inline static int64_t get_capacity()
{
return MAX_INTERM_RESULT_ITEM_POOL_CAPACITY;
}
private:
int init();
inline common::ObSmallAllocator& get_allocator(const bool is_memory_item)
{
return is_memory_item ? mem_item_allocator_ : disk_item_allocator_;
}
private:
static ObIntermResultItemPool* instance_;
bool inited_;
common::ObSmallAllocator mem_item_allocator_;
common::ObSmallAllocator disk_item_allocator_;
private:
DISALLOW_COPY_AND_ASSIGN(ObIntermResultItemPool);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_ITEM_POOL_ */

View File

@ -0,0 +1,437 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_interm_result_manager.h"
#include "sql/executor/ob_interm_result_pool.h"
#include "share/ob_debug_sync.h"
#include "share/ob_thread_mgr.h"
#include "sql/executor/ob_interm_result_item.h"
namespace oceanbase {
namespace sql {
using namespace oceanbase::common;
using namespace common::hash;
class ObUpdateIRExpireTime {
public:
ObUpdateIRExpireTime(const int64_t expire_time) : ret_(OB_SUCCESS), expire_time_(expire_time)
{}
void operator()(common::hash::HashMapPair<ObIntermResultInfo, ObIntermResult*>& entry)
{
if (OB_ISNULL(entry.second)) {
ret_ = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret_));
} else {
entry.second->set_expire_time(expire_time_);
}
}
public:
int ret_;
private:
const int64_t expire_time_;
};
ObIntermResultManager* ObIntermResultManager::instance_ = NULL;
ObIntermResultGC::ObIntermResultGC()
: ir_map_(NULL),
ir_manager_(NULL),
cur_time_(0),
allocator_(ObModIds::OB_SQL_EXECUTOR_INTERM_RESULT_EXPIRE_IR),
expire_irs_(allocator_),
invalid_time_ir_count_(0)
{}
ObIntermResultGC::~ObIntermResultGC()
{}
void ObIntermResultGC::reset()
{
ir_map_ = NULL;
ir_manager_ = NULL;
cur_time_ = 0;
expire_irs_.reset();
invalid_time_ir_count_ = 0;
// reset allocator in the end
allocator_.reset();
}
void ObIntermResultGC::operator()(common::hash::HashMapPair<ObIntermResultInfo, ObIntermResult*>& entry)
{
if (OB_ISNULL(entry.second)) {
LOG_ERROR("null ptr");
} else {
int ret = OB_SUCCESS;
int64_t time_diff = entry.second->get_expire_time() - cur_time_;
if (OB_UNLIKELY(entry.second->get_expire_time() <= 0)) {
LOG_ERROR("invalid ir expire time. skip gc", "time", entry.second->get_expire_time());
} else {
if (time_diff < 0 && OB_SUCC(entry.second->try_begin_recycle())) {
if (OB_FAIL(expire_irs_.push_back(entry))) {
LOG_ERROR("fail to push back to expire_irs_", K(ret), K(entry), K(expire_irs_.size()));
}
} else if (time_diff > 1800000000) {
invalid_time_ir_count_++;
}
}
}
}
void ObIntermResultGC::runTimerTask()
{
static int64_t gc_run_count = 0;
int ret = common::OB_SUCCESS;
common::hash::HashMapPair<ObIntermResultInfo, ObIntermResult*> entry;
if (OB_ISNULL(ir_map_) || OB_ISNULL(ir_manager_)) {
LOG_WARN("the interm result map of the GC class is NULL", K_(ir_map), K_(ir_manager));
} else {
expire_irs_.reset();
allocator_.reset();
cur_time_ = ::oceanbase::common::ObTimeUtility::current_time();
ir_map_->foreach_refactored(*this);
if (ir_map_->size() > 0 || expire_irs_.size() > 0 || 0 == gc_run_count % 900) {
LOG_INFO("Interm result recycle",
"total_interm_result_count",
ir_map_->size(),
"life_too_long_interm_result_count",
invalid_time_ir_count_,
"ready_to_free_interm_result_count",
expire_irs_.size());
}
while (OB_SUCC(expire_irs_.pop_front(entry))) {
if (OB_FAIL(ir_manager_->free_result(entry.first))) {
if (common::OB_NEED_RETRY == ret) {
LOG_DEBUG("free mr result failed, need retry");
} else {
LOG_WARN("free mr result failed", K(ret));
}
} else {
LOG_DEBUG("free mr result success");
}
}
invalid_time_ir_count_ = 0;
expire_irs_.reset();
allocator_.reset();
}
gc_run_count++;
}
void ObIntermResultGC::set_ir_map(common::hash::ObHashMap<ObIntermResultInfo, ObIntermResult*>* ir_map)
{
ir_map_ = ir_map;
}
void ObIntermResultGC::set_ir_manager(ObIntermResultManager* ir_manager)
{
ir_manager_ = ir_manager;
}
ObIntermResultManager::ObIntermResultManager()
: inited_(false), ir_map_(), ir_gc_(), gc_delay_time_(DEFAULT_INTERM_RESULT_GC_DELAY_TIME), ir_pool_(NULL)
{}
ObIntermResultManager::~ObIntermResultManager()
{
reset();
}
void ObIntermResultManager::reset()
{
int ret = OB_SUCCESS;
if (OB_FAIL(ir_map_.clear())) {
LOG_ERROR("fail to clear interm result map", K(ret));
}
TG_DESTROY(lib::TGDefIDs::IntermResGC);
ir_gc_.reset();
gc_delay_time_ = DEFAULT_INTERM_RESULT_GC_DELAY_TIME;
ir_pool_ = NULL;
inited_ = false;
}
int ObIntermResultManager::build_instance()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(NULL != instance_)) {
ret = OB_INIT_TWICE;
LOG_ERROR("instance is not NULL, build twice", K(ret));
} else if (OB_UNLIKELY(NULL == (instance_ = OB_NEW(ObIntermResultManager, ObModIds::OB_SQL_EXECUTOR)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("instance is NULL, unexpected", K(ret));
} else if (OB_FAIL(instance_->init())) {
OB_DELETE(ObIntermResultManager, ObModIds::OB_SQL_EXECUTOR, instance_);
instance_ = NULL;
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("fail to init interm result manager", K(ret));
}
return ret;
}
ObIntermResultManager* ObIntermResultManager::get_instance()
{
ObIntermResultManager* instance = NULL;
if (OB_UNLIKELY(OB_ISNULL(instance_) || !instance_->inited_)) {
LOG_ERROR("instance is NULL or not inited", K(instance_));
} else {
instance = instance_;
}
return instance;
}
int ObIntermResultManager::init()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(inited_)) {
ret = OB_INIT_TWICE;
} else if (OB_ISNULL(ir_pool_ = ObIntermResultPool::get_instance())) {
ret = OB_ERR_UNEXPECTED;
_OB_LOG(ERROR, "fail to get iterm result pool instance");
} else if (OB_FAIL(ir_map_.create(INTERM_RMAP_BUCKET_SIZE, ObModIds::OB_SQL_EXECUTOR_INTERM_RESULT_MAP))) {
LOG_WARN("fail to create ir map", K(ret));
} else if (OB_FAIL(TG_START(lib::TGDefIDs::IntermResGC))) {
LOG_WARN("fail to init timer", K(ret));
} else {
ir_gc_.set_ir_map(&ir_map_);
ir_gc_.set_ir_manager(this);
if (OB_FAIL(TG_SCHEDULE(lib::TGDefIDs::IntermResGC, ir_gc_, gc_delay_time_, true))) {
LOG_WARN("fail to schedule timer", K(ret));
} else {
inited_ = true;
}
}
return ret;
}
int ObIntermResultManager::update_expire_time(const ObIntermResultInfo& ir_info, const int64_t expire_time)
{
int ret = OB_SUCCESS;
if (!inited_) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (!ir_info.is_init() || expire_time <= 0) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(ir_info), K(expire_time));
} else {
ObUpdateIRExpireTime updater(expire_time);
if (OB_FAIL(ir_map_.atomic_refactored(ir_info, updater))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_ENTRY_NOT_EXIST;
} else {
LOG_WARN("hash table atomic failed", K(ret), K(ir_info));
}
} else if (OB_SUCCESS != updater.ret_) {
ret = updater.ret_;
LOG_WARN("update expire time failed", K(ret));
}
}
return ret;
}
int ObIntermResultManager::get_result(const ObIntermResultInfo& ir_info, ObIntermResultIterator& iter)
{
int ret = OB_SUCCESS;
ObIntermResultRead ir_read;
// ir_read try increase reference count of the map result
if (OB_SUCCESS == (ret = ir_map_.atomic_refactored(ir_info, ir_read))) {
ret = ir_read.get_ret();
if (OB_SUCC(ret)) {
// notify the iter that the reference count of the interm result is increased
if (OB_FAIL(iter.set_interm_result(ir_info, ir_read.get_value(), true))) {
LOG_ERROR("fail to set interm result", K(ret));
} else {
LOG_DEBUG("get interm result");
}
} else {
// fail to increase the reference count, the interm result must be recycling
// ret = common::OB_ENTRY_NOT_EXIST;
LOG_WARN("can not increase the cnt, the interm result should be recycling.", K(ret));
}
} else if (OB_HASH_NOT_EXIST == ret) {
ret = common::OB_ENTRY_NOT_EXIST;
LOG_DEBUG("the required interm result is not exist.", K(ret));
} else {
LOG_WARN("cannot get the interm result", K(ret));
}
return ret;
}
int ObIntermResultManager::get_result_item(
const ObIntermResultInfo& ir_info, const int64_t index, ObIntermResultItem& result_item, int64_t& total_cnt)
{
int ret = OB_SUCCESS;
ObIntermResultRead ir_read;
if (OB_FAIL(ir_map_.atomic_refactored(ir_info, ir_read))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_ENTRY_NOT_EXIST;
LOG_DEBUG("the request item result is not exist", K(ret));
} else {
LOG_WARN("get interm result failed", K(ret), K(ir_info));
}
} else if (OB_FAIL(ir_read.get_ret())) {
LOG_WARN("interm result read failed", K(ret), K(ir_info));
} else if (OB_ISNULL(ir_read.get_value())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL item result", K(ret));
} else {
ObIntermResult* ir = ir_read.get_value();
total_cnt = ir->get_scanner_count();
ObIIntermResultItem* item = NULL;
if (total_cnt > 0) {
if (OB_FAIL(ir->get_item(index, item))) {
LOG_WARN("get item failed", K(ret));
} else if (OB_ISNULL(item)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("item is NULL", K(ret));
} else {
if (item->in_memory()) {
if (OB_FAIL(result_item.assign(*static_cast<ObIntermResultItem*>(item)))) {
LOG_WARN("interm result item assign failed", K(ret));
}
} else {
if (OB_FAIL(result_item.from_disk_ir_item(*static_cast<ObDiskIntermResultItem*>(item)))) {
LOG_WARN("from disk interm result item failed", K(ret));
}
}
}
}
int tmp_ret = ir->try_dec_cnt();
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("try decrease reference count failed", K(tmp_ret));
if (OB_SUCC(ret)) {
ret = tmp_ret;
}
}
}
return ret;
}
int ObIntermResultManager::add_result(
const ObIntermResultInfo& ir_info, ObIntermResult* interm_result, int64_t expire_time)
{
int ret = OB_SUCCESS;
interm_result->set_expire_time(expire_time);
if (OB_UNLIKELY(!interm_result->rows_is_completed())) {
LOG_WARN("rows in this interm result is not completed", K(ret));
} else if (OB_FAIL(ir_map_.set_refactored(ir_info, interm_result))) {
LOG_WARN("fail to set interm result to map", K(ret));
}
return ret;
}
int ObIntermResultManager::delete_result(const ObIntermResultInfo& ir_info)
{
DEBUG_SYNC(BEFORE_RECYCLE_INTERM_RESULT);
int ret = OB_SUCCESS;
ObIntermResultRecycle ir_recycle;
// ir_recycle try begin recycle for the map result
if (OB_SUCCESS == (ret = ir_map_.atomic_refactored(ir_info, ir_recycle))) {
ret = ir_recycle.get_ret();
if (OB_SUCC(ret)) {
// has begin recycle
free_result(ir_info);
LOG_DEBUG("free interm result");
} else {
// fail to begin recycle
// ret = common::OB_ENTRY_NOT_EXIST;
LOG_WARN("fail to recycle, maybe it has being recycled or some iterator is reading it", K(ret));
}
} else if (OB_HASH_NOT_EXIST == ret) {
ret = common::OB_ENTRY_NOT_EXIST;
LOG_DEBUG("the required interm result is not exist.");
} else {
LOG_WARN("cannot recycle the interm result", K(ret));
}
return ret;
}
int ObIntermResultManager::delete_result(ObIntermResultIterator& iter)
{
int ret = OB_SUCCESS;
ObIntermResultInfo ir_info;
ObIntermResult* ir = iter.ir_;
if (OB_ISNULL(ir)) {
ret = OB_INVALID_ARGUMENT;
} else if (OB_FAIL(iter.get_interm_result_info(ir_info))) {
LOG_WARN("fail to get interm result info from iterator", K(ret));
} else {
iter.reset();
if (OB_FAIL(delete_result(ir_info))) {
LOG_WARN("fail to delete result", K(ret), K(ir_info));
}
}
return ret;
}
int ObIntermResultManager::alloc_result(ObIntermResult*& interm_result)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(ir_pool_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("null ptr", K_(ir_pool), K(ret));
} else {
ret = ir_pool_->alloc_interm_result(interm_result);
}
return ret;
}
int ObIntermResultManager::free_result(ObIntermResult* interm_result)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(ir_pool_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("null ptr", K_(ir_pool), K(ret));
} else {
interm_result->reset();
ret = ir_pool_->free_interm_result(interm_result);
}
return ret;
}
int ObIntermResultManager::free_result(const ObIntermResultInfo& ir_info)
{
int ret = OB_SUCCESS;
ObIntermResult* ir = NULL;
if (OB_ISNULL(ir_pool_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("null ptr", K_(ir_pool), K(ret));
} else if (OB_FAIL(ir_map_.erase_refactored(ir_info, &ir))) {
LOG_WARN("erase interm result from map failed", K(ret));
} else if (OB_ISNULL(ir)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("erase success but the interm result pointer is NULL", K(ret));
} else {
ir->reset();
ret = ir_pool_->free_interm_result(ir);
LOG_DEBUG("free interm result", "slice_id", ir_info.slice_id_);
}
return ret;
}
const common::hash::ObHashMap<ObIntermResultInfo, ObIntermResult*>& ObIntermResultManager::get_ir_map() const
{
return ir_map_;
}
} /* namespace sql */
} /* namespace oceanbase */

View File

@ -0,0 +1,166 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_MANAGER_
#define OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_MANAGER_
#include "share/ob_scanner.h"
#include "sql/executor/ob_interm_result.h"
#include "lib/lock/ob_spin_lock.h"
#include "lib/task/ob_timer.h"
namespace oceanbase {
namespace sql {
class ObIntermResultPool;
class ObIntermResultManager;
class ObIntermResultRead {
public:
ObIntermResultRead() : ret_(common::OB_SUCCESS), value_(NULL)
{}
virtual ~ObIntermResultRead()
{}
void operator()(common::hash::HashMapPair<ObIntermResultInfo, ObIntermResult*>& entry)
{
if (OB_ISNULL(entry.second)) {
ret_ = common::OB_INVALID_ARGUMENT;
} else {
ret_ = entry.second->try_inc_cnt();
value_ = entry.second;
}
}
int get_ret() const
{
return ret_;
}
ObIntermResult* get_value()
{
return value_;
}
private:
int ret_;
ObIntermResult* value_;
};
class ObIntermResultRecycle {
public:
ObIntermResultRecycle() : ret_(common::OB_SUCCESS), value_(NULL)
{}
virtual ~ObIntermResultRecycle()
{}
void operator()(common::hash::HashMapPair<ObIntermResultInfo, ObIntermResult*>& entry)
{
if (OB_ISNULL(entry.second)) {
ret_ = common::OB_INVALID_ARGUMENT;
} else {
ret_ = entry.second->try_begin_recycle();
value_ = entry.second;
}
}
int get_ret() const
{
return ret_;
}
ObIntermResult* get_value()
{
return value_;
}
private:
int ret_;
ObIntermResult* value_;
};
class ObIntermResultGC : public common::ObTimerTask {
public:
ObIntermResultGC();
virtual ~ObIntermResultGC();
void reset();
void operator()(common::hash::HashMapPair<ObIntermResultInfo, ObIntermResult*>& entry);
void runTimerTask();
void set_ir_map(common::hash::ObHashMap<ObIntermResultInfo, ObIntermResult*>* ir_map);
void set_ir_manager(ObIntermResultManager* ir_manager);
private:
DISALLOW_COPY_AND_ASSIGN(ObIntermResultGC);
private:
// ir map
common::hash::ObHashMap<ObIntermResultInfo, ObIntermResult*>* ir_map_;
// ir manager
ObIntermResultManager* ir_manager_;
// current time
int64_t cur_time_;
// list to store expire interm results;
common::ObArenaAllocator allocator_;
common::ObList<common::hash::HashMapPair<ObIntermResultInfo, ObIntermResult*>, common::ObArenaAllocator> expire_irs_;
// how many interm results that have too long expire time
int64_t invalid_time_ir_count_;
};
class ObIntermResultManager {
public:
friend class ObIntermResultGC;
static const int64_t DEFAULT_INTERM_RESULT_GC_DELAY_TIME = 1000000;
static const int64_t INTERM_RMAP_BUCKET_SIZE = 1024;
ObIntermResultManager();
virtual ~ObIntermResultManager();
static int build_instance();
static ObIntermResultManager* get_instance();
void reset();
// return OB_ENTRY_NOT_EXIST for non-exist interm result.
int update_expire_time(const ObIntermResultInfo& ir_info, const int64_t expire_time);
int get_result(const ObIntermResultInfo& ir_info, ObIntermResultIterator& iter);
int get_result_item(
const ObIntermResultInfo& ir_info, const int64_t index, ObIntermResultItem& result_item, int64_t& total_cnt);
int add_result(const ObIntermResultInfo& ir_info, ObIntermResult* interm_result, int64_t expire_time);
int delete_result(const ObIntermResultInfo& ir_info);
int delete_result(ObIntermResultIterator& iter);
int alloc_result(ObIntermResult*& interm_result);
int free_result(ObIntermResult* interm_result);
const common::hash::ObHashMap<ObIntermResultInfo, ObIntermResult*>& get_ir_map() const;
private:
int init();
int free_result(const ObIntermResultInfo& ir_info);
DISALLOW_COPY_AND_ASSIGN(ObIntermResultManager);
private:
static ObIntermResultManager* instance_;
bool inited_;
common::hash::ObHashMap<ObIntermResultInfo, ObIntermResult*> ir_map_;
// GC
ObIntermResultGC ir_gc_;
// gc time delay
int64_t gc_delay_time_;
// interm result pool
ObIntermResultPool* ir_pool_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_INTERM_RESULT_MANAGER_ */
//// end of header file

View File

@ -0,0 +1,166 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "ob_interm_result_pool.h"
#include "ob_interm_result.h"
namespace oceanbase {
namespace sql {
using namespace common;
ObIntermResultPool* ObIntermResultPool::instance_ = NULL;
ObIntermResultPool::ObIntermResultPool() : inited_(false), allocator_(), scanner_allocator_()
{}
ObIntermResultPool::~ObIntermResultPool()
{}
void ObIntermResultPool::reset()
{
int ret = OB_SUCCESS;
if (OB_FAIL(allocator_.destroy())) {
LOG_ERROR("destroy allocator failed", K(ret));
}
if (OB_FAIL(scanner_allocator_.destroy())) {
LOG_ERROR("destroy allocator failed", K(ret));
}
inited_ = false;
}
int ObIntermResultPool::build_instance()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(NULL != instance_)) {
ret = OB_INIT_TWICE;
LOG_ERROR("instance is not NULL, build twice", K(ret));
} else if (OB_ISNULL(instance_ = OB_NEW(ObIntermResultPool, ObModIds::OB_SQL_EXECUTOR))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("instance is NULL, unexpected", K(ret));
} else if (OB_FAIL(instance_->init())) {
OB_DELETE(ObIntermResultPool, ObModIds::OB_SQL_EXECUTOR, instance_);
instance_ = NULL;
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to init interm result pool", K(ret));
} else {
}
return ret;
}
ObIntermResultPool* ObIntermResultPool::get_instance()
{
ObIntermResultPool* instance = NULL;
if (OB_ISNULL(instance_) || OB_UNLIKELY(!instance_->inited_)) {
LOG_ERROR("instance is NULL or not inited", K(instance_));
} else {
instance = instance_;
}
return instance;
}
int ObIntermResultPool::init()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(inited_)) {
ret = OB_INIT_TWICE;
LOG_WARN("interm result pool init twice", K(ret));
} else if (OB_FAIL(allocator_.init(sizeof(ObIntermResult),
ObModIds::OB_SQL_EXECUTOR,
OB_SERVER_TENANT_ID,
OB_MALLOC_NORMAL_BLOCK_SIZE,
1,
INTERM_RESULT_CAPACITY))) {
LOG_WARN(
"inter interm result allocator failed", K(ret), K(sizeof(ObIntermResult)), LITERAL_K(INTERM_RESULT_CAPACITY));
} else if (OB_FAIL(scanner_allocator_.init(sizeof(ObScanner),
ObModIds::OB_SQL_EXECUTOR,
OB_SERVER_TENANT_ID,
OB_MALLOC_MIDDLE_BLOCK_SIZE,
1,
SCANNER_CAPACITY))) {
LOG_WARN("init scanner allocator failed", K(ret), K(sizeof(ObScanner)), LITERAL_K(SCANNER_CAPACITY));
} else {
inited_ = true;
LOG_INFO("initialize interm result pool", LITERAL_K(INTERM_RESULT_CAPACITY), LITERAL_K(SCANNER_CAPACITY));
}
return ret;
}
int ObIntermResultPool::alloc_interm_result(ObIntermResult*& interm_result)
{
int ret = OB_SUCCESS;
void* buf = NULL;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_ISNULL(buf = allocator_.alloc())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc obj", K(ret));
} else {
interm_result = new (buf) ObIntermResult();
}
return ret;
}
int ObIntermResultPool::free_interm_result(ObIntermResult* interm_result)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(interm_result)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("the free interm result is NULL", K(ret));
} else if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else {
if (ObIntermResult::STATE_NORMAL != interm_result->get_state()) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("the state of the free interm result is not STATE_NORMAL, can not free it", K(ret));
} else {
interm_result->~ObIntermResult();
allocator_.free(interm_result);
}
}
return ret;
}
int ObIntermResultPool::alloc_scanner(common::ObScanner*& scanner)
{
int ret = OB_SUCCESS;
void* buf = NULL;
if (!inited_) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_ISNULL(buf = scanner_allocator_.alloc())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc scanner failed", K(ret));
} else {
scanner = new (buf) ObScanner(ObModIds::OB_SQL_EXECUTOR_INTERM_RESULT_SCANNER);
STATIC_ASSERT(
SCANNER_MEM_LIMIT <= common::ObScanner::DEFAULT_MAX_SERIALIZE_SIZE, "scanner exceed max serialize size");
scanner->set_mem_size_limit(SCANNER_MEM_LIMIT);
}
return ret;
}
void ObIntermResultPool::free_scanner(common::ObScanner* scanner)
{
if (NULL != scanner && inited_) {
scanner->~ObScanner();
scanner_allocator_.free(scanner);
}
}
} /* namespace sql */
} /* namespace oceanbase */

View File

@ -0,0 +1,63 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_INTERM_RESULT_POOL_
#define OCEANBASE_SQL_EXECUTOR_INTERM_RESULT_POOL_
#include "lib/lock/ob_spin_lock.h"
#include "lib/allocator/ob_small_allocator.h"
namespace oceanbase {
namespace common {
class ObScanner;
}
namespace sql {
class ObIntermResult;
class ObIntermResultPool {
public:
// Assume than one interm result is 1MB and we need manage 10TB intermediate data,
// we need 10TB/1MB = 10M interm result.
static const int64_t INTERM_RESULT_CAPACITY = 10L << 20; // 10M
// Scanner count is much less than interm result count, because it will be freed
// before interm result add to manager.
static const int64_t SCANNER_CAPACITY = 256L << 10; // 256K
static const int64_t SCANNER_MEM_LIMIT = 8 << 20; // 8MB
ObIntermResultPool();
virtual ~ObIntermResultPool();
static int build_instance();
static ObIntermResultPool* get_instance();
void reset();
int alloc_interm_result(ObIntermResult*& interm_result);
int free_interm_result(ObIntermResult* interm_result);
int alloc_scanner(common::ObScanner*& scanner);
void free_scanner(common::ObScanner* scanner);
private:
int init();
private:
static ObIntermResultPool* instance_;
// this interm result pool is initialized
bool inited_;
// the small allocator
common::ObSmallAllocator allocator_;
common::ObSmallAllocator scanner_allocator_;
DISALLOW_COPY_AND_ASSIGN(ObIntermResultPool);
};
} /* namespace sql */
} /* namespace oceanbase */
#endif /* OCEANBASE_SQL_EXECUTOR_INTERM_RESULT_POOL_ */

View File

@ -0,0 +1,102 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "ob_interm_task_spliter.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/executor/ob_transmit.h"
#include "lib/utility/ob_tracepoint.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObIntermTaskSpliter::ObIntermTaskSpliter()
: prepare_done_flag_(false), next_task_idx_(0), total_task_count_(0), store_()
{}
ObIntermTaskSpliter::~ObIntermTaskSpliter()
{
for (int64_t i = 0; i < store_.count(); ++i) {
ObTaskInfo* t = store_.at(i);
if (OB_LIKELY(NULL != t)) {
t->~ObTaskInfo();
}
}
}
int ObIntermTaskSpliter::prepare()
{
int ret = OB_SUCCESS;
ObPhyOperator* phy_op = NULL;
ObTransmit* transmit_op = NULL;
prepare_done_flag_ = false;
if (OB_I(t1)(OB_ISNULL(plan_ctx_) || OB_ISNULL(allocator_) || OB_ISNULL(job_) || OB_ISNULL(job_conf_))) {
ret = OB_NOT_INIT;
LOG_WARN("param not init", K_(plan_ctx), K_(allocator), K_(job), K_(job_conf));
} else if (OB_I(t2)(OB_UNLIKELY(NULL == (phy_op = job_->get_root_op())) || (!IS_TRANSMIT(phy_op->get_type())))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op is null or not transmit", K(phy_op));
} else {
transmit_op = static_cast<ObTransmit*>(phy_op);
next_task_idx_ = 0;
total_task_count_ = transmit_op->get_split_task_count();
// mark as done
prepare_done_flag_ = true;
}
return ret;
}
int ObIntermTaskSpliter::get_next_task(ObTaskInfo*& task)
{
int ret = OB_SUCCESS;
void* ptr = NULL;
if (OB_UNLIKELY(false == prepare_done_flag_)) {
ret = prepare();
}
// after success prepare
if (OB_SUCC(ret)) {
if (next_task_idx_ >= total_task_count_) {
ret = OB_ITER_END;
} else {
if (OB_I(t1)(OB_UNLIKELY(NULL == (ptr = allocator_->alloc(sizeof(ObTaskInfo)))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail allocate task");
} else {
ObTaskInfo* t = new (ptr) ObTaskInfo(*allocator_);
if (OB_FAIL(store_.push_back(t))) {
LOG_WARN("fail to push taskinfo into store", K(ret));
} else {
ObTaskID ob_task_id;
ObTaskLocation task_loc;
ob_task_id.set_ob_job_id(job_->get_ob_job_id());
ob_task_id.set_task_id(next_task_idx_);
task_loc.set_ob_task_id(ob_task_id);
task_loc.set_server(server_); // ObTaskControl will rewrite server later.
t->set_task_split_type(get_type());
t->set_pull_slice_id(next_task_idx_);
t->set_task_location(task_loc);
t->set_root_op(job_->get_root_op());
t->set_state(OB_TASK_STATE_NOT_INIT);
task = t;
// move to next info
next_task_idx_++;
}
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,48 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_INTERM_TASK_SPLITER_
#define OCEANBASE_SQL_EXECUTOR_INTERM_TASK_SPLITER_
#include "sql/executor/ob_task_spliter.h"
#include "lib/container/ob_se_array.h"
namespace oceanbase {
namespace sql {
class ObIntermTaskSpliter : public ObTaskSpliter {
public:
ObIntermTaskSpliter();
virtual ~ObIntermTaskSpliter();
virtual int get_next_task(ObTaskInfo*& task);
virtual TaskSplitType get_type() const
{
return ObTaskSpliter::INTERM_SPLIT;
}
private:
/* functions */
int prepare();
private:
/* variables */
bool prepare_done_flag_;
int64_t next_task_idx_;
int64_t total_task_count_;
common::ObSEArray<ObTaskInfo*, 8> store_;
private:
DISALLOW_COPY_AND_ASSIGN(ObIntermTaskSpliter);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_INTERM_TASK_SPLITER_ */
//// end of header file

642
src/sql/executor/ob_job.cpp Normal file
View File

@ -0,0 +1,642 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_job.h"
#include "sql/executor/ob_task_event.h"
#include "sql/executor/ob_task_spliter.h"
#include "sql/executor/ob_addrs_provider.h"
#include "sql/executor/ob_transmit.h"
#include "sql/executor/ob_receive.h"
#include "share/ob_define.h"
#include "lib/utility/ob_tracepoint.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/table/ob_table_scan.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/px/ob_px_util.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
using namespace oceanbase::share;
int64_t ObMiniJob::to_string(char* buf, int64_t buf_len) const
{
int64_t pos = 0;
J_OBJ_START();
J_NAME("root_op");
J_COLON();
ObPhysicalPlan::print_tree(buf, buf_len, pos, root_op_);
J_COMMA();
J_NAME("extend_op");
J_COLON();
ObPhysicalPlan::print_tree(buf, buf_len, pos, extend_op_);
J_OBJ_END();
return pos;
}
ObJob::ObJob()
: ob_job_id_(),
is_root_job_(false),
phy_plan_(NULL),
root_op_(NULL),
state_(OB_JOB_STATE_NOT_INIT),
task_spliter_(NULL),
task_splited_(false),
task_control_(),
is_outer_join_child_job_(false),
has_outer_join_child_scan_(false),
has_scan_(false)
{}
ObJob::~ObJob()
{}
// Ideal strategy:
//-The finish-queue size of all jobs is allocated according to the actual number of tasks
// But the difficulty is that the number of tasks in the middle job of this framework
// can only be determined when it is running.
//
// Compromise strategy:
//-leaf job gives small value
//-non-leaf job uses the default large value
int ObJob::init_finish_queue(const ObExecContext& exec_ctx)
{
int ret = OB_SUCCESS;
int64_t finish_queue_size = 0;
if (is_root_job()) {
// do nothing
} else {
if (child_jobs_.count() > 0) {
// Not a leaf job, the size of finish_queue is set to 16K
// The reason why the number of tasks is not analyzed like leaf job is because in job init
// The stage has not really scheduled tasks, how many tasks will a non-leaf job have?
// It is impossible to know, and cannot give an exact value.
const static int64_t TASK_FINISH_QUEUE_MAX_LEN = 1024 * 16;
finish_queue_size = TASK_FINISH_QUEUE_MAX_LEN;
} else {
// leaf job, you can call prepare task control in advance
if (OB_FAIL(prepare_task_control(exec_ctx))) {
LOG_WARN("fail prepare task control", K(ret));
} else {
// + 2 means possible NOP_EVENT and SCHE_ITER_END two special messages
// Need to reserve 2 spaces for them
finish_queue_size = task_control_.get_task_count() + 2;
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(task_control_.init_finish_queue(finish_queue_size))) {
// The finish queue in task control must be initialized in advance,
// otherwise before scheduling
// The pop event from finish queue will fail, causing the main thread to fail on the queue
// Wait for the remote end to return the result.
LOG_WARN("fail init task control", K(ret));
}
}
LOG_TRACE("job finish queue init", "job_id", get_job_id(), K(finish_queue_size), K(ret));
}
return ret;
}
int ObJob::prepare_task_control(const ObExecContext& exec_ctx)
{
int ret = OB_SUCCESS;
ObTaskInfo* task = NULL;
if (OB_I(t1) OB_ISNULL(task_spliter_) || OB_ISNULL(root_op_) || OB_ISNULL(phy_plan_)) {
ret = OB_NOT_INIT;
LOG_WARN("job not init", K_(task_spliter), K_(root_op), K_(phy_plan));
} else {
while (OB_SUCC(ret) && OB_SUCC(task_spliter_->get_next_task(task))) {
if (OB_FAIL(OB_I(t2) task_control_.add_task(task))) {
task_control_.reset();
LOG_WARN("fail add task to taskq", K(ret), "task", to_cstring(task));
}
LOG_DEBUG("add task", K(task), "task", to_cstring(task));
}
if (OB_LIKELY(OB_ITER_END == ret)) {
ret = OB_SUCCESS;
if (OB_SUCC(ret)) {
int64_t stmt_parallel_degree = 0;
if (OB_FAIL(get_parallel_degree(exec_ctx, stmt_parallel_degree))) {
LOG_WARN("fail get parallel degree", K(ret));
} else if (OB_FAIL(task_control_.prepare(stmt_parallel_degree))) {
LOG_WARN("fail to prepare task control", K(ret), K(stmt_parallel_degree));
} else {
task_splited_ = true;
}
}
}
}
return ret;
}
int ObJob::get_parallel_degree(const ObExecContext& exec_ctx, int64_t& stmt_parallel_degree)
{
int ret = OB_SUCCESS;
const ObSQLSessionInfo* session = NULL;
if (OB_ISNULL(session = exec_ctx.get_my_session()) || OB_ISNULL(phy_plan_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("session is NULL", K(ret), KP(phy_plan_), K(session));
} else {
const ObQueryHint& query_hint = phy_plan_->get_query_hint();
stmt_parallel_degree = query_hint.parallel_;
if (ObStmtHint::UNSET_PARALLEL == stmt_parallel_degree) {
// dop not specified in hint. use system variable
if (OB_FAIL(session->get_ob_stmt_parallel_degree(stmt_parallel_degree))) {
LOG_WARN("fail to get ob_stmt_parallel_degree from session", K(ret));
}
}
// When the need_serial_execute is identified in the physical plan,
// the task in the job needs to be executed serially.
if (OB_SUCC(ret)) {
const ObPhysicalPlan* physical_plan = NULL;
if (OB_ISNULL(exec_ctx.get_physical_plan_ctx()) ||
OB_ISNULL(physical_plan = exec_ctx.get_physical_plan_ctx()->get_phy_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("physical plan is null", K(ret));
} else if (physical_plan->get_need_serial_exec() || session->need_serial_exec()) {
stmt_parallel_degree = 1;
}
}
/**
* the system variables in nested session are serialized from other server,
* we can not get min or max value here because the serialize operation handle
* current value only.
* see: OB_DEF_SERIALIZE(ObBasicSessionInfo).
*/
if (OB_SUCC(ret) && !session->is_nested_session() && !session->is_fast_select()) {
if (OB_UNLIKELY(stmt_parallel_degree < 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("stmt_parallel_degree is invalid", K(ret), K(stmt_parallel_degree));
}
}
}
return ret;
}
void ObJob::reset()
{
ob_job_id_.reset();
is_root_job_ = false;
phy_plan_ = NULL;
root_op_ = NULL;
state_ = OB_JOB_STATE_NOT_INIT;
task_spliter_ = NULL;
task_splited_ = false;
task_control_.reset();
is_outer_join_child_job_ = false;
has_outer_join_child_scan_ = false;
}
int ObJob::sort_scan_partition_locations(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
ObSEArray<const ObTableScan*, 16> scan_ops;
if (OB_ISNULL(root_op_)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("root op is NULL");
} else if (OB_FAIL(ObTaskSpliter::find_scan_ops(scan_ops, *root_op_))) {
LOG_WARN("fail to find scan ops", K(ret), "root_op_id", root_op_->get_id());
} else if (OB_UNLIKELY(1 > scan_ops.count())) {
} else {
ObPhyTableLocation* table_loc = NULL;
int64_t base_part_loc_count = -1;
int64_t base_part_order = 1;
for (int64_t i = 0; OB_SUCC(ret) && i < scan_ops.count(); ++i) {
const ObTableScan* scan_op = scan_ops.at(i);
if (OB_ISNULL(scan_op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("scan op can't be null", K(ret));
} else {
table_loc = NULL;
uint64_t table_location_key = scan_op->get_table_location_key();
if (OB_FAIL(ObTaskExecutorCtxUtil::get_phy_table_location_for_update(
ctx, table_location_key, scan_op->get_location_table_id(), table_loc))) {
LOG_WARN("fail to get phy table location", K(ret));
} else if (OB_ISNULL(table_loc)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get phy table location", K(ret));
} else {
if (0 == i) {
const ObPartitionReplicaLocationIArray& base_part_locs = table_loc->get_partition_location_list();
base_part_loc_count = base_part_locs.count();
if (base_part_loc_count > 1) {
base_part_order = base_part_locs.at(1).get_partition_id() - base_part_locs.at(0).get_partition_id();
} else {
base_part_order = 1;
}
if (OB_UNLIKELY(0 == base_part_order)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("partition id in same scan can not be equal", K(ret), K(base_part_order), K(base_part_locs));
}
} else {
ObPartitionReplicaLocationIArray& part_locs = table_loc->get_partition_location_list();
if (OB_UNLIKELY(part_locs.count() != base_part_loc_count)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("part loc count not equal, can not wise join",
K(ret),
K(i),
K(part_locs.count()),
K(base_part_loc_count),
K(part_locs));
} else {
if (part_locs.count() > 1) {
int64_t part_order = part_locs.at(1).get_partition_id() - part_locs.at(0).get_partition_id();
if (OB_UNLIKELY(0 == base_part_order || 0 == part_order)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("partition id in same scan can not be equal",
K(ret),
K(base_part_order),
K(part_order),
K(part_locs));
} else {
// The order of the bottom left scan is different from the order of the current scan,
// then the current scan is arranged in the order of the bottom left scan
if (base_part_order * part_order < 0) {
// At present, the copy efficiency is relatively low.
// If it becomes a performance bottleneck in the future, change it
if (base_part_order > 0) {
std::sort(&part_locs.at(0),
&part_locs.at(0) + part_locs.count(),
ObPartitionReplicaLocation::compare_part_loc_asc);
} else {
std::sort(&part_locs.at(0),
&part_locs.at(0) + part_locs.count(),
ObPartitionReplicaLocation::compare_part_loc_desc);
}
}
}
}
}
}
}
}
}
}
return ret;
}
int ObJob::get_task_control(const ObExecContext& exec_ctx, ObTaskControl*& task_control)
{
int ret = OB_SUCCESS;
if (!task_splited_) {
if (OB_FAIL(prepare_task_control(exec_ctx))) {
LOG_WARN("fail prepare task control", K(ret));
}
}
task_control = &task_control_;
return ret;
}
int ObJob::get_finished_task_locations(ObSArray<ObTaskLocation>& task_locs) const
{
int ret = OB_SUCCESS;
bool is_valid_finished_tasks = false;
ObSEArray<ObTaskInfo*, 8> task_infos;
if (OB_FAIL(OB_I(t1) task_control_.get_finished_tasks(task_infos))) {
LOG_WARN("fail to get finished tasks from task control", K(ret));
} else if (OB_FAIL(is_valid_finished_task_infos(task_infos, is_valid_finished_tasks))) {
LOG_WARN("invalid finished task infos", K(ret), K(task_infos), K(task_control_));
} else if (true == is_valid_finished_tasks) {
for (int64_t i = 0; OB_SUCC(ret) && i < task_infos.count(); ++i) {
ObTaskInfo* task_info = task_infos.at(i);
if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
} else {
const ObTaskLocation& loc = task_info->get_task_location();
if (OB_FAIL(OB_I(t4) task_locs.push_back(loc))) {
LOG_WARN("fail to push to task location array", K(i), K(ret));
}
}
}
} else {
}
return ret;
}
int ObJob::update_job_state(ObExecContext& ctx, ObTaskEvent& evt, bool& job_finished)
{
int ret = OB_SUCCESS;
if (OB_FAIL(task_control_.update_task_state(ctx, evt))) {
LOG_WARN("fail to update task state", K(ret));
} else if (task_control_.all_tasks_finished_or_skipped_or_failed()) {
// this job is finished
set_state(OB_JOB_STATE_FINISHED);
job_finished = true;
} else {
job_finished = false;
}
return ret;
}
int ObJob::get_task_result(uint64_t task_id, ObTaskResult& task_result) const
{
return task_control_.get_task_result(task_id, task_result);
}
int ObJob::append_to_last_failed_task_infos(ObIArray<ObTaskInfo*>& last_failed_task_infos) const
{
int ret = OB_SUCCESS;
ObSEArray<ObTaskInfo*, 32> all_task_infos;
if (OB_FAIL(task_control_.get_all_tasks(all_task_infos))) {
LOG_WARN("fail to get all task infos", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < all_task_infos.count(); ++i) {
ObTaskInfo* task_info = all_task_infos.at(i);
if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task info is NULL", K(ret), K(i));
} else {
switch (task_info->get_state()) {
case OB_TASK_STATE_FINISHED: {
// When the execution of the task fails and the location cache needs to be refreshed
// Only successfully executed participants do not need to be refreshed
// The location cache needs to be refreshed if it is not executed or fails to execute
// The purpose of this is to prevent that after a task fails,
// the partitions involved in other unexecuted tasks may also have location cache changes
// If you only re-flash the failed task each time,
// it will cause the retry to repeatedly encounter a new failed task
break;
}
case OB_TASK_STATE_NOT_INIT:
case OB_TASK_STATE_INITED:
case OB_TASK_STATE_RUNNING:
case OB_TASK_STATE_SKIPPED:
case OB_TASK_STATE_FAILED: {
if (OB_FAIL(last_failed_task_infos.push_back(task_info))) {
LOG_WARN("fail to push back task info into last_failed_task_infos", K(ret), K(*task_info));
}
break;
}
default: {
LOG_ERROR("invalid state", K(*task_info));
break;
}
}
}
}
}
return ret;
}
int ObJob::print_status(char* buf, int64_t buf_len, int64_t& pos, bool ignore_normal_state /* = false*/) const
{
int ret = OB_SUCCESS;
ObArray<ObTaskInfo*> all_task_infos;
if (OB_FAIL(task_control_.get_all_tasks(all_task_infos))) {
LOG_WARN("fail to get all task infos", K(ret));
} else {
int64_t state_not_init_count = 0;
int64_t state_inited_count = 0;
int64_t state_running_count = 0;
int64_t state_finished_count = 0;
int64_t state_skipped_count = 0;
int64_t state_failed_count = 0;
bool is_normal_state = false;
if (OB_FAIL(J_OBJ_START())) {
LOG_WARN("fail to print obj start", K(ret));
} else {
J_KV(N_TASK_COUNT, all_task_infos.count());
if (OB_FAIL(J_COMMA())) {
LOG_WARN("fail to print comma", K(ret));
}
}
for (int64_t i = 0, print_count = 0; OB_SUCC(ret) && i < all_task_infos.count(); ++i) {
ObTaskInfo* task_info = all_task_infos.at(i);
if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task info is NULL", K(ret), K(i));
} else {
switch (task_info->get_state()) {
case OB_TASK_STATE_NOT_INIT: {
state_not_init_count++;
is_normal_state = true;
break;
}
case OB_TASK_STATE_INITED: {
state_inited_count++;
is_normal_state = true;
break;
}
case OB_TASK_STATE_RUNNING: {
state_running_count++;
is_normal_state = false;
break;
}
case OB_TASK_STATE_FINISHED: {
state_finished_count++;
is_normal_state = true;
break;
}
case OB_TASK_STATE_SKIPPED: {
state_skipped_count++;
is_normal_state = false;
break;
}
case OB_TASK_STATE_FAILED: {
state_failed_count++;
is_normal_state = false;
break;
}
default: {
LOG_ERROR("invalid state", K(task_info->get_state()));
break;
}
}
if (OB_FAIL(ret)) {
} else if (ignore_normal_state && is_normal_state) {
} else if (print_count > 0 && OB_FAIL(J_COMMA())) {
LOG_WARN("fail to print comma", K(ret), K(i), K(*task_info));
} else {
const ObTaskLocation& task_loc = task_info->get_task_location();
BUF_PRINTF("task_info:{");
J_KV("loc", task_loc.get_server());
J_KV("ctrl", task_loc.get_ctrl_server());
J_KV("eid", task_loc.get_execution_id());
J_KV("jid", task_loc.get_job_id());
J_KV("tid", task_loc.get_task_id());
J_KV("pull_sid", task_info->get_pull_slice_id());
J_KV("state", task_info->get_state());
BUF_PRINTF("}");
print_count++;
}
}
}
if (OB_SUCC(ret)) {
BUF_PRINTF("state statistics:{");
if (0 != state_not_init_count) {
J_KV("not_init", state_not_init_count);
}
if (0 != state_inited_count) {
J_KV("inited", state_inited_count);
}
if (0 != state_running_count) {
J_KV("running", state_running_count);
}
if (0 != state_finished_count) {
J_KV("finished", state_finished_count);
}
if (0 != state_skipped_count) {
J_KV("skipped", state_skipped_count);
}
if (0 != state_failed_count) {
J_KV("failed", state_failed_count);
}
BUF_PRINTF("}");
if (OB_FAIL(J_OBJ_END())) {
LOG_WARN("fail to print obj end", K(ret));
}
}
}
return ret;
}
int ObJob::find_child_job(uint64_t root_op_id, ObJob*& job) const
{
int ret = OB_ENTRY_NOT_EXIST;
for (int64_t i = 0; OB_ENTRY_NOT_EXIST == ret && i < child_jobs_.count(); ++i) {
ObJob* child_job = child_jobs_.at(i);
if (OB_I(t1) OB_ISNULL(child_job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child job is NULL", K(ret));
} else if (child_job->get_root_op_id() == root_op_id) {
job = child_job;
ret = OB_SUCCESS;
}
}
return ret;
}
int ObJob::is_valid_finished_task_infos(const ObIArray<ObTaskInfo*>& finished_tasks, bool& is_valid) const
{
int ret = OB_SUCCESS;
is_valid = true;
if (finished_tasks.count() != task_control_.get_task_count()) {
ObArray<ObTaskInfo*> all_task_infos;
if (OB_FAIL(task_control_.get_all_tasks(all_task_infos))) {
LOG_WARN("fail to get all task infos", K(ret));
}
for (int64_t i = 0; OB_SUCC(ret) && true == is_valid && i < all_task_infos.count(); ++i) {
ObTaskInfo* task_info = all_task_infos.at(i);
if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("task info is NULL", K(ret), K(i), K(all_task_infos.count()));
} else if (OB_UNLIKELY(OB_TASK_STATE_FINISHED != task_info->get_state() &&
OB_TASK_STATE_SKIPPED != task_info->get_state())) {
is_valid = false;
LOG_WARN("some task fail",
"finished_task_count",
finished_tasks.count(),
"total_task_count",
task_control_.get_task_count(),
K(*task_info),
"task_control",
to_cstring(task_control_));
}
}
}
return ret;
}
int ObJob::job_can_exec(bool& can_exec)
{
int ret = OB_SUCCESS;
int64_t child_count = child_jobs_.count();
can_exec = true;
for (int64_t i = 0; OB_SUCC(ret) && can_exec && i < child_count; i++) {
ObJob* child_job = NULL;
if (OB_FAIL(child_jobs_.at(i, child_job))) {
LOG_WARN("fail to get child job", K(ret));
} else if (OB_ISNULL(child_job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child_job is NULL", K(ret));
} else {
can_exec = child_job->parent_can_exec();
}
}
return ret;
}
int ObJob::append_finished_slice_events(common::ObIArray<const ObSliceEvent*>& slice_events, bool skip_empty)
{
return task_control_.append_finished_slice_events(slice_events, skip_empty);
}
int ObJob::need_skip_empty_result(bool& skip_empty) const
{
int ret = OB_SUCCESS;
if (OB_ISNULL(root_op_) || OB_ISNULL(root_op_->get_parent())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op or root parent op is NULL", K(ret));
} else {
skip_empty = (PHY_TASK_ORDER_RECEIVE != root_op_->get_parent()->get_type());
}
return ret;
}
int ObJob::child_need_repart(bool& need_repart_part, bool& need_repart_subpart) const
{
int ret = OB_SUCCESS;
if (child_jobs_.count() > 0) {
ObJob* child_job = child_jobs_.at(0);
if (OB_ISNULL(child_job) || OB_ISNULL(child_job->root_op_) || !IS_TRANSMIT(child_job->root_op_->get_type())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child job or child root op is NULL or child root op is not transmit", K(ret));
} else {
static_cast<ObTransmit*>(child_job->root_op_)->need_repart(need_repart_part, need_repart_subpart);
}
} else {
need_repart_part = false;
need_repart_subpart = false;
}
return ret;
}
DEF_TO_STRING(ObJob)
{
int64_t pos = 0;
J_OBJ_START();
J_KV(K_(ob_job_id), K_(is_root_job), K_(state));
J_COMMA();
J_NAME(N_PLAN_TREE);
J_COLON();
print_plan_tree(buf, buf_len, pos, root_op_);
J_OBJ_END();
return pos;
}
void ObJob::print_plan_tree(char* buf, const int64_t buf_len, int64_t& pos, const ObPhyOperator* phy_op) const
{
if (!OB_ISNULL(phy_op)) {
J_OBJ_START();
J_KV(N_OP, ob_phy_operator_type_str(phy_op->get_type()));
J_COMMA();
J_KV(N_OP_ID, phy_op->get_id());
int64_t child_num = phy_op->get_child_num();
if (child_num > 0 && !IS_RECEIVE(phy_op->get_type())) {
J_COMMA();
J_NAME("child_op");
J_COLON();
J_ARRAY_START();
for (int32_t i = 0; i < child_num; i++) {
if (i > 0) {
J_COMMA();
}
print_plan_tree(buf, buf_len, pos, phy_op->get_child(i));
}
J_ARRAY_END();
}
J_OBJ_END();
}
}

289
src/sql/executor/ob_job.h Normal file
View File

@ -0,0 +1,289 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_JOB_
#define OCEANBASE_SQL_EXECUTOR_JOB_
#include "lib/queue/ob_lighty_queue.h"
#include "sql/executor/ob_job_id.h"
#include "sql/executor/ob_task_id.h"
#include "sql/executor/ob_slice_id.h"
#include "sql/executor/ob_task_info.h"
#include "sql/executor/ob_task_executor.h"
#include "sql/executor/ob_task_control.h"
#include "sql/engine/ob_physical_plan.h"
namespace oceanbase {
namespace sql {
enum ObJobState {
OB_JOB_STATE_NOT_INIT,
OB_JOB_STATE_INITED,
OB_JOB_STATE_RUNNING,
OB_JOB_STATE_FINISHED,
};
class ObPhyOperator;
class ObTaskEvent;
class ObTaskSpliter;
class ObPhysicalPlanCtx;
class ObAddrsProvider;
class ObExecContext;
class ObOpSpec;
class ObMiniJob {
public:
ObMiniJob() : phy_plan_(NULL), root_op_(NULL), extend_op_(NULL), root_spec_(NULL), extend_spec_(NULL)
{}
inline const ObPhysicalPlan* get_phy_plan() const
{
return phy_plan_;
}
inline void set_phy_plan(const ObPhysicalPlan* phy_plan)
{
phy_plan_ = phy_plan;
}
inline const ObPhyOperator* get_root_op() const
{
return root_op_;
}
inline void set_root_op(const ObPhyOperator* root_op)
{
root_op_ = root_op;
}
inline const ObPhyOperator* get_extend_op() const
{
return extend_op_;
}
inline void set_extend_op(const ObPhyOperator* extend_op)
{
extend_op_ = extend_op;
}
inline const ObOpSpec* get_root_spec() const
{
return root_spec_;
}
inline void set_root_spec(const ObOpSpec* root_spec)
{
root_spec_ = root_spec;
}
inline const ObOpSpec* get_extend_spec() const
{
return extend_spec_;
}
inline void set_extend_spec(const ObOpSpec* extend_op)
{
extend_spec_ = extend_op;
}
DECLARE_TO_STRING;
private:
const ObPhysicalPlan* phy_plan_;
const ObPhyOperator* root_op_;
const ObPhyOperator* extend_op_;
// for new engine
const ObOpSpec* root_spec_;
const ObOpSpec* extend_spec_;
};
class ObJob {
public:
ObJob();
virtual ~ObJob();
// different task needs different spliter
void set_task_spliter(ObTaskSpliter* spliter)
{
task_spliter_ = spliter;
}
int init_finish_queue(const ObExecContext& exec_ctx);
// @deprecated
int add_depending_job(const ObJob* job);
int sort_scan_partition_locations(ObExecContext& ctx);
int get_task_control(const ObExecContext& ctx, ObTaskControl*& task_control);
int get_finished_task_locations(common::ObSArray<ObTaskLocation>& task_locs) const;
ObJobState get_state()
{
return state_;
}
void set_state(ObJobState state)
{
state_ = state;
}
int update_job_state(ObExecContext& ctx, ObTaskEvent& evt, bool& job_finished);
int signal_schedule_error(int64_t nop_event)
{
return task_control_.signal_schedule_error(nop_event);
}
int get_task_result(uint64_t task_id, ObTaskResult& task_result) const;
common::ObLightyQueue& get_finish_queue()
{
return task_control_.get_finish_queue();
}
int append_to_last_failed_task_infos(common::ObIArray<ObTaskInfo*>& last_failed_task_infos) const;
int print_status(char* buf, int64_t buf_len, int64_t& pos, bool ignore_normal_state = false) const;
int find_child_job(uint64_t root_op_id, ObJob*& job) const;
void reset(); // TODO 2014-11-18, need reset ds,
inline void set_root_job()
{
is_root_job_ = true;
task_control_.set_root_job();
}
inline bool is_root_job() const
{
return is_root_job_;
}
inline void set_phy_plan(ObPhysicalPlan* phy_plan)
{
phy_plan_ = phy_plan;
if (phy_plan) {
// statement without select will not do partition level retry
task_control_.set_is_select_plan(phy_plan->is_select_plan());
}
}
inline const ObPhysicalPlan* get_phy_plan() const
{
return phy_plan_;
}
inline void set_root_op(ObPhyOperator* op)
{
root_op_ = op;
}
inline ObPhyOperator* get_root_op()
{
return root_op_;
}
inline void set_ob_job_id(const ObJobID& ob_job_id)
{
ob_job_id_ = ob_job_id;
}
inline const ObJobID& get_ob_job_id()
{
return ob_job_id_;
}
inline uint64_t get_execution_id() const
{
return ob_job_id_.get_execution_id();
}
inline uint64_t get_job_id() const
{
return ob_job_id_.get_job_id();
}
inline uint64_t get_root_op_id() const
{
return ob_job_id_.get_root_op_id();
}
inline bool all_tasks_run() const
{
return task_control_.all_tasks_run();
}
inline void set_parent_job(ObJob* parent_job);
inline int append_child_job(ObJob* job)
{
return child_jobs_.push_back(job);
}
inline int get_child_job(int64_t idx, ObJob*& job) const
{
return child_jobs_.at(idx, job);
}
inline int64_t get_child_count() const
{
return child_jobs_.count();
}
inline bool parent_can_exec()
{
return OB_JOB_STATE_FINISHED == state_;
}
int job_can_exec(bool& can_exec);
int append_finished_slice_events(common::ObIArray<const ObSliceEvent*>& slice_events, bool skip_empty);
int need_skip_empty_result(bool& skip_empty) const;
int child_need_repart(bool& with_part, bool& with_subpart) const;
inline void set_outer_join_child_job(bool outer_join_child_job)
{
is_outer_join_child_job_ = outer_join_child_job;
}
inline void set_outer_join_child_scan(bool outer_join_child_scan)
{
has_outer_join_child_scan_ = outer_join_child_scan;
}
inline void set_scan(bool has_scan)
{
has_scan_ = has_scan;
}
inline bool is_outer_join_child_job() const
{
return is_outer_join_child_job_;
}
inline bool has_outer_join_child_job() const;
inline bool has_outer_join_child_scan() const
{
return has_outer_join_child_scan_;
}
inline bool has_child_job() const
{
return get_child_count() > 0;
}
inline bool has_scan_op() const
{
return has_scan_;
}
DECLARE_TO_STRING;
private:
int prepare_task_control(const ObExecContext& exec_ctx);
int get_parallel_degree(const ObExecContext& exec_ctx, int64_t& stmt_parallel_degree);
int is_valid_finished_task_infos(const common::ObIArray<ObTaskInfo*>& task_infos, bool& is_valid) const;
void print_plan_tree(char* buf, const int64_t buf_len, int64_t& pos, const ObPhyOperator* phy_op) const;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObJob);
private:
static const int64_t MAX_CHILD_JOB_NUM = 16;
ObJobID ob_job_id_;
bool is_root_job_;
ObPhysicalPlan* phy_plan_;
ObPhyOperator* root_op_;
ObJobState state_;
// control task split and task's status
ObTaskSpliter* task_spliter_;
bool task_splited_;
ObTaskControl task_control_;
common::ObSEArray<ObJob*, 4> child_jobs_;
bool is_outer_join_child_job_;
bool has_outer_join_child_scan_;
bool has_scan_;
};
inline bool ObJob::has_outer_join_child_job() const
{
bool ret = false;
for (int64_t i = 0; i < child_jobs_.count(); i++) {
if (!OB_ISNULL(child_jobs_.at(i)) && child_jobs_.at(i)->is_outer_join_child_job()) {
ret = true;
break;
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_JOB_ */
//// end of header file

View File

@ -0,0 +1,35 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_job_conf.h"
#include "sql/executor/ob_task_spliter.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/ob_sql_utils.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
ObJobConf::ObJobConf()
: task_split_type_(ObTaskSpliter::INVALID_SPLIT), table_id_(OB_INVALID_ID), index_id_(OB_INVALID_ID)
{}
ObJobConf::~ObJobConf()
{}
void ObJobConf::reset()
{
task_split_type_ = ObTaskSpliter::INVALID_SPLIT;
table_id_ = OB_INVALID_ID;
index_id_ = OB_INVALID_ID;
}

View File

@ -0,0 +1,81 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_JOB_CONF_
#define OCEANBASE_SQL_EXECUTOR_JOB_CONF_
#include "share/ob_define.h"
#include "lib/container/ob_array.h"
#include "sql/engine/expr/ob_sql_expression.h"
#include "sql/rewrite/ob_query_range.h"
#include "sql/ob_phy_table_location.h"
namespace oceanbase {
namespace sql {
typedef common::ObIArray<common::ObNewRange> RangeIArray;
typedef common::ObSEArray<common::ObNewRange, 8> RangeSEArray;
class ObJobConf {
public:
ObJobConf();
virtual ~ObJobConf();
void reset();
void set_scan_table_id(const uint64_t table_id, const uint64_t index_id);
int get_scan_table_id(uint64_t& table_id, uint64_t& index_id);
inline int get_task_split_type() const
{
return task_split_type_;
}
inline void set_task_split_type(int type)
{
task_split_type_ = type;
}
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObJobConf);
private:
// Task Split Type
int task_split_type_;
// If a table scan Job, we need this info:
uint64_t table_id_;
uint64_t index_id_;
// Shuffle Method
// Transmit Method
// Root Job Parameters
// Parallel Parameters
// etc...
//
};
inline void ObJobConf::set_scan_table_id(const uint64_t table_id, const uint64_t index_id)
{
table_id_ = table_id;
index_id_ = index_id;
}
inline int ObJobConf::get_scan_table_id(uint64_t& table_id, uint64_t& index_id)
{
table_id = table_id_;
index_id = index_id_;
return common::OB_SUCCESS;
}
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_JOB_CONF_ */
//// end of header file

View File

@ -0,0 +1,464 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_job_control.h"
#include "sql/executor/ob_task_event.h"
#include "sql/executor/ob_transmit.h"
#include "sql/executor/ob_fifo_receive.h"
#include "lib/utility/ob_tracepoint.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
namespace sql {
using namespace oceanbase::common;
volatile uint64_t ObJobControl::global_job_id_ = 0;
ObJobControl::ObJobControl() : jobs_(), local_job_id_(0)
{}
ObJobControl::~ObJobControl()
{
for (int64_t i = 0; i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (NULL != job) {
job->~ObJob();
}
}
}
void ObJobControl::reset()
{
local_job_id_ = 0;
for (int64_t i = 0; i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (NULL != job) {
job->reset();
}
}
jobs_.reset();
}
int ObJobControl::all_jobs_finished(bool& is_finished) const
{
int ret = OB_SUCCESS;
bool finished = true;
for (int64_t i = 0; OB_SUCC(ret) && true == finished && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_I(t1) OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
finished = false;
LOG_ERROR("job is NULL", K(i), K(ret));
} else if (OB_JOB_STATE_FINISHED != job->get_state()) {
finished = false;
} else {
// empty
}
}
is_finished = finished;
return ret;
}
int ObJobControl::all_jobs_finished_except_root_job(bool& is_finished) const
{
int ret = OB_SUCCESS;
bool finished = true;
if (jobs_.count() < 1) {
ret = OB_ERR_UNEXPECTED;
finished = false;
LOG_ERROR("count of jobs is less than 1", K(ret), "job_count", jobs_.count());
} else {
for (int64_t i = 0; OB_SUCC(ret) && true == finished && i < jobs_.count() - 1; ++i) {
ObJob* job = jobs_.at(i);
if (OB_I(t1) OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
finished = false;
LOG_ERROR("job is NULL", K(i), K(ret));
} else if (OB_JOB_STATE_FINISHED != job->get_state()) {
finished = false;
} else {
// empty
}
}
}
is_finished = finished;
return ret;
}
int ObJobControl::create_job(
ObIAllocator& allocator, const ObExecutionID& ob_execution_id, uint64_t root_op_id, ObJob*& job) const
{
int ret = OB_SUCCESS;
void* tmp = NULL;
job = NULL;
if (OB_I(t1) OB_ISNULL(tmp = allocator.alloc(sizeof(ObJob)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc ObJob", K(ret), K(ob_execution_id));
} else if (OB_I(t2) OB_ISNULL(job = new (tmp) ObJob)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to new ObJob", K(ret), K(ob_execution_id));
} else {
uint64_t job_id = 0;
if (ObSqlExecutionIDMap::is_outer_id(ob_execution_id.get_execution_id())) {
local_job_id_ += 1;
job_id = local_job_id_;
} else {
job_id = ATOMIC_FAA(&global_job_id_, 1);
}
ObJobID ob_job_id;
ob_job_id.set_ob_execution_id(ob_execution_id);
ob_job_id.set_job_id(job_id);
ob_job_id.set_root_op_id(root_op_id);
job->set_ob_job_id(ob_job_id);
}
return ret;
}
int ObJobControl::find_job_by_job_id(uint64_t job_id, ObJob*& job) const
{
int ret = OB_ENTRY_NOT_EXIST;
for (int64_t i = 0; OB_ENTRY_NOT_EXIST == ret && i < jobs_.count(); ++i) {
ObJob* tmp_job = jobs_.at(i);
if (OB_I(t1) OB_ISNULL(tmp_job)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job is NULL", K(ret));
} else if (tmp_job->get_job_id() == job_id) {
job = tmp_job;
ret = OB_SUCCESS;
}
}
return ret;
}
int ObJobControl::find_job_by_root_op_id(uint64_t root_op_id, ObJob*& job) const
{
int ret = OB_ENTRY_NOT_EXIST;
for (int64_t i = 0; OB_ENTRY_NOT_EXIST == ret && i < jobs_.count(); ++i) {
ObJob* tmp_job = jobs_.at(i);
if (OB_I(t1) OB_ISNULL(tmp_job)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job is NULL", K(ret));
} else if (tmp_job->get_root_op_id() == root_op_id) {
job = tmp_job;
ret = OB_SUCCESS;
}
}
return ret;
}
int ObJobControl::get_running_jobs(ObIArray<ObJob*>& jobs) const
{
int ret = OB_SUCCESS;
jobs.reset();
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_I(t1) OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job is NULL", K(ret));
} else if (OB_JOB_STATE_RUNNING == job->get_state() && OB_FAIL(jobs.push_back(job))) {
LOG_WARN("fail to push back job", K(ret), K(*job));
}
}
return ret;
}
int ObJobControl::get_all_jobs(ObIArray<ObJob*>& jobs) const
{
int ret = OB_SUCCESS;
jobs.reset();
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_I(t1) OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job is NULL", K(ret));
} else if (OB_FAIL(jobs.push_back(job))) {
LOG_WARN("fail to push back job", K(ret), K(*job));
}
}
return ret;
}
int ObJobControl::get_all_jobs_except_root_job(ObIArray<ObJob*>& jobs) const
{
int ret = OB_SUCCESS;
jobs.reset();
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_I(t1) OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job is NULL", K(ret));
} else if (!job->is_root_job() && OB_FAIL(jobs.push_back(job))) {
LOG_WARN("fail to push back job", K(ret), K(*job));
}
}
return ret;
}
int ObJobControl::sort_job_scan_part_locs(ObExecContext& ctx)
{
UNUSED(ctx);
return OB_SUCCESS;
}
int ObJobControl::init_job_finish_queue(ObExecContext& ctx)
{
UNUSED(ctx);
return OB_SUCCESS;
}
// int ObJobControl::arrange_jobs()
//{
// return jobs_quick_sort(jobs_, 0, jobs_.count() - 1);
//}
//
// int ObJobControl::jobs_quick_sort(ObIArray<ObJob *> &jobs,
// int64_t low,
// int64_t high)
//{
// int ret = OB_SUCCESS;
// if (low < high) {
// int64_t i = low;
// int64_t j = high;
// ObJob *temp = jobs.at(i);
// if (OB_ISNULL(temp)) {
// ret = OB_ERR_UNEXPECTED;
// LOG_WARN("job is NULL", K(ret), K(i), K(low), K(high));
// }
// while (OB_SUCC(ret) && i < j) {
// while ((jobs.at(j)->get_priority() <= temp->get_priority()) && (i < j)) {
// j--;
// }
// jobs.at(i) = jobs.at(j);
// while ((jobs.at(i)->get_priority() >= temp->get_priority()) && (i < j)) {
// i++;
// }
// jobs.at(j) = jobs.at(i);
// }
// if (OB_SUCC(ret)) {
// jobs.at(i) = temp;
// if (OB_FAIL(jobs_quick_sort(jobs, low, i - 1))) {
// LOG_WARN("fail to jobs quick sort left", K(ret),
// K(low), K(high), K(i), K(j));
// } else if (OB_FAIL(jobs_quick_sort(jobs, j + 1, high))) {
// LOG_WARN("fail to jobs quick sort right", K(ret),
// K(low), K(high), K(i), K(j));
// }
// }
// }
// return ret;
//}
int ObJobControl::build_jobs_ctx(ObExecContext& ctx)
{
int ret = OB_SUCCESS;
int64_t job_count = jobs_.count();
for (int64_t i = job_count - 1; OB_SUCC(ret) && i >= 0; i--) {
if (OB_ISNULL(jobs_[i])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("jobs_[i] is NULL", K(ret), K(i), K(job_count));
} else if (OB_FAIL(build_job_ctx(ctx, *jobs_[i]))) {
LOG_WARN("fail build job op input");
}
}
return ret;
}
int ObJobControl::get_last_failed_task_infos(ObIArray<ObTaskInfo*>& last_failed_task_infos) const
{
int ret = OB_SUCCESS;
last_failed_task_infos.reset();
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job is NULL", K(ret), K(i));
} else if (OB_FAIL(job->append_to_last_failed_task_infos(last_failed_task_infos))) {
LOG_WARN("fail to append last failed task infos", K(ret), K(i), K(*job));
}
}
return ret;
}
int ObJobControl::print_status(char* buf, int64_t buf_len, bool ignore_normal_state /* = false*/) const
{
int ret = OB_SUCCESS;
int64_t pos = 0;
if (OB_FAIL(J_OBJ_START())) {
LOG_WARN("fail to print obj start", K(ret));
} else {
J_KV(N_JOB_COUNT, jobs_.count());
if (OB_FAIL(J_COMMA())) {
LOG_WARN("fail to print comma", K(ret));
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job is NULL", K(ret), K(i));
} else if (OB_FAIL(job->print_status(buf, buf_len, pos, ignore_normal_state))) {
LOG_WARN("fail to print job status", K(ret), K(i), K(*job));
} else if (i < jobs_.count() - 1 && OB_FAIL(J_COMMA())) {
LOG_WARN("fail to print comma", K(ret), K(i), K(*job));
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(J_OBJ_END())) {
LOG_WARN("fail to print obj end", K(ret));
}
if (OB_SIZE_OVERFLOW == ret) {
LOG_WARN("buf overflow, truncate it", K(ret), K(buf_len), K(pos));
ret = OB_SUCCESS;
}
return ret;
}
int ObJobControl::build_job_ctx(ObExecContext& query_ctx, ObJob& job)
{
int ret = OB_SUCCESS;
ObTransmitInput* transmit_input = NULL;
ObPhyOperator* root_op = job.get_root_op();
if (OB_ISNULL(root_op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op is NULL", K(ret), K(job));
} else if (IS_TRANSMIT(root_op->get_type()) || job.is_root_job()) {
ObTransmit* transmit_op = static_cast<ObTransmit*>(root_op);
if (OB_FAIL(build_phy_op_input(query_ctx, root_op, &job))) {
LOG_WARN("fail to build physical operator input", K(ret));
} else if (!job.is_root_job()) {
if (OB_I(t2) OB_ISNULL(transmit_input = GET_PHY_OP_INPUT(ObTransmitInput, query_ctx, transmit_op->get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("transmit input is NULL", K(ret), "op_id", transmit_op->get_id());
} else {
transmit_input->set_job(&job);
}
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("root op must be transmit operator except root job", K(ret), "type", root_op->get_type(), K(job));
}
return ret;
}
// recursively build op input for current job
int ObJobControl::alloc_phy_op_input(ObExecContext& ctx, ObPhyOperator* op)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op is NULL", K(ret));
} else if (OB_FAIL(op->create_operator_input(ctx))) {
LOG_WARN("fail create operator input", K(ret));
}
for (int32_t i = 0; OB_SUCC(ret) && i < op->get_child_num(); ++i) {
ObPhyOperator* child_op = op->get_child(i);
// no more search if reach the begining of next job
if (OB_ISNULL(child_op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child op is NULL", K(ret));
} else if (OB_FAIL(alloc_phy_op_input(ctx, child_op))) {
LOG_WARN("fail to alloc child op input", K(ret), K(i));
}
}
return ret;
}
// recursively build op input for current job
int ObJobControl::build_phy_op_input(ObExecContext& job_ctx, ObPhyOperator* op, ObJob* job)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op is NULL", K(ret));
}
for (int32_t i = 0; OB_SUCC(ret) && i < op->get_child_num(); ++i) {
ObPhyOperator* child_op = op->get_child(i);
// no more search if reach the begining of next job
if (OB_ISNULL(child_op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child op is NULL", K(ret));
} else if (!IS_TRANSMIT(child_op->get_type())) {
if (OB_FAIL(build_phy_op_input(job_ctx, op->get_child(i), job))) {
LOG_WARN("fail to build child op input", K(ret), K(i));
}
}
}
if (OB_SUCC(ret)) {
if (IS_ASYNC_RECEIVE(op->get_type()) && NULL != job && !job->is_root_job()) {
ObPhyOperator* child_op = NULL;
ObJob* child_job = NULL;
ObDistributedReceiveInput* receive_input = NULL;
if (1 != op->get_child_num()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("receive child count is not 1", K(ret), K(op->get_child_num()));
} else if (OB_ISNULL(child_op = op->get_child(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("receive child is NULL", K(ret));
} else if (!IS_TRANSMIT(child_op->get_type())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("receive child is not transmit", K(ret), K(op->get_type()));
} else if (OB_FAIL(job->find_child_job(child_op->get_id(), child_job))) {
LOG_WARN("fail to find child job", K(ret), K(child_op->get_id()));
} else if (OB_ISNULL(child_job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child job is NULL", K(ret));
} else if (OB_ISNULL(receive_input = GET_PHY_OP_INPUT(ObDistributedReceiveInput, job_ctx, op->get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("receive input is NULL", K(ret), "op_id", op->get_id());
} else {
receive_input->set_child_job_id(child_job->get_job_id());
}
} else {
// nothing.
}
}
return ret;
}
DEF_TO_STRING(ObJobControl)
{
int64_t pos = 0;
J_OBJ_START();
J_NAME(N_JOB_TREE);
J_COLON();
print_job_tree(buf, buf_len, pos, jobs_.at(0));
J_OBJ_END();
return pos;
}
void ObJobControl::print_job_tree(char* buf, const int64_t buf_len, int64_t& pos, ObJob* job) const
{
J_OBJ_START();
J_KV(N_JOB, job);
int64_t child_count = job->get_child_count();
if (child_count > 0) {
J_COMMA();
J_NAME(N_CHILD_JOB);
J_COLON();
J_ARRAY_START();
ObJob* child_job = NULL;
for (int64_t i = 0; i < child_count; i++) {
if (i > 0) {
J_COMMA();
}
(void)job->get_child_job(i, child_job);
print_job_tree(buf, buf_len, pos, child_job);
}
J_ARRAY_END();
}
J_OBJ_END();
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,81 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_JOB_CONTROL_
#define OCEANBASE_SQL_EXECUTOR_OB_JOB_CONTROL_
#include "lib/container/ob_se_array.h"
#include "sql/executor/ob_job.h"
namespace oceanbase {
namespace sql {
class ObTaskEvent;
class ObJobControl {
public:
explicit ObJobControl();
virtual ~ObJobControl();
static int alloc_phy_op_input(ObExecContext& ctx, ObPhyOperator* op);
static int build_phy_op_input(ObExecContext& ctx, ObPhyOperator* op, ObJob* job = NULL);
void reset();
virtual int sort_job_scan_part_locs(ObExecContext& ctx);
virtual int init_job_finish_queue(ObExecContext& ctx);
// int arrange_jobs();
/*
* @input ob_execution_id is used to distinguish the results of different executions in IRM,
* only Distributed mode is effective,other mode is INVALID_ID
*/
int create_job(
common::ObIAllocator& alloc, const ObExecutionID& ob_execution_id, uint64_t root_op_id, ObJob*& job) const;
int find_job_by_job_id(uint64_t job_id, ObJob*& job) const;
int find_job_by_root_op_id(uint64_t root_op_id, ObJob*& job) const;
virtual int get_ready_jobs(common::ObIArray<ObJob*>& jobs, bool serial_sched = false) const = 0;
virtual int get_running_jobs(common::ObIArray<ObJob*>& jobs) const;
int get_all_jobs(common::ObIArray<ObJob*>& jobs) const;
int get_all_jobs_except_root_job(common::ObIArray<ObJob*>& jobs) const;
int all_jobs_finished(bool& is_finished) const;
int all_jobs_finished_except_root_job(bool& is_finished) const;
// build op input of the current job
int build_jobs_ctx(ObExecContext& ctx);
int get_last_failed_task_infos(common::ObIArray<ObTaskInfo*>& last_failed_task_infos) const;
int print_status(char* buf, int64_t buf_len, bool ignore_normal_state = false) const;
inline int add_job(ObJob* job)
{
return jobs_.push_back(job);
}
inline int64_t get_job_count() const
{
return jobs_.count();
}
DECLARE_TO_STRING;
private:
int build_job_ctx(ObExecContext& ctx, ObJob& job);
int build_job_op_input(ObExecContext& ctx, ObJob& job);
// int jobs_quick_sort(common::ObIArray<ObJob *> &jobs,
// int64_t low,
// int64_t high);
void print_job_tree(char* buf, const int64_t buf_len, int64_t& pos, ObJob* job) const;
protected:
common::ObSEArray<ObJob*, 2> jobs_; // remote plan has two jobs
private:
mutable uint64_t local_job_id_;
static volatile uint64_t global_job_id_;
DISALLOW_COPY_AND_ASSIGN(ObJobControl);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_JOB_CONTROL_ */

View File

@ -0,0 +1,25 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include "share/ob_define.h"
#include "lib/utility/serialization.h"
#include "sql/executor/ob_job_id.h"
#include "lib/json/ob_yson.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
OB_SERIALIZE_MEMBER(ObJobID, ob_execution_id_, job_id_, root_op_id_);
DEFINE_TO_YSON_KV(ObJobID, OB_ID(execution_id), ob_execution_id_, OB_ID(job_id), job_id_);
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,125 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_JOB_ID_
#define OCEANBASE_SQL_EXECUTOR_OB_JOB_ID_
#include "sql/executor/ob_execution_id.h"
namespace oceanbase {
namespace sql {
class ObJobID final {
OB_UNIS_VERSION(1);
public:
ObJobID(const ObExecutionID& ob_execution_id, uint64_t job_id)
: ob_execution_id_(ob_execution_id), job_id_(job_id), root_op_id_(0)
{}
ObJobID() : ob_execution_id_(), job_id_(common::OB_INVALID_ID), root_op_id_(0)
{}
inline void set_ob_execution_id(const ObExecutionID& ob_execution_id)
{
ob_execution_id_ = ob_execution_id;
}
inline void set_server(const common::ObAddr& server)
{
ob_execution_id_.set_server(server);
}
inline void set_dist_task_type()
{
ob_execution_id_.set_dist_task_type();
}
inline void set_mini_task_type()
{
ob_execution_id_.set_mini_task_type();
}
inline void set_execution_id(const uint64_t execution_id)
{
ob_execution_id_.set_execution_id(execution_id);
}
inline void set_job_id(const uint64_t job_id)
{
job_id_ = job_id;
}
inline void set_root_op_id(uint64_t root_op_id)
{
root_op_id_ = root_op_id;
}
inline const ObExecutionID& get_ob_execution_id() const
{
return ob_execution_id_;
}
inline const common::ObAddr& get_server() const
{
return ob_execution_id_.get_server();
}
inline uint64_t get_task_type() const
{
return ob_execution_id_.get_task_type();
}
inline bool is_dist_task_type() const
{
return ob_execution_id_.is_dist_task_type();
}
inline bool is_mini_task_type() const
{
return ob_execution_id_.is_mini_task_type();
}
inline uint64_t get_execution_id() const
{
return ob_execution_id_.get_execution_id();
}
inline uint64_t get_job_id() const
{
return job_id_;
}
inline uint64_t get_root_op_id() const
{
return root_op_id_;
}
inline int64_t hash() const
{
return ob_execution_id_.hash() + job_id_;
}
inline bool equal(const ObJobID& id) const
{
return id.ob_execution_id_.equal(ob_execution_id_) && id.job_id_ == job_id_;
}
inline bool operator==(const ObJobID& id) const
{
return equal(id);
}
inline bool is_valid() const
{
return ob_execution_id_.is_valid() && common::OB_INVALID_ID != job_id_;
}
inline void reset()
{
ob_execution_id_.reset();
job_id_ = common::OB_INVALID_ID;
}
TO_STRING_KV(N_OB_EXECUTION_ID, ob_execution_id_, N_JOB_ID, job_id_);
DECLARE_TO_YSON_KV;
private:
/* variables */
ObExecutionID ob_execution_id_;
uint64_t job_id_;
uint64_t root_op_id_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_JOB_ID_ */
//// end of header file

View File

@ -0,0 +1,206 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/ob_sql_define.h"
#include "sql/executor/ob_addrs_provider.h"
#include "sql/executor/ob_job_parser.h"
#include "sql/executor/ob_transmit.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/join/ob_join.h"
#include "lib/profile/ob_perf_event.h"
#include "lib/json/ob_json_print_utils.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObJobParser::ObJobParser()
{}
ObJobParser::~ObJobParser()
{}
/* entry function
* @input ObPhysicalPlan include operator tree
* @input ob_execution_id current execution id
* @input ObTaskSpliterFactory Responsible for construct TaskSpliter
* @input ObAddrsProviderFactory Responsible for construct AddrsProvider
* @output ObJobControl
*/
int ObJobParser::parse_job(ObExecContext& exec_ctx, ObPhysicalPlan* phy_plan, const ObExecutionID& exec_id,
ObTaskSpliterFactory& spfactory, ObJobControl& job_ctrl) const
{
NG_TRACE(parse_job_begin);
int ret = OB_SUCCESS;
ObPhyOperator* root = NULL;
if (OB_ISNULL(phy_plan)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("plan is NULL", K(ret));
} else if (OB_ISNULL(root = phy_plan->get_main_query())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("root op of plan is NULL", K(ret));
} else {
int task_split_type = ObTaskSpliter::LOCAL_IDENTITY_SPLIT;
ObJob* root_job = NULL;
if (OB_FAIL(create_job(exec_ctx, phy_plan, root, exec_id, job_ctrl, task_split_type, spfactory, root_job))) {
LOG_WARN("fail to create job", K(ret));
} else if (OB_ISNULL(root_job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root_job is NULL", K(ret));
} else if (FALSE_IT(root_job->set_root_job())) {
} else if (OB_FAIL(split_jobs(exec_ctx, phy_plan, root, exec_id, job_ctrl, spfactory, *root_job))) {
LOG_WARN("fail to split jobs", K(ret));
} else {
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(ObJobControl::alloc_phy_op_input(exec_ctx, root))) {
LOG_WARN("fail alloc phy ops input", K(ret));
} else if (OB_FAIL(job_ctrl.build_jobs_ctx(exec_ctx))) {
LOG_WARN("fail build job input", K(ret));
} else if (OB_FAIL(job_ctrl.sort_job_scan_part_locs(exec_ctx))) {
LOG_WARN("fail to sort job scan partition locations", K(ret));
} else if (OB_FAIL(job_ctrl.init_job_finish_queue(exec_ctx))) {
LOG_WARN("fail init job", K(ret));
} else {
// sanity check for early stage debug, can be removed after code stabilized
if (OB_UNLIKELY(job_ctrl.get_job_count() <= 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("job count should > 0", K(ret), K(job_ctrl.get_job_count()));
}
}
}
NG_TRACE(parse_job_end);
return ret;
}
int ObJobParser::split_jobs(ObExecContext& exec_ctx, ObPhysicalPlan* phy_plan, ObPhyOperator* phy_op,
const ObExecutionID& exec_id, ObJobControl& job_ctrl, ObTaskSpliterFactory& spfactory, ObJob& cur_job) const
{
int ret = OB_SUCCESS;
ObTransmit* transmit_op = NULL;
ObJob* job = NULL;
if (NULL == phy_op) {
// op is NULL, do nothing
} else if (OB_ISNULL(phy_plan)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("phy_plan is NULL", K(ret));
} else if (!IS_TRANSMIT(phy_op->get_type())) {
if (0 == phy_op->get_child_num()) {
cur_job.set_scan(true);
if (is_outer_join_child(*phy_op)) {
cur_job.set_outer_join_child_scan(true);
}
}
} else if (OB_ISNULL(transmit_op = static_cast<ObTransmit*>(phy_op))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("transmit op is NULL", K(ret), K(phy_op));
} else {
int task_split_type = transmit_op->get_job_conf().get_task_split_type();
if (OB_FAIL(create_job(exec_ctx, phy_plan, phy_op, exec_id, job_ctrl, task_split_type, spfactory, job))) {
LOG_WARN("fail to create job", K(ret), K(exec_id));
} else if (OB_FAIL(cur_job.append_child_job(job))) {
LOG_WARN("fail to add child job", K(ret), K(exec_id));
} else if (is_outer_join_child(*phy_op)) {
job->set_outer_join_child_job(true);
}
}
for (int32_t i = 0; OB_SUCC(ret) && i < phy_op->get_child_num(); ++i) {
if (OB_FAIL(split_jobs(
exec_ctx, phy_plan, phy_op->get_child(i), exec_id, job_ctrl, spfactory, NULL != job ? *job : cur_job))) {
LOG_WARN("fail to split jobs for child op", K(ret), K(exec_id), K(i));
} else {
}
}
return ret;
}
int ObJobParser::create_job(ObExecContext& exec_ctx, ObPhysicalPlan* phy_plan, ObPhyOperator* phy_op,
const ObExecutionID& exec_id, ObJobControl& job_ctrl, const int task_split_type, ObTaskSpliterFactory& spfactory,
ObJob*& job) const
{
int ret = OB_SUCCESS;
job = NULL;
if (OB_ISNULL(phy_op) || OB_ISNULL(phy_plan)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("op or phy_plan is NULL", K(ret), K(phy_op), K(phy_plan));
} else if (OB_FAIL(job_ctrl.create_job(exec_ctx.get_allocator(), exec_id, phy_op->get_id(), job))) {
LOG_WARN("fail to create job", K(ret));
} else if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job is NULL", K(ret), K(exec_id));
} else {
job->set_phy_plan(phy_plan);
job->set_root_op(phy_op);
job->set_state(OB_JOB_STATE_INITED);
if (OB_FAIL(job_ctrl.add_job(job))) {
job->~ObJob();
job = NULL;
LOG_WARN("fail add job", K(ret));
} else {
// set Spliter && ServersProvider into Jobs
ObTaskSpliter* task_spliter = NULL;
if (OB_FAIL(spfactory.create(exec_ctx, *job, task_split_type, task_spliter))) {
LOG_WARN("fail create task spliter", "type", task_split_type, K(ret));
} else if (OB_ISNULL(task_spliter)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task_spliter is NULL", K(ret));
} else {
job->set_task_spliter(task_spliter);
}
}
}
return ret;
}
bool ObJobParser::is_outer_join_child(const ObPhyOperator& phy_op) const
{
bool is_outer_join_child_ret = false;
const ObPhyOperator* cur_op = &phy_op;
const ObPhyOperator* parent_op = phy_op.get_parent();
while (!is_outer_join_child_ret && !OB_ISNULL(parent_op) && !IS_TRANSMIT(parent_op->get_type())) {
if (IS_JOIN(parent_op->get_type())) {
const ObJoin* join = static_cast<const ObJoin*>(parent_op);
ObJoinType join_type = join->get_join_type();
switch (join_type) {
case LEFT_OUTER_JOIN:
/*no break*/
case LEFT_ANTI_JOIN:
if (cur_op == join->get_child(0)) {
is_outer_join_child_ret = true;
}
break;
case RIGHT_OUTER_JOIN:
/*no break*/
case RIGHT_ANTI_JOIN:
if (cur_op == join->get_child(1)) {
is_outer_join_child_ret = true;
}
break;
case FULL_OUTER_JOIN:
is_outer_join_child_ret = true;
break;
default:
break;
}
break;
} else if (IS_SET_PHY_OP(parent_op->get_type())) {
is_outer_join_child_ret = true;
}
cur_op = parent_op;
parent_op = parent_op->get_parent();
}
return is_outer_join_child_ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,48 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_JOB_PARSER_
#define OCEANBASE_SQL_EXECUTOR_OB_JOB_PARSER_
#include "sql/engine/ob_physical_plan.h"
#include "sql/executor/ob_job_control.h"
#include "sql/executor/ob_task_spliter_factory.h"
#include "sql/executor/ob_addrs_provider_factory.h"
namespace oceanbase {
namespace sql {
class ObJobParser {
public:
ObJobParser();
virtual ~ObJobParser();
void reset()
{}
int parse_job(ObExecContext& ctx, ObPhysicalPlan* plan, const ObExecutionID& ob_execution_id,
ObTaskSpliterFactory& spliter_factory, ObJobControl& jc) const;
private:
int split_jobs(ObExecContext& ctx, ObPhysicalPlan* phy_plan, ObPhyOperator* op, const ObExecutionID& ob_execution_id,
ObJobControl& jc, ObTaskSpliterFactory& spliter_factory, ObJob& cur_job) const;
int create_job(ObExecContext& ctx, ObPhysicalPlan* phy_plan, ObPhyOperator* op, const ObExecutionID& ob_execution_id,
ObJobControl& jc, const int task_split_type, ObTaskSpliterFactory& spfactory, ObJob*& job) const;
bool is_outer_join_child(const ObPhyOperator& phy_op) const;
private:
DISALLOW_COPY_AND_ASSIGN(ObJobParser);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_JOB_PARSER_ */

View File

@ -0,0 +1,68 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_local_identity_task_spliter.h"
#include "sql/executor/ob_task_info.h"
#include "sql/executor/ob_job.h"
#include "lib/allocator/ob_allocator.h"
#include "sql/engine/ob_phy_operator.h"
#include "lib/utility/ob_tracepoint.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
ObLocalIdentityTaskSpliter::ObLocalIdentityTaskSpliter() : task_(NULL)
{}
ObLocalIdentityTaskSpliter::~ObLocalIdentityTaskSpliter()
{
if (NULL != task_) {
task_->~ObTaskInfo();
task_ = NULL;
}
}
int ObLocalIdentityTaskSpliter::get_next_task(ObTaskInfo*& task)
{
int ret = OB_SUCCESS;
if (OB_I(t1) OB_UNLIKELY(OB_ISNULL(allocator_) || OB_ISNULL(job_))) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(allocator_), K(job_));
} else if (NULL != task_) {
ret = OB_ITER_END;
} else {
void* ptr = allocator_->alloc(sizeof(ObTaskInfo));
if (OB_ISNULL(ptr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail allocate ObTaskInfo", K(ret));
} else {
task_ = new (ptr) ObTaskInfo(*allocator_);
ObTaskID ob_task_id;
ObTaskLocation task_loc;
ob_task_id.set_ob_job_id(job_->get_ob_job_id());
ob_task_id.set_task_id(0);
task_loc.set_ob_task_id(ob_task_id);
task_loc.set_server(server_);
task_->set_task_split_type(get_type());
task_->set_location_idx(0);
task_->set_pull_slice_id(0);
task_->set_task_location(task_loc);
task_->set_root_op(job_->get_root_op());
task_->set_state(OB_TASK_STATE_NOT_INIT);
// in order to ensure that the second call to get_next_task can return OB_ITER_END
task = task_;
}
}
return ret;
}

View File

@ -0,0 +1,47 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_LOCAL_IDENTITY_TASK_SPLITER_
#define OCEANBASE_SQL_EXECUTOR_OB_LOCAL_IDENTITY_TASK_SPLITER_
#include "sql/executor/ob_task_spliter.h"
namespace oceanbase {
namespace sql {
class ObPhysicalPlan;
class ObTaskInfo;
// This class is only used when only one local task is generated.
// In the executor phase, if it is judged that the split type is ObTaskSpliter::LOCAL_IDENTITY_SPLIT,
// it will be directly optimized without the process of splitting the job,
// which is equivalent to the size of this class. Some functions will not be called
class ObLocalIdentityTaskSpliter : public ObTaskSpliter {
public:
ObLocalIdentityTaskSpliter();
virtual ~ObLocalIdentityTaskSpliter();
virtual int get_next_task(ObTaskInfo*& task);
inline virtual TaskSplitType get_type() const
{
return ObTaskSpliter::LOCAL_IDENTITY_SPLIT;
}
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObLocalIdentityTaskSpliter);
private:
ObTaskInfo* task_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_LOCAL_IDENTITY_TASK_SPLITER_ */
//// end of header file

View File

@ -0,0 +1,48 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_local_job_control.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObLocalJobControl::ObLocalJobControl()
{}
ObLocalJobControl::~ObLocalJobControl()
{}
int ObLocalJobControl::get_ready_jobs(ObIArray<ObJob*>& jobs, bool serial_sched) const
{
int ret = OB_SUCCESS;
UNUSED(serial_sched);
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job is NULL", K(ret));
} else if (OB_JOB_STATE_INITED == job->get_state()) {
if (OB_FAIL(jobs.push_back(job))) {
LOG_WARN("fail to push back job", K(ret));
}
}
}
if (OB_SUCC(ret) && 1 != jobs.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("the count of ready jobs is not 1", K(jobs.count()));
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,32 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_LOCAL_JOB_CONTROL_
#define OCEANBASE_SQL_EXECUTOR_OB_LOCAL_JOB_CONTROL_
#include "sql/executor/ob_job_control.h"
namespace oceanbase {
namespace sql {
class ObLocalJobControl : public ObJobControl {
public:
explicit ObLocalJobControl();
virtual ~ObLocalJobControl();
virtual int get_ready_jobs(common::ObIArray<ObJob*>& jobs, bool serial_sched = false) const;
private:
DISALLOW_COPY_AND_ASSIGN(ObLocalJobControl);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_LOCAL_JOB_CONTROL_ */

View File

@ -0,0 +1,80 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "lib/container/ob_array.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/executor/ob_job.h"
#include "sql/executor/ob_task_info.h"
#include "sql/executor/ob_local_job_executor.h"
#include "lib/utility/ob_tracepoint.h"
#include "lib/profile/ob_perf_event.h"
namespace oceanbase {
namespace sql {
using namespace oceanbase::common;
ObLocalJobExecutor::ObLocalJobExecutor() : job_(NULL), executor_(NULL)
{}
ObLocalJobExecutor::~ObLocalJobExecutor()
{}
int ObLocalJobExecutor::execute(ObExecContext& query_ctx)
{
int ret = OB_SUCCESS;
ObTaskInfo* task_info = NULL;
if (OB_I(t1)(OB_ISNULL(job_) || OB_ISNULL(executor_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job_ or executor_ is NULL", K(ret), K(job_), K(executor_));
} else if (OB_FAIL(OB_I(t2) get_executable_task(query_ctx, task_info))) {
LOG_WARN("fail get a executable task.", K(ret));
} else if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task info is NULL", K(ret));
} else if (OB_FAIL(OB_I(t3) executor_->execute(query_ctx, job_, task_info))) {
LOG_WARN("fail execute task.", K(ret), K(*task_info));
} else {
}
return ret;
}
int ObLocalJobExecutor::get_executable_task(ObExecContext& ctx, ObTaskInfo*& task_info)
{
int ret = OB_SUCCESS;
ObTaskControl* tq = NULL;
ObSEArray<ObTaskInfo*, 1> ready_tasks;
if (OB_I(t1)(OB_ISNULL(job_) || OB_ISNULL(executor_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job_ or executor_ is NULL", K(ret), K(job_), K(executor_));
} else if (OB_FAIL(OB_I(t1) job_->get_task_control(ctx, tq))) {
LOG_WARN("fail get task control.", K(ret));
} else if (OB_ISNULL(tq)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succ to get task control, but task control is NULL", K(ret));
} else if (OB_FAIL(OB_I(t2) tq->get_ready_tasks(ready_tasks))) {
LOG_WARN("fail get ready task.", K(ret));
} else if (OB_I(t3) OB_UNLIKELY(1 != ready_tasks.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected ready task count", "ready_tasks_count", ready_tasks.count());
} else if (OB_FAIL(OB_I(t4) ready_tasks.at(0, task_info))) {
LOG_WARN("fail get task from array", K(ret));
} else {
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,54 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_LOCAL_JOB_EXECUTOR_
#define OCEANBASE_SQL_EXECUTOR_OB_LOCAL_JOB_EXECUTOR_
namespace oceanbase {
namespace sql {
class ObTaskInfo;
class ObJob;
class ObTaskExecutor;
class ObExecContext;
class ObLocalJobExecutor {
public:
ObLocalJobExecutor();
virtual ~ObLocalJobExecutor();
void set_job(ObJob& job)
{
job_ = &job;
}
void set_task_executor(ObTaskExecutor& executor)
{
executor_ = &executor;
}
int execute(ObExecContext& ctx);
inline void reset()
{
job_ = NULL;
executor_ = NULL;
}
private:
// disallow copy
ObLocalJobExecutor(const ObLocalJobExecutor& other);
ObLocalJobExecutor& operator=(const ObLocalJobExecutor& ohter);
int get_executable_task(ObExecContext& ctx, ObTaskInfo*& task);
private:
ObJob* job_;
ObTaskExecutor* executor_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_LOCAL_JOB_EXECUTOR_ */
//// end of header file

View File

@ -0,0 +1,153 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_transmit.h"
#include "sql/executor/ob_addrs_provider.h"
#include "sql/executor/ob_local_scheduler.h"
#include "sql/executor/ob_local_job_executor.h"
#include "sql/executor/ob_local_task_executor.h"
#include "sql/executor/ob_job.h"
#include "sql/executor/ob_job_parser.h"
#include "share/ob_define.h"
#include "lib/utility/utility.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/profile/ob_perf_event.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
ObLocalScheduler::ObLocalScheduler()
{}
ObLocalScheduler::~ObLocalScheduler()
{}
int ObLocalScheduler::schedule(ObExecContext& ctx, ObPhysicalPlan* phy_plan)
{
// 1. Split and construct task using ObJobConf info
// 2. Call job.schedule()
int ret = OB_SUCCESS;
ObPhysicalPlanCtx* plan_ctx = NULL;
ObTaskExecutorCtx* executor_ctx = NULL;
ObPhyOperator* root_op = NULL;
LOG_DEBUG("local scheduler start", K(ctx), K(*phy_plan));
if (OB_ISNULL(phy_plan)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("phy_plan not init", K(phy_plan), K(ret));
} else if (OB_UNLIKELY(NULL == (root_op = phy_plan->get_main_query()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root_op not init", K(phy_plan), K(ret));
} else if (OB_UNLIKELY(NULL == (executor_ctx = GET_TASK_EXECUTOR_CTX(ctx)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("executor ctx is NULL", K(ret));
} else if (OB_UNLIKELY(NULL == (plan_ctx = ctx.get_physical_plan_ctx()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("plan ctx is NULL", K(ret));
} else if (OB_UNLIKELY(false == IS_TRANSMIT(root_op->get_type()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op is not transmit op", K(ret), K(*root_op));
} else {
LOG_DEBUG("execute params", K(plan_ctx->get_param_store()));
ObTransmit* transmit_op = static_cast<ObTransmit*>(root_op);
int task_split_type = transmit_op->get_job_conf().get_task_split_type();
ObExecutionID ob_execution_id(executor_ctx->get_self_addr(), OB_INVALID_ID);
/**
* Since the data structure involved in splitting the job is relatively heavy,
* and the process of splitting is time-consuming,
* if only one local task is generated (LOCAL_IDENTITY_SPLIT means that only one local task is generated),
* skip the step of splitting the job
*/
if (ObTaskSpliter::LOCAL_IDENTITY_SPLIT == task_split_type) {
if (OB_FAIL(direct_generate_task_and_execute(ctx, ob_execution_id, root_op))) {
LOG_WARN("fail to directly generate task and execute", K(ret), K(ob_execution_id), K(*root_op));
}
} else {
ObJobParser parser;
ObLocalTaskExecutor task_executor;
ObLocalJobExecutor job_executor;
ObSEArray<ObJob*, 1> jobs;
ObJob* local_job = NULL;
ObLocalJobControl jc;
ObTaskSpliterFactory spfactory;
if (OB_FAIL(parser.parse_job(ctx, phy_plan, ob_execution_id, spfactory, jc))) {
LOG_WARN("fail to parse job for scheduler", K(ret));
} else if (OB_FAIL(jc.get_ready_jobs(jobs))) {
LOG_WARN("fail get jobs", "jb_cnt", jc.get_job_count(), K(ret));
} else if (OB_UNLIKELY(1 != jobs.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected job count");
} else if (OB_UNLIKELY(NULL == (local_job = jobs.at(0)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null job");
} else {
job_executor.set_task_executor(task_executor);
job_executor.set_job(*local_job);
/**
* The execute method finally constructs a Result,
* which contains an executable Op Tree,
* and finally starts to execute and fetch data,
* which is driven in ObResultSet.
*/
if (OB_FAIL(job_executor.execute(ctx))) {
LOG_WARN("fail execute local job");
}
}
}
}
return ret;
}
int ObLocalScheduler::direct_generate_task_and_execute(
ObExecContext& ctx, const ObExecutionID& ob_execution_id, ObPhyOperator* root_op)
{
int ret = OB_SUCCESS;
ObTaskInfo* task = NULL;
void* ptr = NULL;
ObTaskExecutorCtx* executor_ctx = NULL;
if (OB_ISNULL(root_op)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("root_op not init", K(ret));
} else if (OB_UNLIKELY(NULL == (executor_ctx = GET_TASK_EXECUTOR_CTX(ctx)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("executor ctx is NULL", K(ret));
} else if (OB_UNLIKELY(NULL == (ptr = ctx.get_allocator().alloc(sizeof(ObTaskInfo))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
} else if (OB_UNLIKELY(NULL == (task = new (ptr) ObTaskInfo(ctx.get_allocator())))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObTaskInfo", K(ret));
} else {
ObLocalTaskExecutor task_executor;
ObJobID ob_job_id(ob_execution_id, 0);
ObTaskID ob_task_id(ob_job_id, 0);
ObTaskLocation task_loc(executor_ctx->get_self_addr(), ob_task_id);
task->set_task_split_type(ObTaskSpliter::LOCAL_IDENTITY_SPLIT);
task->set_location_idx(0);
task->set_pull_slice_id(0);
task->set_task_location(task_loc);
task->set_root_op(root_op);
task->set_state(OB_TASK_STATE_NOT_INIT);
if (OB_UNLIKELY(false == IS_TRANSMIT(root_op->get_type()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op is not transmit op", K(ret), K(*root_op));
} else if (OB_FAIL(ObJobControl::build_phy_op_input(ctx, root_op))) {
LOG_WARN("fail to build physical operator input", K(ret));
} else if (OB_FAIL(task_executor.execute(ctx, NULL, task))) {
LOG_WARN("fail execute task.", K(ret));
}
task->~ObTaskInfo();
}
return ret;
}

View File

@ -0,0 +1,42 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_LOCAL_SCHEDULER_
#define OCEANBASE_SQL_EXECUTOR_LOCAL_SCHEDULER_
#include "sql/executor/ob_sql_scheduler.h"
#include "sql/executor/ob_task_spliter_factory.h"
#include "sql/executor/ob_addrs_provider_factory.h"
#include "sql/executor/ob_local_job_control.h"
namespace oceanbase {
namespace sql {
class ObLocalScheduler : public ObSqlScheduler {
public:
ObLocalScheduler();
virtual ~ObLocalScheduler();
virtual int schedule(ObExecContext& ctx, ObPhysicalPlan* phy_plan);
private:
int direct_generate_task_and_execute(
ObExecContext& ctx, const ObExecutionID& ob_execution_id, ObPhyOperator* root_op);
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObLocalScheduler);
};
} // namespace sql
} // namespace oceanbase
#endif /* __SQL_EXECUTOR_LOCAL_SCHEDULER_ */
//// end of header file

View File

@ -0,0 +1,65 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_local_task_executor.h"
#include "sql/executor/ob_transmit.h"
#include "sql/executor/ob_job.h"
#include "sql/executor/ob_task_executor_ctx.h"
#include "sql/executor/ob_execute_result.h"
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_physical_plan.h"
#include "lib/utility/utility.h"
#include "sql/engine/ob_exec_context.h"
#include "lib/profile/ob_perf_event.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
ObLocalTaskExecutor::ObLocalTaskExecutor()
{}
ObLocalTaskExecutor::~ObLocalTaskExecutor()
{}
int ObLocalTaskExecutor::execute(ObExecContext& ctx, ObJob* job, ObTaskInfo* task_info)
{
UNUSED(job);
int ret = OB_SUCCESS;
ObPhyOperator* root_op = NULL;
ObTaskExecutorCtx& task_exec_ctx = ctx.get_task_exec_ctx();
ObExecuteResult& exec_result = task_exec_ctx.get_execute_result();
if (OB_ISNULL(task_info)) {
ret = OB_NOT_INIT;
LOG_WARN("job or taskinfo not set", K(task_info));
} else {
if (OB_UNLIKELY(NULL == (root_op = task_info->get_root_op()))) {
ret = OB_NOT_INIT;
LOG_WARN("fail execute task. no query found.", K(root_op), K(ret));
} else if (OB_FAIL(build_task_op_input(ctx, *task_info, *root_op))) {
LOG_WARN("fail to build op input", K(ret));
} else {
// set root op into executor result
exec_result.set_root_op(root_op);
task_info->set_state(OB_TASK_STATE_RUNNING);
}
if (OB_FAIL(ret)) {
task_info->set_state(OB_TASK_STATE_FAILED);
}
}
NG_TRACE_EXT(local_task_completed, OB_ID(ret), ret);
return ret;
}

View File

@ -0,0 +1,35 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_LOCAL_TASK_EXECUTOR_
#define OCEANBASE_SQL_EXECUTOR_LOCAL_TASK_EXECUTOR_
#include "sql/executor/ob_task_executor.h"
namespace oceanbase {
namespace sql {
class ObLocalTaskExecutor : public ObTaskExecutor {
public:
ObLocalTaskExecutor();
virtual ~ObLocalTaskExecutor();
virtual int execute(ObExecContext& ctx, ObJob* job, ObTaskInfo* task_info);
inline virtual void reset()
{
ObTaskExecutor::reset();
}
private:
DISALLOW_COPY_AND_ASSIGN(ObLocalTaskExecutor);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_LOCAL_TASK_EXECUTOR_ */
//// end of header file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,199 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef DEV_SRC_SQL_EXECUTOR_OB_MINI_TASK_EXECUTOR_H_
#define DEV_SRC_SQL_EXECUTOR_OB_MINI_TASK_EXECUTOR_H_
#include "sql/engine/ob_exec_context.h"
#include "sql/executor/ob_trans_result_collector.h"
#include "lib/allocator/ob_safe_arena.h"
namespace oceanbase {
namespace sql {
class ObAPMiniTaskMgr : public common::ObDLinkBase<ObAPMiniTaskMgr> {
static const int64_t MAX_FINISH_QUEUE_CAPACITY = 512;
public:
ObAPMiniTaskMgr()
: ref_count_(0),
mgr_rcode_(common::OB_SUCCESS),
rcode_addrs_(),
allocator_(common::ObModIds::OB_SQL_EXECUTOR_MINI_TASK_EVENT),
trans_result_(),
lock_()
{}
virtual ~ObAPMiniTaskMgr()
{}
int32_t get_type()
{
return 0;
}
static ObAPMiniTaskMgr* alloc();
static void free(ObAPMiniTaskMgr* item);
inline int64_t inc_ref_count()
{
return ATOMIC_AAF((uint64_t*)&ref_count_, 1);
}
inline int64_t def_ref_count()
{
return ATOMIC_SAF((uint64_t*)&ref_count_, 1);
}
int init(ObSQLSessionInfo& session, ObExecutorRpcImpl* exec_rpc);
virtual void reset() override;
void set_mgr_rcode(int mgr_rcode)
{
mgr_rcode_ = mgr_rcode;
}
int atomic_push_mgr_rcode_addr(const ObAddr& addr);
common::ObIArray<ObAddr>& get_rcode_addr()
{
return rcode_addrs_;
}
int get_mgr_rcode() const
{
return mgr_rcode_;
}
int save_task_result(
const common::ObAddr& task_addr, int64_t task_id, int32_t ret_code, const ObMiniTaskResult& result);
int pop_task_event(int64_t timeout, ObMiniTaskEvent*& complete_task);
void close_task_event(ObMiniTaskEvent* task_event);
inline int send_task(const ObTaskInfo& task_info)
{
return trans_result_.send_task(task_info);
}
int merge_trans_result(const ObTaskID& task_id, const ObMiniTaskResult& result);
int set_task_status(const ObTaskID& task_id, ObTaskStatus status);
int wait_all_task(int64_t timeout)
{
return trans_result_.wait_all_task(timeout);
}
private:
int64_t ref_count_;
int mgr_rcode_;
common::ObArray<ObAddr> rcode_addrs_;
common::ObSafeArena allocator_;
common::ObLightyQueue finish_queue_;
ObTransResultCollector trans_result_;
// for the on_timeout() of mini task callback.
mutable common::ObSpinLock lock_;
};
typedef common::ObGlobalFactory<ObAPMiniTaskMgr, 1, common::ObModIds::OB_SQL_EXECUTOR_MINI_TASK_MGR>
ObAPMiniTaskMgrGFactory;
typedef common::ObTCFactory<ObAPMiniTaskMgr, 1, common::ObModIds::OB_SQL_EXECUTOR_MINI_TASK_MGR>
ObApMiniTaskMgrTCFactory;
inline ObAPMiniTaskMgr* ObAPMiniTaskMgr::alloc()
{
ObAPMiniTaskMgr* ap_mini_task_mgr = NULL;
if (OB_ISNULL(ObApMiniTaskMgrTCFactory::get_instance())) {
SQL_EXE_LOG(ERROR, "get ap mini task mgr factory instance failed");
ap_mini_task_mgr = NULL;
} else {
ap_mini_task_mgr = ObApMiniTaskMgrTCFactory::get_instance()->get(0);
}
return ap_mini_task_mgr;
}
inline void ObAPMiniTaskMgr::free(ObAPMiniTaskMgr* item)
{
if (item != NULL) {
int64_t ref_count = item->def_ref_count();
if (OB_LIKELY(0 == ref_count)) {
// nobody reference this object, so free it
if (OB_ISNULL(ObApMiniTaskMgrTCFactory::get_instance())) {
SQL_EXE_LOG(ERROR, "get ap mini task mgr factory instance failed");
} else {
item->reset();
ObApMiniTaskMgrTCFactory::get_instance()->put(item);
item = NULL;
}
} else if (OB_UNLIKELY(ref_count < 0)) {
SQL_EXE_LOG(ERROR, "ref_count is invalid", K(ref_count));
}
}
}
class ObSQLSessionInfo;
class ObMiniTaskExecutor {
public:
explicit ObMiniTaskExecutor(common::ObIAllocator& allocator) : ap_mini_task_mgr_(NULL)
{
UNUSED(allocator);
}
virtual ~ObMiniTaskExecutor()
{
destroy();
}
void destroy();
int init(ObSQLSessionInfo& session, ObExecutorRpcImpl* exec_rpc);
int merge_trans_result(const ObTaskID& task_id, const ObMiniTaskResult& task_result);
int wait_all_task(int64_t timeout);
static int add_invalid_servers_to_retry_info(
const int ret, const ObIArray<ObAddr>& addr, ObQueryRetryInfo& retry_info);
protected:
int mini_task_local_execute(ObExecContext& query_ctx, ObMiniTask& task, ObMiniTaskResult& task_result);
int sync_fetch_local_result(ObExecContext& ctx, const ObPhyOperator& root_op, common::ObScanner& result);
int sync_fetch_local_result(ObExecContext& ctx, const ObOpSpec& root_spec, ObScanner& result);
int check_scanner_errcode(const ObMiniTaskResult& src);
int check_scanner_errcode(const ObMiniTaskEvent& complete_task, ObMiniTaskRetryInfo& retry_info);
int wait_ap_task_finish(
ObExecContext& ctx, int64_t ap_task_cnt, ObMiniTaskResult& result, ObMiniTaskRetryInfo& retry_info);
int pop_ap_mini_task_event(ObExecContext& ctx, ObMiniTaskEvent*& complete_task);
protected:
ObAPMiniTaskMgr* ap_mini_task_mgr_;
};
class ObDMLMiniTaskExecutor : public ObMiniTaskExecutor {
public:
explicit ObDMLMiniTaskExecutor(common::ObIAllocator& allocator) : ObMiniTaskExecutor(allocator)
{}
virtual ~ObDMLMiniTaskExecutor()
{}
int execute(ObExecContext& ctx, const ObMiniJob& mini_job, common::ObIArray<ObTaskInfo*>& task_list, bool table_first,
ObMiniTaskResult& task_result);
// sync execute
int mini_task_execute(
ObExecContext& ctx, const ObMiniJob& mini_job, ObTaskInfo& task_info, ObMiniTaskResult& task_result);
// async execute
int mini_task_submit(ObExecContext& ctx, const ObMiniJob& mini_job, common::ObIArray<ObTaskInfo*>& task_info_list,
int64_t start_idx, ObMiniTaskResult& task_result);
int build_mini_task_op_input(ObExecContext& ctx, ObTaskInfo& task_info, const ObMiniJob& mini_job);
int build_mini_task_op_input(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& root_op);
int build_mini_task_op_input(ObExecContext& ctx, ObTaskInfo& task_info, const ObOpSpec& root_spec);
int build_mini_task(ObExecContext& ctx, const ObMiniJob& mini_job, ObTaskInfo& task_info, ObMiniTask& task);
};
class ObLookupMiniTaskExecutor : public ObMiniTaskExecutor {
public:
explicit ObLookupMiniTaskExecutor(common::ObIAllocator& allocator) : ObMiniTaskExecutor(allocator)
{}
virtual ~ObLookupMiniTaskExecutor()
{}
int execute(ObExecContext& ctx, common::ObIArray<ObMiniTask>& task_list,
common::ObIArray<ObTaskInfo*>& task_info_list, ObMiniTaskRetryInfo& retry_info, ObMiniTaskResult& task_result);
int execute_one_task(ObExecContext& ctx, ObMiniTask& task, ObTaskInfo* task_info, int64_t& ap_task_cnt,
ObMiniTaskRetryInfo& retry_info);
int fill_lookup_task_op_input(ObExecContext& ctx, ObMiniTask& task, ObTaskInfo* task_info,
const ObPhyOperator& root_op, const bool retry_execution);
int fill_lookup_task_op_input(
ObExecContext& ctx, ObTaskInfo* task_info, const ObOpSpec& root_spec, const bool retry_execution);
int retry_overflow_task(ObExecContext& ctx, common::ObIArray<ObMiniTask>& task_list,
common::ObIArray<ObTaskInfo*>& task_info_list, ObMiniTaskRetryInfo& retry_info, ObMiniTaskResult& task_result);
};
} // namespace sql
} // namespace oceanbase
#endif /* DEV_SRC_SQL_EXECUTOR_OB_MINI_TASK_EXECUTOR_H_ */

View File

@ -0,0 +1,181 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "lib/hash/ob_iteratable_hashset.h"
#include "share/partition_table/ob_partition_location_cache.h"
#include "sql/executor/ob_multiinsert_task_spliter.h"
#include "sql/engine/dml/ob_table_insert.h"
#include "sql/engine/expr/ob_sql_expression.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_phy_operator_type.h"
#include "share/partition_table/ob_partition_location.h"
#include "sql/executor/ob_job_conf.h"
using namespace oceanbase::common;
using namespace oceanbase::share;
namespace oceanbase {
namespace sql {
ObMultiInsertTaskSpliter::ObMultiInsertTaskSpliter()
: phy_table_loc_(NULL), prepare_done_flag_(false), store_(), next_task_idx_(0)
{}
ObMultiInsertTaskSpliter::~ObMultiInsertTaskSpliter()
{
for (int64_t i = 0; i < store_.count(); ++i) {
ObTaskInfo* t = store_.at(i);
if (OB_LIKELY(NULL != t)) {
t->~ObTaskInfo();
}
}
}
int ObMultiInsertTaskSpliter::prepare()
{
int ret = OB_SUCCESS;
ObPhyOperator* root_op = NULL;
prepare_done_flag_ = false;
if (OB_ISNULL(plan_ctx_) || OB_ISNULL(exec_ctx_) || OB_ISNULL(allocator_) || OB_ISNULL(job_) ||
OB_ISNULL(job_conf_)) {
ret = OB_NOT_INIT;
LOG_WARN("param not init", K_(plan_ctx), K_(exec_ctx), K_(allocator), K_(job), K_(job_conf));
} else if (OB_UNLIKELY(NULL == (root_op = job_->get_root_op()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op is NULL", K(ret));
} else {
ObSEArray<const ObTableModify*, 16> insert_ops;
if (OB_FAIL(ObTaskSpliter::find_insert_ops(insert_ops, *root_op))) {
LOG_WARN("fail to find insert ops", K(ret), "root_op_id", root_op->get_id());
} else if (OB_UNLIKELY(1 != insert_ops.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("there must be one and only one insert op in distr mode", K(ret), K(common::lbt()));
} else {
// const ObPhyTableLocation *table_loc = NULL;
const ObTableModify* insert_op = insert_ops.at(0);
if (OB_ISNULL(insert_op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("insert op can't be null", K(ret));
} else {
uint64_t table_location_key = insert_op->get_table_id();
if (OB_FAIL(ObTaskExecutorCtxUtil::get_phy_table_location(
*exec_ctx_, table_location_key, table_location_key, phy_table_loc_))) {
LOG_WARN("fail to get phy table location", K(ret));
} else if (OB_ISNULL(phy_table_loc_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get phy table location", K(ret), K(phy_table_loc_));
} else {
prepare_done_flag_ = true;
}
}
}
}
return ret;
}
int ObMultiInsertTaskSpliter::get_next_task(ObTaskInfo*& task)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(allocator_)) {
ret = OB_ERR_UNEXPECTED;
;
LOG_WARN("unexpected error. allocator is null", K(ret));
} else if (OB_UNLIKELY(false == prepare_done_flag_)) {
ret = prepare();
}
// after success prepare
if (OB_SUCC(ret)) {
void* ptr = NULL;
ObTaskInfo::ObRangeLocation range_loc(*allocator_);
if (OB_FAIL(get_next_range_location(range_loc))) {
if (OB_ITER_END != ret) {
LOG_WARN("fail get next task", K(ret));
}
} else if (OB_UNLIKELY(NULL == (ptr = allocator_->alloc(sizeof(ObTaskInfo))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail allocate task", K(ret));
} else {
ObTaskInfo* t = new (ptr) ObTaskInfo(*allocator_);
if (OB_FAIL(store_.push_back(t))) {
LOG_WARN("fail to push taskinfo into store", K(ret));
} else if (OB_FAIL(t->set_range_location(range_loc))) {
LOG_WARN("fail to set range_location", K(ret), K(range_loc));
} else {
ObTaskID ob_task_id;
ObTaskLocation task_loc;
ob_task_id.set_ob_job_id(job_->get_ob_job_id());
ob_task_id.set_task_id(next_task_idx_);
task_loc.set_ob_task_id(ob_task_id);
task_loc.set_server(range_loc.server_);
t->set_task_split_type(get_type());
t->set_pull_slice_id(next_task_idx_);
t->set_location_idx(next_task_idx_);
t->set_task_location(task_loc);
t->set_root_op(job_->get_root_op());
t->set_state(OB_TASK_STATE_NOT_INIT);
// job_->set_scan_job();
task = t;
// move to next info
next_task_idx_++;
}
}
}
return ret;
}
int ObMultiInsertTaskSpliter::get_next_range_location(ObTaskInfo::ObRangeLocation& range_loc)
{
int ret = OB_SUCCESS;
static const int64_t TABLE_COUNT = 1;
range_loc.reset();
const ObPartitionReplicaLocation* part_location = NULL;
if (OB_ISNULL(phy_table_loc_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("phy table loation is null", K(ret), K(phy_table_loc_));
} else if (OB_UNLIKELY(next_task_idx_ >= phy_table_loc_->get_partition_cnt())) {
ret = OB_ITER_END;
} else if (OB_ISNULL(part_location = phy_table_loc_->get_part_replic_by_index(next_task_idx_))) {
// A partition corresponds to a task
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error. can not get part location", K(ret), K(*phy_table_loc_));
} else {
const ObPartitionReplicaLocation& part_loc = *part_location;
const ObReplicaLocation& rep_loc = part_loc.get_replica_location();
if (!rep_loc.is_valid()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid replica loc", K(ret), K(rep_loc));
} else {
range_loc.server_ = rep_loc.server_;
if (OB_FAIL(range_loc.part_locs_.init(TABLE_COUNT))) {
LOG_WARN("init part_locs_ failed", K(ret));
} else {
ObTaskInfo::ObPartLoc task_part_loc;
if (OB_FAIL(part_loc.get_partition_key(task_part_loc.partition_key_))) {
LOG_WARN("fail to get partition key", K(ret), K(part_loc));
} else if (FALSE_IT(task_part_loc.renew_time_ = part_loc.get_renew_time())) {
} else if (!task_part_loc.is_valid()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid task partition location", K(ret), K(task_part_loc), K(part_loc));
} else if (OB_FAIL(range_loc.part_locs_.push_back(task_part_loc))) {
LOG_WARN("fail to push back partition key", K(ret), K(task_part_loc));
}
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,52 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_MULTIINSERT_TASK_SPLITER_
#define OCEANBASE_SQL_EXECUTOR_MULTIINSERT_TASK_SPLITER_
#include "sql/executor/ob_multiscan_task_spliter.h"
#include "lib/container/ob_array.h"
#include "lib/hash/ob_placement_hashmap.h"
#include "lib/hash/ob_iteratable_hashset.h"
namespace oceanbase {
namespace sql {
class ObMultiInsertTaskSpliter : public ObTaskSpliter {
public:
ObMultiInsertTaskSpliter();
virtual ~ObMultiInsertTaskSpliter();
virtual int get_next_task(ObTaskInfo*& task);
virtual TaskSplitType get_type() const
{
return ObTaskSpliter::INSERT_SPLIT;
}
private:
/* functions */
int prepare();
int get_next_range_location(ObTaskInfo::ObRangeLocation& range_loc);
private:
/* variables */
const ObPhyTableLocation* phy_table_loc_;
bool prepare_done_flag_;
common::ObSEArray<ObTaskInfo*, 2> store_;
int64_t next_task_idx_;
private:
/* other */
DISALLOW_COPY_AND_ASSIGN(ObMultiInsertTaskSpliter);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_MULTIINSERT_TASK_SPLITER_ */

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,205 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_MULTISCAN_TASK_SPLITER_
#define OCEANBASE_SQL_EXECUTOR_MULTISCAN_TASK_SPLITER_
#include "share/schema/ob_table_schema.h"
#include "sql/executor/ob_task_spliter.h"
#include "lib/container/ob_array.h"
#include "lib/hash/ob_placement_hashmap.h"
#include "lib/hash/ob_iteratable_hashset.h"
namespace oceanbase {
namespace sql {
class ObIntraPartitionTaskSpliter : public ObTaskSpliter {
public:
ObIntraPartitionTaskSpliter();
virtual ~ObIntraPartitionTaskSpliter();
virtual int get_next_task(ObTaskInfo*& task);
virtual TaskSplitType get_type() const
{
return ObTaskSpliter::INTRA_PARTITION_SPLIT;
}
private:
int prepare();
int get_part_and_ranges(
const share::ObPartitionReplicaLocation*& part_rep_loc, const ObSplittedRanges*& splitted_ranges);
int get_scan_ranges(const ObSplittedRanges& splitted_ranges, ObTaskInfo::ObPartLoc& part_loc);
private:
const ObPhyTableLocation* table_loc_;
const ObPartitionReplicaLocationIArray* part_rep_loc_list_;
const ObSplittedRangesIArray* splitted_ranges_list_;
int64_t next_task_id_;
int64_t part_idx_;
int64_t range_idx_;
bool prepare_done_;
};
class ObDistributedTaskSpliter : public ObTaskSpliter {
private:
enum ObMatchType {
MT_ONLY_MATCH = 0,
MT_ALL_PART = 1,
MT_ALL_SLICE = 2,
MT_ALL_BOTH = MT_ALL_PART | MT_ALL_SLICE,
};
struct ObPartComparer {
public:
ObPartComparer(common::ObIArray<ObShuffleKeys>& shuffle_keys, bool cmp_part, bool cmp_subpart, int sort_order);
virtual ~ObPartComparer();
bool operator()(int64_t idx1, int64_t idx2);
int get_ret() const
{
return ret_;
}
private:
common::ObIArray<ObShuffleKeys>& shuffle_keys_;
bool cmp_part_;
bool cmp_subpart_;
int sort_order_; // asc: 1, desc: -1.
int ret_;
};
struct ObSliceComparer {
public:
ObSliceComparer(bool cmp_part, bool cmp_subpart, int sort_order);
virtual ~ObSliceComparer();
bool operator()(const ObSliceEvent* slice1, const ObSliceEvent* slice2);
int get_ret() const
{
return ret_;
}
private:
bool cmp_part_;
bool cmp_subpart_;
int sort_order_; // asc: 1, desc: -1.
int ret_;
};
struct ObPhyTableLoc {
public:
ObPhyTableLoc()
: table_loc_(NULL),
depend_table_keys_(common::ObModIds::OB_SQL_EXECUTOR_TASK_SPLITER, OB_MALLOC_NORMAL_BLOCK_SIZE)
{}
virtual ~ObPhyTableLoc()
{}
void reset()
{
table_loc_ = NULL;
depend_table_keys_.reset();
}
bool is_valid() const
{
return NULL != table_loc_;
}
const ObPhyTableLocation* get_table_loc() const
{
return table_loc_;
}
int set_table_loc(const ObPhyTableLocation* table_loc)
{
int ret = common::OB_SUCCESS;
if (OB_ISNULL(table_loc)) {
ret = common::OB_INVALID_ARGUMENT;
SQL_EXE_LOG(ERROR, "table loc is NULL", K(ret), K(table_loc));
} else {
table_loc_ = table_loc;
}
return ret;
}
const common::ObIArray<ObPartitionKey>& get_depend_table_keys() const
{
return depend_table_keys_;
}
int add_depend_table_key(ObPartitionKey& depend_table_key)
{
return depend_table_keys_.push_back(depend_table_key);
}
TO_STRING_KV(K_(table_loc), K_(depend_table_keys));
private:
const ObPhyTableLocation* table_loc_;
common::ObSEArray<ObPartitionKey, 1> depend_table_keys_;
};
public:
ObDistributedTaskSpliter();
virtual ~ObDistributedTaskSpliter();
virtual int get_next_task(ObTaskInfo*& task);
virtual TaskSplitType get_type() const
{
return ObTaskSpliter::DISTRIBUTED_SPLIT;
}
private:
int prepare();
int init_match_type();
int init_table_locations(ObPhyOperator* root_op);
int check_table_locations();
int init_part_shuffle_keys();
int sort_part_shuffle_keys();
int get_shuffle_keys(
const share::schema::ObTableSchema& table_schema, const ObPartitionKey& part_key, ObShuffleKeys& shuffle_keys);
int init_child_task_results();
int sort_child_slice_shuffle_keys();
int compare_head_part_slice(int& cmp);
int task_add_head_part(ObTaskInfo*& task_info);
int task_add_head_slices(ObTaskInfo& task_info);
int task_add_empty_part(ObTaskInfo*& task_info);
int task_add_empty_slice(ObTaskInfo& task_info);
int get_task_location(const ObSliceID& ob_slice_id, ObTaskLocation& task_location);
int calc_head_slice_count();
bool need_all_part()
{
return match_type_ & MT_ALL_PART;
}
bool need_all_slice()
{
return match_type_ & MT_ALL_SLICE;
}
int get_or_create_task_info(const common::ObAddr& task_server, ObTaskInfo*& task_info);
int64_t get_total_part_cnt() const;
int get_task_runner_server(common::ObAddr& runner_server) const;
int need_split_task_by_partition(bool& by_partition) const;
private:
// table informations.
common::ObSEArray<ObPhyTableLoc, 8> table_locations_;
common::ObSEArray<ObShuffleKeys, 8> part_shuffle_keys_;
common::ObSEArray<int64_t, 8> part_idxs_;
// child task result informations.
common::ObSEArray<const ObSliceEvent*, 16> child_slices_;
// iteration informations.
ObMatchType match_type_; // like join type, inner, left/right outer, full.
int64_t next_task_id_;
int64_t head_part_idx_;
int64_t head_slice_idx_;
int64_t head_slice_count_;
int sort_order_; // asc: 1, desc: -1.
bool head_slice_matched_;
// others.
bool repart_part_;
bool repart_subpart_;
bool prepare_done_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDistributedTaskSpliter);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_MULTISCAN_TASK_SPLITER_ */
//// end of header file

View File

@ -0,0 +1,55 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_random_addrs_provider.h"
#include <stdio.h>
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObRandomAddrsProvider::ObRandomAddrsProvider() : servers_()
{}
ObRandomAddrsProvider::~ObRandomAddrsProvider()
{}
int ObRandomAddrsProvider::select_servers(int64_t select_count, common::ObIArray<ObAddr>& servers)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(select_count <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid select count", K(ret), K(select_count));
} else {
servers.reset();
int64_t mod = servers_.count();
int64_t selected_server_count = 0;
while (OB_SUCC(ret) && mod > 0 && selected_server_count < select_count) {
int64_t select_idx = rand() % mod;
ObAddr server = servers_.at(select_idx);
if (OB_FAIL(servers.push_back(server))) {
LOG_WARN("fail to push back server", K(ret), K(server));
} else {
// swap
servers_.at(select_idx) = servers_.at(mod - 1);
servers_.at(mod - 1) = server;
mod--;
selected_server_count++;
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,35 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_RANDOM_ADDRS_PROVIDER_
#define OCEANBASE_SQL_EXECUTOR_OB_RANDOM_ADDRS_PROVIDER_
#include "sql/executor/ob_addrs_provider.h"
#include "lib/container/ob_se_array.h"
namespace oceanbase {
namespace sql {
class ObRandomAddrsProvider : public ObAddrsProvider {
public:
ObRandomAddrsProvider();
virtual ~ObRandomAddrsProvider();
virtual int select_servers(int64_t select_count, common::ObIArray<common::ObAddr>& servers);
VIRTUAL_TO_STRING_KV(K_(servers));
private:
common::ObSEArray<common::ObAddr, 32> servers_;
DISALLOW_COPY_AND_ASSIGN(ObRandomAddrsProvider);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_RANDOM_ADDRS_PROVIDER_ */

View File

@ -0,0 +1,80 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_range_hash_key_getter.h"
#include "share/schema/ob_table_schema.h"
using namespace oceanbase::sql;
using namespace oceanbase::common;
using namespace oceanbase::share::schema;
int ObRangeHashKeyGetter::get_part_subpart_obj_idxs(int64_t& part_obj_idx, int64_t& subpart_obj_idx) const
{
int ret = OB_SUCCESS;
if (OB_INVALID_ID == repartition_table_id_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("slice table id is invalid", K(ret));
} else {
if (OB_SUCC(ret) && repart_columns_.count() > 0) {
if (repart_columns_.count() > 1) {
ret = OB_NOT_IMPLEMENT;
LOG_WARN("multi columns partition key is not supported now", K(ret));
} else {
part_obj_idx = repart_columns_.at(0).index_;
}
} else {
part_obj_idx = -1;
}
if (OB_SUCC(ret) && repart_sub_columns_.count() > 0) {
if (repart_sub_columns_.count() > 1) {
ret = OB_NOT_IMPLEMENT;
LOG_WARN("multi columns subpartition key is not supported now", K(ret));
} else {
subpart_obj_idx = repart_sub_columns_.at(0).index_;
}
} else {
subpart_obj_idx = -1;
}
}
return ret;
}
/*
int ObRangeHashKeyGetter::get_part_subpart_idx(const ObTableSchema *table_schema,
int64_t slice_idx,
int64_t &part_idx,
int64_t &subpart_idx) const
{
int ret = OB_SUCCESS;
if (NULL != table_schema) {
int64_t part_obj_idx = -1;
int64_t subpart_obj_idx = -1;
if (OB_FAIL(get_part_subpart_obj_idxs(part_obj_idx, subpart_obj_idx))) {
LOG_WARN("fail to get part and subpart obj idxs", K(ret));
}
if (part_obj_idx < 0 && subpart_obj_idx < 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("part or subpart obj idx should not less than 0", K(part_obj_idx), K(subpart_obj_idx), K(ret));
} else if (part_obj_idx >= 0 && subpart_obj_idx >= 0) {
part_idx = slice_idx / table_schema->get_sub_part_num();
subpart_idx = slice_idx % table_schema->get_sub_part_num();
} else {
part_idx = (part_obj_idx >= 0) ? slice_idx : -1;
subpart_idx = (subpart_obj_idx >= 0) ? slice_idx : -1;
}
} else {
part_idx = 0;
subpart_idx = 0;
}
return ret;
}*/

View File

@ -0,0 +1,55 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef _OB_SQ_OB_RANGE_HASH_KEY_GETTER_H_
#define _OB_SQ_OB_RANGE_HASH_KEY_GETTER_H_
#include "common/ob_partition_key.h"
#include "sql/executor/ob_transmit.h"
#include "sql/executor/ob_task_event.h"
namespace oceanbase {
namespace share {
namespace schema {
class ObTableSchema;
}
} // namespace share
namespace sql {
class ObRangeHashKeyGetter {
public:
ObRangeHashKeyGetter(const int64_t& repartition_table_id,
const common::ObFixedArray<ObTransmitRepartColumn, common::ObIAllocator>& repart_columns,
const common::ObFixedArray<ObTransmitRepartColumn, common::ObIAllocator>& repart_sub_columns)
: repartition_table_id_(repartition_table_id),
repart_columns_(repart_columns),
repart_sub_columns_(repart_sub_columns)
{}
~ObRangeHashKeyGetter() = default;
int get_part_subpart_obj_idxs(int64_t& part_obj_idx, int64_t& subpart_obj_idx) const;
// int get_part_subpart_idx(const share::schema::ObTableSchema *table_schema,
// int64_t slice_idx,
// int64_t &part_idx,
// int64_t &subpart_idx) const;
private:
const int64_t& repartition_table_id_;
const common::ObFixedArray<ObTransmitRepartColumn, common::ObIAllocator>& repart_columns_;
const common::ObFixedArray<ObTransmitRepartColumn, common::ObIAllocator>& repart_sub_columns_;
};
} // namespace sql
} // namespace oceanbase
#endif

View File

@ -0,0 +1,128 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_receive.h"
#include "sql/executor/ob_transmit.h"
#include "sql/executor/ob_job.h"
#include "share/ob_define.h"
#include "sql/engine/ob_exec_context.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
//
//
//////////////// ObReceiveInput ////////////////////
//
//
ObReceiveInput::ObReceiveInput()
: pull_slice_id_(common::OB_INVALID_ID), child_job_id_(common::OB_INVALID_ID), child_op_id_(common::OB_INVALID_ID)
{}
ObReceiveInput::~ObReceiveInput()
{}
void ObReceiveInput::reset()
{
pull_slice_id_ = OB_INVALID_ID;
child_job_id_ = OB_INVALID_ID;
child_op_id_ = OB_INVALID_ID;
}
int ObReceiveInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& cur_op)
{
int ret = OB_SUCCESS;
ObTransmitInput* transmit_input = NULL;
ObJob* child_job = NULL;
// meta data
pull_slice_id_ = task_info.get_pull_slice_id();
// That's a long way to get the child job of cur_op:
// cur_op -> child_op -> child_op_input -> job
task_locs_.reset();
for (int32_t i = 0; OB_SUCC(ret) && i < cur_op.get_child_num(); ++i) {
ObPhyOperator* trans_op = cur_op.get_child(i);
if (OB_ISNULL(trans_op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail get child", K(ret));
} else if (!IS_TRANSMIT(trans_op->get_type())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child op is not ObTransmit", K(ret), K(cur_op.get_id()), K(trans_op->get_type()));
} else if (OB_ISNULL(transmit_input = GET_PHY_OP_INPUT(ObTransmitInput, ctx, trans_op->get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tranmit op ctx is NULL", K(ret), K(cur_op.get_id()));
} else if (OB_ISNULL(child_job = transmit_input->get_job())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("child op's job is NULL", K(ret));
} else if (OB_FAIL(child_job->get_finished_task_locations(task_locs_))) {
LOG_WARN("fail to get finished task locations", K(ret));
} else {
child_job_id_ = child_job->get_job_id();
child_op_id_ = trans_op->get_id();
}
}
return ret;
}
int ObReceiveInput::get_result_location(const int64_t child_job_id, const int64_t child_task_id, ObAddr& svr) const
{
UNUSED(child_job_id);
UNUSED(child_task_id);
UNUSED(svr);
// find data in task_locs_
return OB_NOT_IMPLEMENT;
}
OB_SERIALIZE_MEMBER(ObReceiveInput, pull_slice_id_, child_job_id_, task_locs_);
//
//
//////////////// ObReceiveCtx ////////////////////
//
//
ObReceive::ObReceiveCtx::ObReceiveCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx)
{}
ObReceive::ObReceiveCtx::~ObReceiveCtx()
{}
//
//
//////////////// ObReceive ////////////////////
//
//
ObReceive::ObReceive(ObIAllocator& alloc)
: ObSingleChildPhyOperator(alloc),
partition_order_specified_(false),
need_set_affected_row_(false),
is_merge_sort_(false)
{}
ObReceive::~ObReceive()
{}
int ObReceive::switch_iterator(ObExecContext& ctx) const
{
UNUSED(ctx);
// exchange operator not support switch iterator, return OB_ITER_END directly
return OB_ITER_END;
}
OB_SERIALIZE_MEMBER((ObReceive, ObSingleChildPhyOperator), partition_order_specified_, need_set_affected_row_);
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,142 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef __OB_SQLL_EXECUTOR_RECEIVE_OPERATOR__
#define __OB_SQLL_EXECUTOR_RECEIVE_OPERATOR__
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_single_child_phy_operator.h"
#include "sql/executor/ob_task_location.h"
#include "sql/executor/ob_slice_id.h"
#include "share/ob_scanner.h"
#include "lib/container/ob_array_serialization.h"
#define IS_RECEIVE(type) \
(((type) == PHY_FIFO_RECEIVE) || ((type) == PHY_FIFO_RECEIVE_V2) || ((type) == PHY_PX_FIFO_RECEIVE) || \
((type) == PHY_PX_MERGE_SORT_RECEIVE) || ((type) == PHY_PX_FIFO_COORD) || ((type) == PHY_PX_MERGE_SORT_COORD) || \
((type) == PHY_TASK_ORDER_RECEIVE) || ((type) == PHY_MERGE_SORT_RECEIVE) || ((type) == PHY_DIRECT_RECEIVE))
#define IS_ASYNC_RECEIVE(type) \
(((type) == PHY_FIFO_RECEIVE_V2) || ((type) == PHY_TASK_ORDER_RECEIVE) || ((type) == PHY_MERGE_SORT_RECEIVE))
#define IS_TABLE_INSERT(type) \
(((type) == PHY_INSERT) || ((type) == PHY_REPLACE) || ((type) == PHY_INSERT_ON_DUP) || \
((type) == PHY_INSERT_RETURNING))
namespace oceanbase {
namespace sql {
class ObReceiveInput : public ObIPhyOperatorInput {
friend class ObReceive;
OB_UNIS_VERSION_V(1);
public:
ObReceiveInput();
virtual ~ObReceiveInput();
virtual void reset() override;
// Setup
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
// Use
inline uint64_t get_pull_slice_id() const
{
return pull_slice_id_;
}
inline int64_t get_child_job_id() const
{
return child_job_id_;
}
inline uint64_t get_child_op_id() const
{
return child_op_id_;
};
int get_result_location(const int64_t child_job_id, const int64_t child_task_id, common::ObAddr& svr) const;
protected:
uint64_t pull_slice_id_;
int64_t child_job_id_;
uint64_t child_op_id_;
common::ObSArray<ObTaskLocation> task_locs_;
};
class ObReceive : public ObSingleChildPhyOperator {
OB_UNIS_VERSION_V(1);
protected:
class ObReceiveCtx : public ObPhyOperatorCtx {
public:
explicit ObReceiveCtx(ObExecContext& ctx);
virtual ~ObReceiveCtx();
virtual void destroy()
{
ObPhyOperatorCtx::destroy_base();
}
};
public:
explicit ObReceive(common::ObIAllocator& alloc);
virtual ~ObReceive();
virtual int switch_iterator(ObExecContext& ctx) const override;
void set_partition_order_specified(bool order_specified)
{
partition_order_specified_ = order_specified;
}
virtual bool is_receive_op() const override
{
return true;
}
void set_is_merge_sort(bool is_merge_sort)
{
is_merge_sort_ = is_merge_sort;
}
bool is_merge_sort() const
{
return is_merge_sort_;
}
void set_need_set_affected_row(bool b)
{
need_set_affected_row_ = b;
}
bool get_need_set_affected_row() const
{
return need_set_affected_row_;
}
virtual OperatorOpenOrder get_operator_open_order(ObExecContext& ctx) const
{
UNUSED(ctx);
return OPEN_SELF_ONLY;
}
virtual int drain_exch(ObExecContext& ctx) const override
{
// Drain exchange is used in parallelism execution,
// do nothing for old fashion distributed execution.
UNUSED(ctx);
return common::OB_SUCCESS;
}
protected:
bool partition_order_specified_;
bool need_set_affected_row_;
bool is_merge_sort_;
private:
// disallow copy
ObReceive(const ObReceive& other);
ObReceive& operator=(const ObReceive& ohter);
};
} // namespace sql
} // namespace oceanbase
#endif /* __OB_SQLL_EXECUTOR_RECEIVE_OPERATOR__ */
//// end of header file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,237 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OBDEV_SRC_SQL_EXECUTOR_OB_REMOTE_EXECUTOR_PROCESSOR_H_
#define OBDEV_SRC_SQL_EXECUTOR_OB_REMOTE_EXECUTOR_PROCESSOR_H_
#include "sql/executor/ob_executor_rpc_processor.h"
namespace observer {
class ObGlobalContext;
}
namespace oceanbase {
namespace sql {
template <typename T>
class ObRemoteBaseExecuteP : public obrpc::ObRpcProcessor<T> {
public:
ObRemoteBaseExecuteP(const observer::ObGlobalContext& gctx, bool is_execute_remote_plan = false)
: obrpc::ObRpcProcessor<T>(),
gctx_(gctx),
exec_ctx_(CURRENT_CONTEXT.get_arena_allocator(), gctx.session_mgr_),
vt_iter_factory_(*gctx_.vt_iter_creator_),
sql_ctx_(),
trans_state_(),
exec_errcode_(common::OB_SUCCESS),
process_timestamp_(0),
exec_start_timestamp_(0),
exec_end_timestamp_(0),
has_send_result_(false),
is_execute_remote_plan_(is_execute_remote_plan)
{
obrpc::ObRpcProcessor<T>::set_preserve_recv_data();
}
virtual ~ObRemoteBaseExecuteP()
{}
int64_t get_exec_start_timestamp() const
{
return exec_start_timestamp_;
}
int64_t get_exec_end_timestamp() const
{
return exec_end_timestamp_;
}
int64_t get_process_timestamp() const
{
return process_timestamp_;
}
int64_t get_single_process_timestamp() const
{
return exec_start_timestamp_;
}
bool is_execute_remote_plan() const
{
return is_execute_remote_plan_;
}
protected:
int base_init();
int base_before_process(
int64_t tenant_schema_version, int64_t sys_schema_version, const DependenyTableStore& dependency_tables);
int auto_start_phy_trans(ObPartitionLeaderArray& leader_parts);
int auto_end_phy_trans(bool rollback, const common::ObPartitionArray& participants);
int execute_remote_plan(ObExecContext& exec_ctx, const ObPhysicalPlan& plan);
int execute_with_sql(ObRemoteTask& task);
int sync_send_result(ObExecContext& exec_ctx, const ObPhysicalPlan& plan, common::ObScanner& scanner);
virtual int send_result_to_controller(ObExecContext& exec_ctx, const ObPhysicalPlan& plan) = 0;
virtual void record_exec_timestamp(bool is_first, ObExecTimestamp& exec_timestamp)
{
ObExecStatUtils::record_exec_timestamp(*this, is_first, exec_timestamp);
}
void record_sql_audit_and_plan_stat(const ObPhysicalPlan* plan, ObSQLSessionInfo* session, ObExecRecord exec_record,
ObExecTimestamp exec_timestamp, ObWaitEventDesc& max_wait_desc, ObWaitEventStat& total_wait_desc);
int base_before_response(common::ObScanner& scanner);
int base_after_process();
void base_cleanup();
virtual void clean_result_buffer() = 0;
bool query_can_retry_in_remote(int& last_err, int& err, ObSQLSessionInfo& session, int64_t& retry_times);
protected:
const observer::ObGlobalContext& gctx_;
sql::ObDesExecContext exec_ctx_;
ObSqlPartitionLocationCache partition_location_cache_;
observer::ObVirtualTableIteratorFactory vt_iter_factory_;
sql::ObSqlCtx sql_ctx_;
share::schema::ObSchemaGetterGuard schema_guard_;
/*
* Used to record whether the transaction statement has been executed,
* and then determine whether the corresponding end statement needs to be executed
*/
TransState trans_state_;
int exec_errcode_;
int64_t process_timestamp_;
int64_t exec_start_timestamp_;
int64_t exec_end_timestamp_;
bool has_send_result_;
bool is_execute_remote_plan_; // only execute remote physical_plan not sql_string
};
/* Handle remote single partition situation (REMOTE) */
class ObRpcRemoteExecuteP : public ObRemoteBaseExecuteP<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_EXECUTE> > {
public:
ObRpcRemoteExecuteP(const observer::ObGlobalContext& gctx) : ObRemoteBaseExecuteP(gctx, true)
{}
virtual ~ObRpcRemoteExecuteP()
{}
virtual int init();
protected:
virtual int send_result_to_controller(ObExecContext& exec_ctx, const ObPhysicalPlan& plan) override;
virtual int before_process();
virtual int process();
virtual int before_response();
virtual int after_process();
virtual void cleanup();
virtual void clean_result_buffer() override;
private:
int get_participants(common::ObPartitionLeaderArray& pla);
private:
ObPhysicalPlan phy_plan_;
};
class ObRpcRemoteSyncExecuteP
: public ObRemoteBaseExecuteP<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_SYNC_EXECUTE> > {
public:
ObRpcRemoteSyncExecuteP(const observer::ObGlobalContext& gctx) : ObRemoteBaseExecuteP(gctx)
{}
virtual ~ObRpcRemoteSyncExecuteP()
{}
virtual int init();
protected:
virtual int send_result_to_controller(ObExecContext& exec_ctx, const ObPhysicalPlan& plan) override;
virtual int before_process();
virtual int process();
virtual int before_response();
virtual int after_process();
virtual void cleanup();
virtual void clean_result_buffer() override;
};
class ObRpcRemoteASyncExecuteP
: public ObRemoteBaseExecuteP<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_ASYNC_EXECUTE> > {
public:
ObRpcRemoteASyncExecuteP(const observer::ObGlobalContext& gctx)
: ObRemoteBaseExecuteP(gctx), remote_result_(), is_from_batch_(false)
{}
virtual ~ObRpcRemoteASyncExecuteP()
{}
virtual int init();
ObRemoteTask& get_arg()
{
return arg_;
}
virtual int before_process();
virtual int process();
virtual int before_response();
virtual int after_process();
virtual void cleanup();
void set_from_batch()
{
is_from_batch_ = true;
}
protected:
virtual int send_result_to_controller(ObExecContext& exec_ctx, const ObPhysicalPlan& plan) override;
int send_remote_result(ObRemoteResult& remote_result);
virtual void clean_result_buffer() override;
private:
ObRemoteResult remote_result_;
bool is_from_batch_;
};
class ObRpcRemotePostResultP
: public obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_POST_RESULT> > {
public:
ObRpcRemotePostResultP(const observer::ObGlobalContext& gctx) : gctx_(gctx), is_from_batch_(false)
{
set_preserve_recv_data();
}
virtual ~ObRpcRemotePostResultP()
{}
void set_from_batch()
{
is_from_batch_ = true;
}
virtual int init();
ObRemoteResult& get_arg()
{
return arg_;
}
virtual int before_process()
{
int ret = common::OB_SUCCESS;
if (!is_from_batch_) {
ret = obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_POST_RESULT> >::before_process();
}
return ret;
}
virtual int process();
virtual int before_response()
{
int ret = common::OB_SUCCESS;
if (!is_from_batch_) {
ret = obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_POST_RESULT> >::before_response();
}
return ret;
}
virtual int after_process()
{
int ret = common::OB_SUCCESS;
if (!is_from_batch_) {
ret = obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_POST_RESULT> >::after_process();
}
return ret;
}
virtual void cleanup()
{
if (!is_from_batch_) {
obrpc::ObRpcProcessor<obrpc::ObExecutorRpcProxy::ObRpc<obrpc::OB_REMOTE_POST_RESULT> >::cleanup();
}
}
private:
const observer::ObGlobalContext& gctx_;
bool is_from_batch_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OBDEV_SRC_SQL_EXECUTOR_OB_REMOTE_EXECUTOR_PROCESSOR_H_ */

View File

@ -0,0 +1,116 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_remote_identity_task_spliter.h"
#include "sql/executor/ob_task_info.h"
#include "sql/executor/ob_job.h"
#include "sql/executor/ob_receive.h"
#include "lib/allocator/ob_allocator.h"
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/table/ob_table_scan.h"
#include "lib/container/ob_array.h"
#include "lib/utility/ob_tracepoint.h"
#include "sql/engine/ob_exec_context.h"
using namespace oceanbase::common;
using namespace oceanbase::share;
namespace oceanbase {
namespace sql {
ObRemoteIdentityTaskSpliter::ObRemoteIdentityTaskSpliter() : ObTaskSpliter(), task_(NULL)
{}
ObRemoteIdentityTaskSpliter::~ObRemoteIdentityTaskSpliter()
{
if (OB_LIKELY(NULL != task_)) {
task_->~ObTaskInfo();
task_ = NULL;
}
}
int ObRemoteIdentityTaskSpliter::get_next_task(ObTaskInfo*& task)
{
int ret = OB_SUCCESS;
ObPhyOperator* root_op = NULL;
void* ptr = NULL;
if (OB_I(t1)(OB_ISNULL(plan_ctx_) || OB_ISNULL(exec_ctx_) || OB_ISNULL(allocator_) || OB_ISNULL(job_) ||
OB_ISNULL(job_conf_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("plan ctx or executor ctx or allocator or job or job conf is NULL",
K(ret),
K(plan_ctx_),
K(exec_ctx_),
K(allocator_),
K(job_),
K(job_conf_));
} else if (NULL != task_) {
ret = OB_ITER_END;
} else if (OB_ISNULL(root_op = job_->get_root_op())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("root op is NULL", K(ret));
} else {
ObPhyTableLocationIArray& table_locations = exec_ctx_->get_task_exec_ctx().get_table_locations();
if (OB_UNLIKELY(table_locations.count() < 1)) { // t1 join t2 may have multiple locations
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get phy table location", K(ret), "expect", 1, "acutal", table_locations.count());
} else {
const ObPartitionReplicaLocationIArray& partition_loc_list = table_locations.at(0).get_partition_location_list();
// the case of t1 union t1, the case of t1(p0) union t2(p0), etc.,
// Both are in remote mode, but the count of table_loc_list may be greater than 1
// The optimizer must ensure that in remote mode, the locations of all tables are the same,
// and they are all single partitions.
if (OB_UNLIKELY(1 > partition_loc_list.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("there must be at least one table location", K(partition_loc_list.count()));
} else {
const ObReplicaLocation& replica_loc = partition_loc_list.at(0).get_replica_location();
if (!replica_loc.is_valid()) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("replica location is invalid", K(ret), K(partition_loc_list.at(0)));
} else if (OB_ISNULL(ptr = allocator_->alloc(sizeof(ObTaskInfo)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc ObTaskInfo", K(ret));
} else {
task_ = new (ptr) ObTaskInfo(*allocator_);
ObTaskID ob_task_id;
ObTaskLocation task_loc;
ob_task_id.set_ob_job_id(job_->get_ob_job_id());
ob_task_id.set_task_id(0);
task_loc.set_ob_task_id(ob_task_id);
task_loc.set_server(replica_loc.server_);
task_->set_task_split_type(get_type());
task_->set_pull_slice_id(0);
task_->set_location_idx(0);
task_->set_task_location(task_loc);
task_->set_root_op(job_->get_root_op());
task_->set_state(OB_TASK_STATE_NOT_INIT);
if (OB_FAIL(task_->init_location_idx_array(1))) {
LOG_WARN("init location idx array failed", K(ret));
} else if (OB_FAIL(task_->add_location_idx(0))) {
LOG_WARN("add location index to task failed", K(ret));
} else {
// The purpose of task_ as a class member is
// to ensure that the second call to get_next_task can return OB_ITER_END
task = task_;
}
}
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,41 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_REMOTE_IDENTITY_TASK_SPLITER_
#define OCEANBASE_SQL_EXECUTOR_OB_REMOTE_IDENTITY_TASK_SPLITER_
#include "sql/executor/ob_task_spliter.h"
#include "lib/container/ob_array.h"
namespace oceanbase {
namespace sql {
class ObPhysicalPlan;
class ObTaskInfo;
class ObRemoteIdentityTaskSpliter : public ObTaskSpliter {
public:
ObRemoteIdentityTaskSpliter();
virtual ~ObRemoteIdentityTaskSpliter();
virtual int get_next_task(ObTaskInfo*& task);
inline virtual TaskSplitType get_type() const
{
return ObTaskSpliter::REMOTE_IDENTITY_SPLIT;
}
private:
ObTaskInfo* task_;
private:
DISALLOW_COPY_AND_ASSIGN(ObRemoteIdentityTaskSpliter);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_REMOTE_IDENTITY_TASK_SPLITER_ */

View File

@ -0,0 +1,49 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_remote_job_control.h"
using namespace oceanbase::common;
namespace oceanbase {
namespace sql {
ObRemoteJobControl::ObRemoteJobControl()
{}
ObRemoteJobControl::~ObRemoteJobControl()
{}
int ObRemoteJobControl::get_ready_jobs(ObIArray<ObJob*>& jobs, bool serial_sched) const
{
int ret = OB_SUCCESS;
UNUSED(serial_sched);
for (int64_t i = 0; OB_SUCC(ret) && i < jobs_.count(); ++i) {
ObJob* job = jobs_.at(i);
if (OB_ISNULL(job)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job is NULL", K(ret));
} else if (OB_JOB_STATE_INITED == job->get_state()) {
if (OB_FAIL(jobs.push_back(job))) {
LOG_WARN("fail to push back job", K(ret), K(i));
}
}
}
if (OB_FAIL(ret)) {
} else if (2 != jobs.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("the count of ready jobs is not 2", K(jobs.count()));
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,32 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_REMOTE_JOB_CONTROL_
#define OCEANBASE_SQL_EXECUTOR_OB_REMOTE_JOB_CONTROL_
#include "sql/executor/ob_job_control.h"
namespace oceanbase {
namespace sql {
class ObRemoteJobControl : public ObJobControl {
public:
explicit ObRemoteJobControl();
virtual ~ObRemoteJobControl();
virtual int get_ready_jobs(common::ObIArray<ObJob*>& jobs, bool serial_schedule = false) const;
private:
DISALLOW_COPY_AND_ASSIGN(ObRemoteJobControl);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_REMOTE_JOB_CONTROL_ */

View File

@ -0,0 +1,79 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "lib/container/ob_array.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/executor/ob_job.h"
#include "sql/executor/ob_task_info.h"
#include "sql/executor/ob_remote_job_executor.h"
namespace oceanbase {
namespace sql {
using namespace oceanbase::common;
ObRemoteJobExecutor::ObRemoteJobExecutor() : job_(NULL), executor_(NULL)
{}
ObRemoteJobExecutor::~ObRemoteJobExecutor()
{}
int ObRemoteJobExecutor::execute(ObExecContext& query_ctx)
{
int ret = OB_SUCCESS;
ObTaskInfo* task_info = NULL;
if (OB_ISNULL(job_) || OB_ISNULL(executor_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job_ or executor_ is NULL", K(ret), K(job_), K(executor_));
} else if (OB_FAIL(get_executable_task(query_ctx, task_info))) { // get task info
LOG_WARN("fail get a executable task", K(ret));
} else if (OB_ISNULL(task_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task info is NULL", K(ret));
} else if (OB_FAIL(executor_->execute(query_ctx, job_,
task_info))) { // job_ + task_info as task's frame and param
LOG_WARN("fail execute task", K(ret), K(*task_info));
} else {
}
return ret;
}
int ObRemoteJobExecutor::get_executable_task(ObExecContext& ctx, ObTaskInfo*& task_info)
{
int ret = OB_SUCCESS;
ObTaskControl* tq = NULL;
ObArray<ObTaskInfo*> ready_tasks;
if (OB_ISNULL(job_) || OB_ISNULL(executor_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("job_ or executor_ is NULL", K(ret), K(job_), K(executor_));
} else if (OB_FAIL(job_->get_task_control(ctx, tq))) {
LOG_WARN("fail get task control", K(ret));
} else if (OB_ISNULL(tq)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("succ to get task control, but task control is NULL", K(ret));
} else if (OB_FAIL(tq->get_ready_tasks(ready_tasks))) {
LOG_WARN("fail get ready task", K(ret));
} else if (OB_UNLIKELY(1 != ready_tasks.count())) {
LOG_WARN("unexpected ready task count", "count", ready_tasks.count());
} else if (OB_FAIL(ready_tasks.at(0, task_info))) {
LOG_WARN("fail get task from array", K(ret));
} else {
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,56 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_OB_REMOTE_JOB_EXECUTOR_
#define OCEANBASE_SQL_EXECUTOR_OB_REMOTE_JOB_EXECUTOR_
namespace oceanbase {
namespace sql {
class ObTaskInfo;
class ObJob;
class ObTaskExecutor;
class ObExecContext;
class ObRemoteJobExecutor {
public:
ObRemoteJobExecutor();
virtual ~ObRemoteJobExecutor();
// set job waiting for schedule
void set_job(ObJob& job)
{
job_ = &job;
}
void set_task_executor(ObTaskExecutor& executor)
{
executor_ = &executor;
}
// schedule a job, distribute and execute tasks in the job.
int execute(ObExecContext& ctx);
inline void reset()
{
job_ = NULL;
executor_ = NULL;
}
private:
// disallow copy
ObRemoteJobExecutor(const ObRemoteJobExecutor& other);
ObRemoteJobExecutor& operator=(const ObRemoteJobExecutor& ohter);
int get_executable_task(ObExecContext& ctx, ObTaskInfo*& task);
private:
ObJob* job_;
ObTaskExecutor* executor_;
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_OB_REMOTE_JOB_EXECUTOR_ */
//// end of header file

View File

@ -0,0 +1,331 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/executor/ob_remote_scheduler.h"
#include "sql/executor/ob_remote_job_control.h"
#include "sql/executor/ob_task_spliter_factory.h"
#include "sql/executor/ob_addrs_provider_factory.h"
#include "sql/executor/ob_remote_job_executor.h"
#include "sql/executor/ob_remote_task_executor.h"
#include "sql/executor/ob_local_job_executor.h"
#include "sql/executor/ob_local_task_executor.h"
#include "sql/executor/ob_job.h"
#include "share/partition_table/ob_partition_location.h"
#include "sql/executor/ob_job_parser.h"
#include "sql/executor/ob_task_executor_ctx.h"
#include "sql/engine/ob_phy_operator.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "share/ob_define.h"
#include "lib/utility/utility.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/ob_query_exec_ctx_mgr.h"
namespace oceanbase {
using namespace oceanbase::common;
namespace sql {
ObRemoteScheduler::ObRemoteScheduler()
{}
ObRemoteScheduler::~ObRemoteScheduler()
{}
int ObRemoteScheduler::schedule(ObExecContext& ctx, ObPhysicalPlan* phy_plan)
{
int ret = OB_SUCCESS;
if (ctx.use_remote_sql()) {
if (OB_ISNULL(ctx.get_sql_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is null", K(ret), K(ctx.get_sql_ctx()));
} else {
if (ctx.get_sql_ctx()->is_execute_async_) {
if (OB_FAIL(async_execute_with_sql(ctx, phy_plan))) {
LOG_WARN("async execute with sql failed", K(ret));
}
} else if (OB_FAIL(execute_with_sql(ctx, phy_plan))) {
LOG_WARN("execute with sql failed", K(ret));
}
}
} else if (OB_FAIL(execute_with_plan(ctx, phy_plan))) {
LOG_WARN("execute with plan failed", K(ret));
}
return ret;
}
int ObRemoteScheduler::execute_with_plan(ObExecContext& ctx, ObPhysicalPlan* phy_plan)
{
// 1. Split and construct task using ObJobConf info
// 2. Call job.schedule()
int ret = OB_SUCCESS;
ObJobParser parser;
ObLocalTaskExecutor local_task_executor;
ObLocalJobExecutor local_job_executor;
ObRemoteTaskExecutor remote_task_executor;
ObRemoteJobExecutor remote_job_executor;
ObSEArray<ObJob*, 2> jobs;
ObJob* root_job = NULL;
ObJob* remote_job = NULL;
ObRemoteJobControl jc;
ObTaskSpliterFactory task_factory;
ObPhysicalPlanCtx* plan_ctx = NULL;
ObTaskExecutorCtx& task_exec_ctx = ctx.get_task_exec_ctx();
if (OB_ISNULL(phy_plan)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("not init", K(phy_plan), K(ret));
} else if (OB_FAIL(task_exec_ctx.reset_and_init_stream_handler())) {
LOG_WARN("reset and init stream handler failed", K(ret));
} else if (OB_UNLIKELY(NULL == (plan_ctx = ctx.get_physical_plan_ctx()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("plan ctx is NULL", K(ret));
} else {
ObExecutionID ob_execution_id;
ob_execution_id.set_server(task_exec_ctx.get_self_addr());
ob_execution_id.set_execution_id(OB_INVALID_ID);
if (OB_FAIL(parser.parse_job(ctx, phy_plan, ob_execution_id, task_factory, jc))) {
LOG_WARN("fail parse job for scheduler.", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(jc.get_ready_jobs(jobs))) {
LOG_WARN("fail get jobs.", K(ret));
} else if (OB_UNLIKELY(2 != jobs.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected job count. expect 2, actual", "job_count", jobs.count());
} else if (OB_UNLIKELY(NULL == (root_job = jobs.at(0)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null job", K(ret));
} else if (OB_UNLIKELY(NULL == (remote_job = jobs.at(1)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null job", K(ret));
} else {
local_job_executor.set_task_executor(local_task_executor);
local_job_executor.set_job(*root_job);
remote_job_executor.set_task_executor(remote_task_executor);
remote_job_executor.set_job(*remote_job);
if (OB_FAIL(remote_job_executor.execute(ctx))) {
LOG_WARN("fail execute remote job", K(ret));
} else if (OB_FAIL(local_job_executor.execute(ctx))) {
LOG_WARN("fail execute local job", K(ret));
}
}
}
if (OB_FAIL(ret)) {
int print_ret = OB_SUCCESS;
const static int64_t MAX_JC_STATUS_BUF_LEN = 4096;
char jc_status_buf[MAX_JC_STATUS_BUF_LEN];
if (OB_SUCCESS != (print_ret = jc.print_status(jc_status_buf, MAX_JC_STATUS_BUF_LEN))) {
LOG_WARN("fail to print job control status", K(ret), K(print_ret), LITERAL_K(MAX_JC_STATUS_BUF_LEN));
} else {
LOG_WARN("fail to schedule, print job's status", K(ret), K(print_ret), "job_status", jc_status_buf);
}
}
return ret;
}
int ObRemoteScheduler::build_remote_task(
ObExecContext& ctx, ObRemoteTask& remote_task, const DependenyTableStore& dependency_tables)
{
int ret = OB_SUCCESS;
ObPhysicalPlanCtx* plan_ctx = ctx.get_physical_plan_ctx();
ObTaskExecutorCtx& task_exec_ctx = ctx.get_task_exec_ctx();
ObSQLSessionInfo* session = nullptr;
if (OB_FAIL(remote_task.assign_dependency_tables(dependency_tables))) {
LOG_WARN("fail to assign dependency_tables", K(ret));
}
remote_task.set_ctrl_server(ctx.get_addr());
remote_task.set_session(ctx.get_my_session());
remote_task.set_query_schema_version(
task_exec_ctx.get_query_tenant_begin_schema_version(), task_exec_ctx.get_query_sys_begin_schema_version());
remote_task.set_remote_sql_info(&plan_ctx->get_remote_sql_info());
const share::ObPartitionReplicaLocation* replica_loc = nullptr;
ObPhyTableLocationIArray& table_locations = task_exec_ctx.get_table_locations();
if (OB_UNLIKELY(table_locations.empty()) ||
OB_ISNULL(replica_loc = table_locations.at(0).get_part_replic_by_index(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid argument", K(ret), K(table_locations));
} else if (OB_ISNULL(session = ctx.get_my_session())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is null", K(ret));
} else {
remote_task.set_runner_svr(replica_loc->get_replica_location().server_);
ObTaskID task_id;
task_id.set_execution_id(session->get_current_execution_id());
task_id.set_server(ctx.get_addr());
task_id.set_task_id(0);
remote_task.set_task_id(task_id);
ObITaskExecCtx* cur_task_ctx = nullptr;
ObQueryExecCtx* query_ctx = nullptr;
if (OB_NOT_NULL(query_ctx = ctx.get_query_exec_ctx()) &&
OB_NOT_NULL(cur_task_ctx = query_ctx->get_cur_task_ctx())) {
ObRemoteTaskCtx* task_ctx = static_cast<ObRemoteTaskCtx*>(cur_task_ctx);
task_ctx->set_runner_svr(remote_task.get_runner_svr());
}
}
return ret;
}
int ObRemoteScheduler::execute_with_sql(ObExecContext& ctx, ObPhysicalPlan* phy_plan)
{
int ret = OB_SUCCESS;
RemoteExecuteStreamHandle* handler = NULL;
ObSQLSessionInfo* session = ctx.get_my_session();
ObPhysicalPlanCtx* plan_ctx = ctx.get_physical_plan_ctx();
ObTaskExecutorCtx& task_exec_ctx = ctx.get_task_exec_ctx();
ObExecuteResult& exec_result = task_exec_ctx.get_execute_result();
ObExecutorRpcImpl* rpc = NULL;
ObQueryRetryInfo* retry_info = NULL;
ObRemoteTask task;
bool has_sent_task = false;
bool has_transfer_err = false;
bool has_merge_err = false;
if (OB_ISNULL(phy_plan) || OB_ISNULL(session) || OB_ISNULL(plan_ctx)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(phy_plan), K(session), K(plan_ctx));
} else if (OB_FAIL(task_exec_ctx.reset_and_init_stream_handler())) {
LOG_WARN("reset and init stream handler failed", K(ret));
} else if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(ctx, handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_FAIL(ObTaskExecutorCtxUtil::get_task_executor_rpc(ctx, rpc))) {
LOG_WARN("fail get executor rpc", K(ret));
} else if (OB_ISNULL(session) || OB_ISNULL(plan_ctx) || OB_ISNULL(handler) || OB_ISNULL(rpc) ||
OB_ISNULL(retry_info = &session->get_retry_info_for_update())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("unexpected null ptr", K(ret), K(session), K(plan_ctx), K(handler), K(rpc), K(retry_info));
} else if (FALSE_IT(handler->set_use_remote_protocol_v2())) {
} else if (OB_FAIL(handler->reset_and_init_result())) {
LOG_WARN("fail to reset and init result", K(ret));
} else if (OB_FAIL(build_remote_task(ctx, task, phy_plan->get_dependency_table()))) {
LOG_WARN("build remote task failed", K(ret), K(task));
} else {
LOG_DEBUG("execute remote task", K(task));
if (NULL == phy_plan->get_root_op_spec()) {
exec_result.set_root_op(phy_plan->get_main_query());
} else {
LOG_DEBUG("static engine remote execute");
ObOperator* op = NULL;
if (OB_FAIL(phy_plan->get_root_op_spec()->create_operator(ctx, op))) {
LOG_WARN("create operator from spec failed", K(ret));
} else if (OB_ISNULL(op)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("created operator is NULL", K(ret));
} else {
exec_result.set_static_engine_root(op);
}
}
}
if (OB_SUCC(ret)) {
ObScanner* scanner = NULL;
ObExecutorRpcCtx rpc_ctx(session->get_rpc_tenant_id(),
plan_ctx->get_timeout_timestamp(),
ctx.get_task_exec_ctx().get_min_cluster_version(),
retry_info,
ctx.get_my_session(),
plan_ctx->is_plain_select_stmt());
if (OB_FAIL(
rpc->task_execute_v2(rpc_ctx, task, task.get_runner_svr(), *handler, has_sent_task, has_transfer_err))) {
bool skip_failed_tasks = false;
int check_ret = OB_SUCCESS;
int add_ret = OB_SUCCESS;
if (is_data_not_readable_err(ret)) {
if (OB_UNLIKELY(
OB_SUCCESS != (add_ret = retry_info->add_invalid_server_distinctly(task.get_runner_svr(), true)))) {
LOG_WARN(
"fail to add remote addr to invalid servers distinctly", K(ret), K(add_ret), K(task), K(*retry_info));
}
}
}
int saved_ret = ret;
if (OB_ISNULL(scanner = handler->get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task result is NULL", K(ret));
} else if (OB_FAIL(session->get_trans_result().merge_result(scanner->get_trans_result()))) {
has_merge_err = true;
LOG_WARN("fail to merge trans result",
K(ret),
"session_trans_result",
session->get_trans_result(),
"scanner_trans_result",
scanner->get_trans_result());
} else {
LOG_DEBUG("execute trans_result",
"session_trans_result",
session->get_trans_result(),
"scanner_trans_result",
scanner->get_trans_result());
}
if (OB_SUCCESS != saved_ret) {
ret = saved_ret;
}
NG_TRACE_EXT(remote_task_completed, OB_ID(ret), ret, OB_ID(runner_svr), task.get_runner_svr(), OB_ID(task), task);
}
if (OB_FAIL(ret)) {
if (has_sent_task && (has_transfer_err || has_merge_err)) {
LOG_WARN("need set_incomplete", K(has_transfer_err), K(has_merge_err));
session->get_trans_result().set_incomplete();
}
}
return ret;
}
int ObRemoteScheduler::async_execute_with_sql(ObExecContext& ctx, ObPhysicalPlan* phy_plan)
{
int ret = OB_SUCCESS;
ObSQLSessionInfo* session = ctx.get_my_session();
ObPhysicalPlanCtx* plan_ctx = ctx.get_physical_plan_ctx();
ObExecutorRpcImpl* rpc = NULL;
ObQueryRetryInfo* retry_info = NULL;
ObRemoteTask task;
bool has_sent_task = false;
bool has_transfer_err = false;
bool has_merge_err = false;
if (OB_FAIL(build_remote_task(ctx, task, phy_plan->get_dependency_table()))) {
LOG_WARN("build remote task failed", K(ret), K(task));
} else if (OB_FAIL(ObTaskExecutorCtxUtil::get_task_executor_rpc(ctx, rpc))) {
LOG_WARN("fail get executor rpc", K(ret));
} else {
// ObExecutorRpcCtx rpc_ctx(session->get_rpc_tenant_id(),
// plan_ctx->get_timeout_timestamp(),
// ctx.get_task_exec_ctx().get_min_cluster_version(),
// retry_info,
// ctx.get_my_session(),
// plan_ctx->is_plain_select_stmt());
// if (OB_FAIL(rpc->remote_task_submit(rpc_ctx, task, task.get_runner_svr(), has_sent_task))) {
// LOG_WARN("remote task submit failed", K(ret));
// }
if (OB_FAIL(rpc->remote_task_batch_submit(session->get_rpc_tenant_id(),
task.get_runner_svr(),
session->get_local_ob_org_cluster_id(),
task,
has_sent_task))) {
LOG_WARN("remote task batch submit failed", K(ret), K(task));
}
if (OB_FAIL(ret)) {
if (has_sent_task && (has_transfer_err || has_merge_err)) {
LOG_WARN("need set_incomplete", K(has_transfer_err), K(has_merge_err));
session->get_trans_result().set_incomplete();
}
}
LOG_DEBUG("async execute with sql", K(ret), K(task));
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,43 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_EXECUTOR_REMOTE_SCHEDULER_
#define OCEANBASE_SQL_EXECUTOR_REMOTE_SCHEDULER_
#include "share/ob_define.h"
#include "sql/plan_cache/ob_cache_object.h"
namespace oceanbase {
namespace sql {
class ObPhysicalPlan;
class ObExecContext;
class ObRemoteTask;
class ObRemoteScheduler {
public:
ObRemoteScheduler();
virtual ~ObRemoteScheduler();
int schedule(ObExecContext& ctx, ObPhysicalPlan* phy_plan);
int async_execute_with_sql(ObExecContext& ctx, ObPhysicalPlan*);
private:
int execute_with_plan(ObExecContext& ctx, ObPhysicalPlan* phy_plan);
int execute_with_sql(ObExecContext& ctx, ObPhysicalPlan* phy_plan);
int build_remote_task(ObExecContext& ctx, ObRemoteTask& remote_task, const DependenyTableStore& dependency_tables);
// variable
// functions
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObRemoteScheduler);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SQL_EXECUTOR_REMOTE_SCHEDULER_ */
//// end of header file

View File

@ -0,0 +1,188 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_EXE
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/ob_physical_plan_ctx.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/executor/ob_executor_rpc_impl.h"
#include "sql/executor/ob_remote_task_executor.h"
#include "sql/executor/ob_task.h"
#include "sql/executor/ob_task_executor_ctx.h"
#include "sql/engine/ob_phy_operator.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
ObRemoteTaskExecutor::ObRemoteTaskExecutor()
{}
ObRemoteTaskExecutor::~ObRemoteTaskExecutor()
{}
int ObRemoteTaskExecutor::execute(ObExecContext& query_ctx, ObJob* job, ObTaskInfo* task_info)
{
int ret = OB_SUCCESS;
RemoteExecuteStreamHandle* handler = NULL;
ObSQLSessionInfo* session = query_ctx.get_my_session();
ObPhysicalPlanCtx* plan_ctx = query_ctx.get_physical_plan_ctx();
ObExecutorRpcImpl* rpc = NULL;
ObQueryRetryInfo* retry_info = NULL;
ObTask task;
bool has_sent_task = false;
bool has_transfer_err = false;
bool has_merge_err = false;
if (OB_ISNULL(task_info)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("task info is NULL", K(ret));
} else {
if (OB_FAIL(ObTaskExecutorCtxUtil::get_stream_handler(query_ctx, handler))) {
LOG_WARN("fail get task response handler", K(ret));
} else if (OB_FAIL(ObTaskExecutorCtxUtil::get_task_executor_rpc(query_ctx, rpc))) {
LOG_WARN("fail get executor rpc", K(ret));
} else if (OB_ISNULL(session) || OB_ISNULL(plan_ctx) || OB_ISNULL(handler) || OB_ISNULL(rpc) ||
OB_ISNULL(retry_info = &session->get_retry_info_for_update())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("unexpected null ptr", K(ret), K(session), K(plan_ctx), K(handler), K(rpc), K(retry_info));
} else if (OB_FAIL(build_task(query_ctx, *job, *task_info, task))) {
LOG_WARN("fail build task", K(ret), K(job), K(task_info));
} else if (OB_FAIL(handler->reset_and_init_result())) {
LOG_WARN("fail to reset and init result", K(ret));
} else {
ObScanner* scanner = NULL;
task_info->set_state(OB_TASK_STATE_RUNNING);
ObExecutorRpcCtx rpc_ctx(session->get_rpc_tenant_id(),
plan_ctx->get_timeout_timestamp(),
query_ctx.get_task_exec_ctx().get_min_cluster_version(),
retry_info,
query_ctx.get_my_session(),
plan_ctx->is_plain_select_stmt());
if (OB_FAIL(rpc->task_execute(
rpc_ctx, task, task_info->get_task_location().get_server(), *handler, has_sent_task, has_transfer_err))) {
bool skip_failed_tasks = false;
int check_ret = OB_SUCCESS;
int add_ret = OB_SUCCESS;
if (is_data_not_readable_err(ret)) {
// add server to retry info
if (OB_UNLIKELY(OB_SUCCESS != (add_ret = retry_info->add_invalid_server_distinctly(
task_info->get_task_location().get_server(), true)))) {
LOG_WARN("fail to add remote addr to invalid servers distinctly",
K(ret),
K(add_ret),
K(task_info->get_task_location().get_server()),
K(*retry_info));
}
}
if (OB_SUCCESS != (check_ret = should_skip_failed_tasks(*task_info, skip_failed_tasks))) {
// check fail, set ret to check_ret
LOG_WARN("fail to check if it should skip failed tasks", K(ret), K(check_ret), K(*job));
ret = check_ret;
} else if (true == skip_failed_tasks) {
// should skip failed tasks, log user warning and skip it, and set handler's error code to
// OB_ERR_TASK_SKIPPED, than return OB_SUCCESS
task_info->set_state(OB_TASK_STATE_SKIPPED);
LOG_WARN("fail to do task on the remote server, log user warning and skip it",
K(ret),
K(task_info->get_task_location().get_server()),
K(*job));
LOG_USER_WARN(OB_ERR_TASK_SKIPPED,
to_cstring(task_info->get_task_location().get_server()),
common::ob_errpkt_errno(ret, lib::is_oracle_mode()));
handler->set_result_code(OB_ERR_TASK_SKIPPED);
ret = OB_SUCCESS;
} else {
// let user see ret
LOG_WARN("fail post task", K(ret));
}
}
int saved_ret = ret;
if (OB_ISNULL(scanner = handler->get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task result is NULL", K(ret));
} else if (OB_FAIL(session->get_trans_result().merge_result(scanner->get_trans_result()))) {
has_merge_err = true;
LOG_WARN("fail to merge trans result",
K(ret),
"session_trans_result",
session->get_trans_result(),
"scanner_trans_result",
scanner->get_trans_result());
} else {
LOG_DEBUG("execute trans_result",
"session_trans_result",
session->get_trans_result(),
"scanner_trans_result",
scanner->get_trans_result());
}
if (OB_SUCCESS != saved_ret) {
ret = saved_ret;
}
NG_TRACE_EXT(remote_task_completed,
OB_ID(ret),
ret,
OB_ID(runner_svr),
task_info->get_task_location().get_server(),
OB_ID(task),
task);
}
if (OB_FAIL(ret)) {
task_info->set_state(OB_TASK_STATE_FAILED);
// set incomplete flag on rpc error
if (has_sent_task && (has_transfer_err || has_merge_err)) {
LOG_WARN("need set_incomplete", K(has_transfer_err), K(has_merge_err));
session->get_trans_result().set_incomplete();
}
}
}
return ret;
}
int ObRemoteTaskExecutor::build_task(ObExecContext& query_ctx, ObJob& job, ObTaskInfo& task_info, ObTask& task)
{
int ret = OB_SUCCESS;
/* serialize:
* 1. ObPhysicalPlanCtx
* 2. ObPhyOperator Tree
* 3. ObPhyOperator Tree Input
*/
ObPhyOperator* root_op = NULL;
const ObPhysicalPlan* phy_plan = NULL;
if (OB_UNLIKELY(NULL == (root_op = job.get_root_op()))) {
ret = OB_NOT_INIT;
LOG_WARN("root op not set", K(ret));
} else if (OB_UNLIKELY(NULL == (phy_plan = root_op->get_phy_plan()))) {
ret = OB_NOT_INIT;
LOG_WARN("physical plan is NULL", K(ret));
} else if (OB_FAIL(build_task_op_input(query_ctx, task_info, *root_op))) {
LOG_WARN("fail build op inputs", K(ret));
} else {
const ObTaskInfo::ObRangeLocation& range_loc = task_info.get_range_location();
for (int64_t i = 0; OB_SUCC(ret) && i < range_loc.part_locs_.count(); ++i) {
if (OB_FAIL(task.add_partition_key(range_loc.part_locs_.at(i).partition_key_))) {
LOG_WARN("fail to add partition key into ObTask", K(ret), K(i), K(range_loc.part_locs_.at(i).partition_key_));
} else if (OB_FAIL(task.assign_ranges(range_loc.part_locs_.at(i).scan_ranges_))) {
LOG_WARN("assign range failed", K(ret));
}
}
if (OB_SUCC(ret)) {
task.set_ctrl_server(job.get_ob_job_id().get_server());
task.set_runner_server(task_info.get_task_location().get_server());
task.set_ob_task_id(task_info.get_task_location().get_ob_task_id());
task.set_serialize_param(query_ctx, *root_op, *phy_plan);
}
}
return ret;
}

Some files were not shown because too many files have changed in this diff Show More