init push
This commit is contained in:
		
							
								
								
									
										40
									
								
								src/sql/executor/ob_addrs_provider.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										40
									
								
								src/sql/executor/ob_addrs_provider.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										78
									
								
								src/sql/executor/ob_addrs_provider_factory.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										78
									
								
								src/sql/executor/ob_addrs_provider_factory.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										38
									
								
								src/sql/executor/ob_addrs_provider_factory.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										38
									
								
								src/sql/executor/ob_addrs_provider_factory.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										448
									
								
								src/sql/executor/ob_bkgd_dist_task.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										448
									
								
								src/sql/executor/ob_bkgd_dist_task.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										159
									
								
								src/sql/executor/ob_bkgd_dist_task.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										159
									
								
								src/sql/executor/ob_bkgd_dist_task.h
									
									
									
									
									
										Normal 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_ | ||||
							
								
								
									
										790
									
								
								src/sql/executor/ob_cmd_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										790
									
								
								src/sql/executor/ob_cmd_executor.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										34
									
								
								src/sql/executor/ob_cmd_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										34
									
								
								src/sql/executor/ob_cmd_executor.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										383
									
								
								src/sql/executor/ob_determinate_task_spliter.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										383
									
								
								src/sql/executor/ob_determinate_task_spliter.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										54
									
								
								src/sql/executor/ob_determinate_task_spliter.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										54
									
								
								src/sql/executor/ob_determinate_task_spliter.h
									
									
									
									
									
										Normal 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_ | ||||
							
								
								
									
										530
									
								
								src/sql/executor/ob_determinate_task_transmit.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										530
									
								
								src/sql/executor/ob_determinate_task_transmit.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										226
									
								
								src/sql/executor/ob_determinate_task_transmit.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										226
									
								
								src/sql/executor/ob_determinate_task_transmit.h
									
									
									
									
									
										Normal 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_ | ||||
							
								
								
									
										405
									
								
								src/sql/executor/ob_direct_receive.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										405
									
								
								src/sql/executor/ob_direct_receive.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										98
									
								
								src/sql/executor/ob_direct_receive.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										98
									
								
								src/sql/executor/ob_direct_receive.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										326
									
								
								src/sql/executor/ob_direct_receive_op.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										326
									
								
								src/sql/executor/ob_direct_receive_op.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										68
									
								
								src/sql/executor/ob_direct_receive_op.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										68
									
								
								src/sql/executor/ob_direct_receive_op.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										104
									
								
								src/sql/executor/ob_direct_transmit.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										104
									
								
								src/sql/executor/ob_direct_transmit.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										84
									
								
								src/sql/executor/ob_direct_transmit.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										84
									
								
								src/sql/executor/ob_direct_transmit.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										26
									
								
								src/sql/executor/ob_direct_transmit_op.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										26
									
								
								src/sql/executor/ob_direct_transmit_op.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										73
									
								
								src/sql/executor/ob_direct_transmit_op.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										73
									
								
								src/sql/executor/ob_direct_transmit_op.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										100
									
								
								src/sql/executor/ob_distributed_job_control.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										100
									
								
								src/sql/executor/ob_distributed_job_control.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										35
									
								
								src/sql/executor/ob_distributed_job_control.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										35
									
								
								src/sql/executor/ob_distributed_job_control.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										191
									
								
								src/sql/executor/ob_distributed_job_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										191
									
								
								src/sql/executor/ob_distributed_job_executor.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										65
									
								
								src/sql/executor/ob_distributed_job_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										65
									
								
								src/sql/executor/ob_distributed_job_executor.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										1372
									
								
								src/sql/executor/ob_distributed_scheduler.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1372
									
								
								src/sql/executor/ob_distributed_scheduler.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										288
									
								
								src/sql/executor/ob_distributed_scheduler.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										288
									
								
								src/sql/executor/ob_distributed_scheduler.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										377
									
								
								src/sql/executor/ob_distributed_task_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										377
									
								
								src/sql/executor/ob_distributed_task_executor.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										57
									
								
								src/sql/executor/ob_distributed_task_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										57
									
								
								src/sql/executor/ob_distributed_task_executor.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										72
									
								
								src/sql/executor/ob_distributed_task_runner.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										72
									
								
								src/sql/executor/ob_distributed_task_runner.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										33
									
								
								src/sql/executor/ob_distributed_task_runner.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										33
									
								
								src/sql/executor/ob_distributed_task_runner.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										592
									
								
								src/sql/executor/ob_distributed_transmit.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										592
									
								
								src/sql/executor/ob_distributed_transmit.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										171
									
								
								src/sql/executor/ob_distributed_transmit.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										171
									
								
								src/sql/executor/ob_distributed_transmit.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										266
									
								
								src/sql/executor/ob_execute_result.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										266
									
								
								src/sql/executor/ob_execute_result.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										127
									
								
								src/sql/executor/ob_execute_result.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										127
									
								
								src/sql/executor/ob_execute_result.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										40
									
								
								src/sql/executor/ob_execution_id.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										40
									
								
								src/sql/executor/ob_execution_id.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										122
									
								
								src/sql/executor/ob_execution_id.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										122
									
								
								src/sql/executor/ob_execution_id.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										327
									
								
								src/sql/executor/ob_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										327
									
								
								src/sql/executor/ob_executor.cpp
									
									
									
									
									
										Normal 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; | ||||
| } | ||||
							
								
								
									
										52
									
								
								src/sql/executor/ob_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										52
									
								
								src/sql/executor/ob_executor.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										661
									
								
								src/sql/executor/ob_executor_rpc_impl.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										661
									
								
								src/sql/executor/ob_executor_rpc_impl.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										506
									
								
								src/sql/executor/ob_executor_rpc_impl.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										506
									
								
								src/sql/executor/ob_executor_rpc_impl.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										1752
									
								
								src/sql/executor/ob_executor_rpc_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1752
									
								
								src/sql/executor/ob_executor_rpc_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										549
									
								
								src/sql/executor/ob_executor_rpc_processor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										549
									
								
								src/sql/executor/ob_executor_rpc_processor.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										29
									
								
								src/sql/executor/ob_executor_rpc_proxy.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										29
									
								
								src/sql/executor/ob_executor_rpc_proxy.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										127
									
								
								src/sql/executor/ob_executor_rpc_proxy.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										127
									
								
								src/sql/executor/ob_executor_rpc_proxy.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										2390
									
								
								src/sql/executor/ob_fifo_receive.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										2390
									
								
								src/sql/executor/ob_fifo_receive.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										510
									
								
								src/sql/executor/ob_fifo_receive.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										510
									
								
								src/sql/executor/ob_fifo_receive.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										738
									
								
								src/sql/executor/ob_interm_result.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										738
									
								
								src/sql/executor/ob_interm_result.cpp
									
									
									
									
									
										Normal 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 */ | ||||
							
								
								
									
										240
									
								
								src/sql/executor/ob_interm_result.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										240
									
								
								src/sql/executor/ob_interm_result.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										262
									
								
								src/sql/executor/ob_interm_result_item.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										262
									
								
								src/sql/executor/ob_interm_result_item.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										132
									
								
								src/sql/executor/ob_interm_result_item.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										132
									
								
								src/sql/executor/ob_interm_result_item.h
									
									
									
									
									
										Normal 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_ | ||||
							
								
								
									
										161
									
								
								src/sql/executor/ob_interm_result_item_pool.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										161
									
								
								src/sql/executor/ob_interm_result_item_pool.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										67
									
								
								src/sql/executor/ob_interm_result_item_pool.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										67
									
								
								src/sql/executor/ob_interm_result_item_pool.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										437
									
								
								src/sql/executor/ob_interm_result_manager.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										437
									
								
								src/sql/executor/ob_interm_result_manager.cpp
									
									
									
									
									
										Normal 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 */ | ||||
							
								
								
									
										166
									
								
								src/sql/executor/ob_interm_result_manager.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										166
									
								
								src/sql/executor/ob_interm_result_manager.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										166
									
								
								src/sql/executor/ob_interm_result_pool.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										166
									
								
								src/sql/executor/ob_interm_result_pool.cpp
									
									
									
									
									
										Normal 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 */ | ||||
							
								
								
									
										63
									
								
								src/sql/executor/ob_interm_result_pool.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										63
									
								
								src/sql/executor/ob_interm_result_pool.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										102
									
								
								src/sql/executor/ob_interm_task_spliter.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										102
									
								
								src/sql/executor/ob_interm_task_spliter.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										48
									
								
								src/sql/executor/ob_interm_task_spliter.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										48
									
								
								src/sql/executor/ob_interm_task_spliter.h
									
									
									
									
									
										Normal 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
									
								
							
							
						
						
									
										642
									
								
								src/sql/executor/ob_job.cpp
									
									
									
									
									
										Normal 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
									
								
							
							
						
						
									
										289
									
								
								src/sql/executor/ob_job.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										35
									
								
								src/sql/executor/ob_job_conf.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										35
									
								
								src/sql/executor/ob_job_conf.cpp
									
									
									
									
									
										Normal 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; | ||||
| } | ||||
							
								
								
									
										81
									
								
								src/sql/executor/ob_job_conf.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										81
									
								
								src/sql/executor/ob_job_conf.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										464
									
								
								src/sql/executor/ob_job_control.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										464
									
								
								src/sql/executor/ob_job_control.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										81
									
								
								src/sql/executor/ob_job_control.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										81
									
								
								src/sql/executor/ob_job_control.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										25
									
								
								src/sql/executor/ob_job_id.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										25
									
								
								src/sql/executor/ob_job_id.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										125
									
								
								src/sql/executor/ob_job_id.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										125
									
								
								src/sql/executor/ob_job_id.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										206
									
								
								src/sql/executor/ob_job_parser.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										206
									
								
								src/sql/executor/ob_job_parser.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										48
									
								
								src/sql/executor/ob_job_parser.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										48
									
								
								src/sql/executor/ob_job_parser.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										68
									
								
								src/sql/executor/ob_local_identity_task_spliter.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										68
									
								
								src/sql/executor/ob_local_identity_task_spliter.cpp
									
									
									
									
									
										Normal 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; | ||||
| } | ||||
							
								
								
									
										47
									
								
								src/sql/executor/ob_local_identity_task_spliter.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										47
									
								
								src/sql/executor/ob_local_identity_task_spliter.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										48
									
								
								src/sql/executor/ob_local_job_control.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										48
									
								
								src/sql/executor/ob_local_job_control.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										32
									
								
								src/sql/executor/ob_local_job_control.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										32
									
								
								src/sql/executor/ob_local_job_control.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										80
									
								
								src/sql/executor/ob_local_job_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										80
									
								
								src/sql/executor/ob_local_job_executor.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										54
									
								
								src/sql/executor/ob_local_job_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										54
									
								
								src/sql/executor/ob_local_job_executor.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										153
									
								
								src/sql/executor/ob_local_scheduler.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										153
									
								
								src/sql/executor/ob_local_scheduler.cpp
									
									
									
									
									
										Normal 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; | ||||
| } | ||||
							
								
								
									
										42
									
								
								src/sql/executor/ob_local_scheduler.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										42
									
								
								src/sql/executor/ob_local_scheduler.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										65
									
								
								src/sql/executor/ob_local_task_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										65
									
								
								src/sql/executor/ob_local_task_executor.cpp
									
									
									
									
									
										Normal 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; | ||||
| } | ||||
							
								
								
									
										35
									
								
								src/sql/executor/ob_local_task_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										35
									
								
								src/sql/executor/ob_local_task_executor.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										1026
									
								
								src/sql/executor/ob_mini_task_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1026
									
								
								src/sql/executor/ob_mini_task_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										199
									
								
								src/sql/executor/ob_mini_task_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										199
									
								
								src/sql/executor/ob_mini_task_executor.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										181
									
								
								src/sql/executor/ob_multiinsert_task_spliter.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										181
									
								
								src/sql/executor/ob_multiinsert_task_spliter.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										52
									
								
								src/sql/executor/ob_multiinsert_task_spliter.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										52
									
								
								src/sql/executor/ob_multiinsert_task_spliter.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										1009
									
								
								src/sql/executor/ob_multiscan_task_spliter.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1009
									
								
								src/sql/executor/ob_multiscan_task_spliter.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										205
									
								
								src/sql/executor/ob_multiscan_task_spliter.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										205
									
								
								src/sql/executor/ob_multiscan_task_spliter.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										55
									
								
								src/sql/executor/ob_random_addrs_provider.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										55
									
								
								src/sql/executor/ob_random_addrs_provider.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										35
									
								
								src/sql/executor/ob_random_addrs_provider.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										35
									
								
								src/sql/executor/ob_random_addrs_provider.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										80
									
								
								src/sql/executor/ob_range_hash_key_getter.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										80
									
								
								src/sql/executor/ob_range_hash_key_getter.cpp
									
									
									
									
									
										Normal 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; | ||||
| }*/ | ||||
							
								
								
									
										55
									
								
								src/sql/executor/ob_range_hash_key_getter.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										55
									
								
								src/sql/executor/ob_range_hash_key_getter.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										128
									
								
								src/sql/executor/ob_receive.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										128
									
								
								src/sql/executor/ob_receive.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										142
									
								
								src/sql/executor/ob_receive.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										142
									
								
								src/sql/executor/ob_receive.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										1472
									
								
								src/sql/executor/ob_remote_executor_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1472
									
								
								src/sql/executor/ob_remote_executor_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										237
									
								
								src/sql/executor/ob_remote_executor_processor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										237
									
								
								src/sql/executor/ob_remote_executor_processor.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										116
									
								
								src/sql/executor/ob_remote_identity_task_spliter.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										116
									
								
								src/sql/executor/ob_remote_identity_task_spliter.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										41
									
								
								src/sql/executor/ob_remote_identity_task_spliter.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										41
									
								
								src/sql/executor/ob_remote_identity_task_spliter.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										49
									
								
								src/sql/executor/ob_remote_job_control.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										49
									
								
								src/sql/executor/ob_remote_job_control.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										32
									
								
								src/sql/executor/ob_remote_job_control.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										32
									
								
								src/sql/executor/ob_remote_job_control.h
									
									
									
									
									
										Normal 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_ */ | ||||
							
								
								
									
										79
									
								
								src/sql/executor/ob_remote_job_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										79
									
								
								src/sql/executor/ob_remote_job_executor.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										56
									
								
								src/sql/executor/ob_remote_job_executor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										56
									
								
								src/sql/executor/ob_remote_job_executor.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										331
									
								
								src/sql/executor/ob_remote_scheduler.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										331
									
								
								src/sql/executor/ob_remote_scheduler.cpp
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										43
									
								
								src/sql/executor/ob_remote_scheduler.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										43
									
								
								src/sql/executor/ob_remote_scheduler.h
									
									
									
									
									
										Normal 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 | ||||
							
								
								
									
										188
									
								
								src/sql/executor/ob_remote_task_executor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										188
									
								
								src/sql/executor/ob_remote_task_executor.cpp
									
									
									
									
									
										Normal 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
		Reference in New Issue
	
	Block a user
	 oceanbase-admin
					oceanbase-admin