add tableapi to opensource release
This commit is contained in:
		| @ -578,7 +578,6 @@ PCODE_DEF(OB_TABLE_API_LOGIN, 0x1101) | |||||||
| PCODE_DEF(OB_TABLE_API_EXECUTE, 0x1102) | PCODE_DEF(OB_TABLE_API_EXECUTE, 0x1102) | ||||||
| PCODE_DEF(OB_TABLE_API_BATCH_EXECUTE, 0x1103) | PCODE_DEF(OB_TABLE_API_BATCH_EXECUTE, 0x1103) | ||||||
| PCODE_DEF(OB_TABLE_API_EXECUTE_QUERY, 0x1104) | PCODE_DEF(OB_TABLE_API_EXECUTE_QUERY, 0x1104) | ||||||
| PCODE_DEF(OB_TABLE_API_QUERY_AND_MUTATE, 0x1105) |  | ||||||
|  |  | ||||||
| // Event Job API | // Event Job API | ||||||
| PCODE_DEF(OB_RUN_EVENT_JOB, 0x1201) | PCODE_DEF(OB_RUN_EVENT_JOB, 0x1201) | ||||||
|  | |||||||
| @ -240,6 +240,16 @@ ob_set_subtarget(ob_server vt | |||||||
|   virtual_table/ob_all_virtual_backupset_history_mgr.cpp |   virtual_table/ob_all_virtual_backupset_history_mgr.cpp | ||||||
| ) | ) | ||||||
|  |  | ||||||
|  | ob_set_subtarget(ob_server table | ||||||
|  |   table/ob_table_batch_execute_processor.cpp		 | ||||||
|  |   table/ob_table_end_trans_cb.cpp		 | ||||||
|  |   table/ob_table_execute_processor.cpp		 | ||||||
|  |   table/ob_table_query_processor.cpp		 | ||||||
|  |   table/ob_table_rpc_processor.cpp		 | ||||||
|  |   table/ob_table_service.cpp		 | ||||||
|  |   table/ob_table_api_row_iterator.cpp | ||||||
|  | ) | ||||||
|  |  | ||||||
| ob_server_add_pchs(observer | ob_server_add_pchs(observer | ||||||
|   ob_server_struct.h |   ob_server_struct.h | ||||||
|   ob_uniq_task_queue.h |   ob_uniq_task_queue.h | ||||||
|  | |||||||
| @ -45,6 +45,7 @@ | |||||||
| #include "sql/ob_sql_init.h" | #include "sql/ob_sql_init.h" | ||||||
| #include "sql/ob_sql_task.h" | #include "sql/ob_sql_task.h" | ||||||
| #include "observer/ob_server.h" | #include "observer/ob_server.h" | ||||||
|  | #include "observer/table/ob_table_rpc_processor.h" | ||||||
| #include "sql/ob_sql_init.h" | #include "sql/ob_sql_init.h" | ||||||
| #include "sql/dtl/ob_dtl.h" | #include "sql/dtl/ob_dtl.h" | ||||||
| #include "sql/ob_sql_init.h" | #include "sql/ob_sql_init.h" | ||||||
| @ -129,6 +130,7 @@ ObServer::ObServer() | |||||||
|       vt_data_service_(root_service_, self_addr_, &config_), |       vt_data_service_(root_service_, self_addr_, &config_), | ||||||
|       cache_size_calculator_(), |       cache_size_calculator_(), | ||||||
|       weak_read_service_(), |       weak_read_service_(), | ||||||
|  |       table_service_(), | ||||||
|       cgroup_ctrl_(), |       cgroup_ctrl_(), | ||||||
|       start_time_(ObTimeUtility::current_time()), |       start_time_(ObTimeUtility::current_time()), | ||||||
|       zone_merged_version_(OB_MERGED_VERSION_INIT), |       zone_merged_version_(OB_MERGED_VERSION_INIT), | ||||||
| @ -198,8 +200,10 @@ int ObServer::init(const ObServerOptions& opts, const ObPLogWriterCfg& log_cfg) | |||||||
|     } |     } | ||||||
|  |  | ||||||
|     if (OB_SUCC(ret)) { |     if (OB_SUCC(ret)) { | ||||||
|       if (OB_FAIL(init_loaddata_global_stat())) { |       if (OB_FAIL(ObTableApiProcessorBase::init_session())) { | ||||||
|         LOG_WARN("fail to init global load data stat map", K(ret)); |         LOG_WARN("failed to init static session", K(ret)); | ||||||
|  |       } else if (OB_FAIL(init_loaddata_global_stat())) { | ||||||
|  |          LOG_WARN("fail to init global load data stat map", K(ret)); | ||||||
|       } |       } | ||||||
|     } |     } | ||||||
|   } |   } | ||||||
| @ -326,6 +330,8 @@ int ObServer::init(const ObServerOptions& opts, const ObPLogWriterCfg& log_cfg) | |||||||
|     LOG_WARN("fail to init long ops monitor instance", K(ret)); |     LOG_WARN("fail to init long ops monitor instance", K(ret)); | ||||||
|   } else if (OB_FAIL(ObCompatModeGetter::instance().init(&sql_proxy_))) { |   } else if (OB_FAIL(ObCompatModeGetter::instance().init(&sql_proxy_))) { | ||||||
|     LOG_WARN("fail to init get compat mode server"); |     LOG_WARN("fail to init get compat mode server"); | ||||||
|  |   } else if (OB_FAIL(table_service_.init(gctx_))) { | ||||||
|  |     LOG_WARN("failed to init table service", K(ret)); | ||||||
|   } else if (OB_FAIL(ObTimerMonitor::get_instance().init())) { |   } else if (OB_FAIL(ObTimerMonitor::get_instance().init())) { | ||||||
|     LOG_WARN("failed to init timer monitor", K(ret)); |     LOG_WARN("failed to init timer monitor", K(ret)); | ||||||
|   } else if (OB_FAIL(ObBGThreadMonitor::get_instance().init())) { |   } else if (OB_FAIL(ObBGThreadMonitor::get_instance().init())) { | ||||||
| @ -1468,6 +1474,7 @@ int ObServer::init_global_context() | |||||||
|   (void)gctx_.set_split_schema_version(OB_INVALID_VERSION); |   (void)gctx_.set_split_schema_version(OB_INVALID_VERSION); | ||||||
|   (void)gctx_.set_split_schema_version_v2(OB_INVALID_VERSION); |   (void)gctx_.set_split_schema_version_v2(OB_INVALID_VERSION); | ||||||
|   gctx_.weak_read_service_ = &weak_read_service_; |   gctx_.weak_read_service_ = &weak_read_service_; | ||||||
|  |   gctx_.table_service_ = &table_service_; | ||||||
|   gctx_.cgroup_ctrl_ = &cgroup_ctrl_; |   gctx_.cgroup_ctrl_ = &cgroup_ctrl_; | ||||||
|   gctx_.schema_status_proxy_ = &schema_status_proxy_; |   gctx_.schema_status_proxy_ = &schema_status_proxy_; | ||||||
|   (void)gctx_.set_upgrade_stage(obrpc::OB_UPGRADE_STAGE_INVALID); |   (void)gctx_.set_upgrade_stage(obrpc::OB_UPGRADE_STAGE_INVALID); | ||||||
|  | |||||||
| @ -19,6 +19,7 @@ | |||||||
| #include "share/stat/ob_user_tab_col_statistics.h" | #include "share/stat/ob_user_tab_col_statistics.h" | ||||||
| #include "share/stat/ob_opt_stat_service.h" | #include "share/stat/ob_opt_stat_service.h" | ||||||
|  |  | ||||||
|  | #include "observer/table/ob_table_service.h" | ||||||
| #include "sql/ob_sql.h" | #include "sql/ob_sql.h" | ||||||
| #include "sql/engine/cmd/ob_load_data_rpc.h" | #include "sql/engine/cmd/ob_load_data_rpc.h" | ||||||
| #include "sql/ob_query_exec_ctx_mgr.h" | #include "sql/ob_query_exec_ctx_mgr.h" | ||||||
| @ -388,6 +389,9 @@ private: | |||||||
|   // Weakly Consistent Read Service |   // Weakly Consistent Read Service | ||||||
|   transaction::ObWeakReadService weak_read_service_; |   transaction::ObWeakReadService weak_read_service_; | ||||||
|  |  | ||||||
|  |   // table service              | ||||||
|  |   ObTableService table_service_; | ||||||
|  |  | ||||||
|   // Tenant isolation resource management |   // Tenant isolation resource management | ||||||
|   omt::ObCgroupCtrl cgroup_ctrl_; |   omt::ObCgroupCtrl cgroup_ctrl_; | ||||||
|  |  | ||||||
|  | |||||||
| @ -266,6 +266,7 @@ ObGlobalContext& ObGlobalContext::operator=(const ObGlobalContext& other) | |||||||
|     sort_dir_ = other.sort_dir_; |     sort_dir_ = other.sort_dir_; | ||||||
|     diag_ = other.diag_; |     diag_ = other.diag_; | ||||||
|     scramble_rand_ = other.scramble_rand_; |     scramble_rand_ = other.scramble_rand_; | ||||||
|  |     table_service_ = other.table_service_; | ||||||
|     cgroup_ctrl_ = other.cgroup_ctrl_; |     cgroup_ctrl_ = other.cgroup_ctrl_; | ||||||
|     inited_ = other.inited_; |     inited_ = other.inited_; | ||||||
|     split_schema_version_ = other.split_schema_version_; |     split_schema_version_ = other.split_schema_version_; | ||||||
|  | |||||||
| @ -84,6 +84,7 @@ class ObCgroupCtrl; | |||||||
| namespace observer { | namespace observer { | ||||||
| class ObService; | class ObService; | ||||||
| class ObVTIterCreator; | class ObVTIterCreator; | ||||||
|  | class ObTableService; | ||||||
|  |  | ||||||
| class ObServerOptions { | class ObServerOptions { | ||||||
| public: | public: | ||||||
| @ -197,6 +198,7 @@ struct ObGlobalContext { | |||||||
|   common::ObString* sort_dir_; |   common::ObString* sort_dir_; | ||||||
|   obmysql::ObDiag* diag_; |   obmysql::ObDiag* diag_; | ||||||
|   common::ObMysqlRandom* scramble_rand_; |   common::ObMysqlRandom* scramble_rand_; | ||||||
|  |   ObTableService* table_service_; | ||||||
|   omt::ObCgroupCtrl* cgroup_ctrl_; |   omt::ObCgroupCtrl* cgroup_ctrl_; | ||||||
|   bool inited_; |   bool inited_; | ||||||
|   int64_t split_schema_version_; |   int64_t split_schema_version_; | ||||||
|  | |||||||
| @ -37,6 +37,11 @@ | |||||||
| #include "observer/ob_rpc_processor_simple.h" | #include "observer/ob_rpc_processor_simple.h" | ||||||
| #include "observer/ob_srv_task.h" | #include "observer/ob_srv_task.h" | ||||||
|  |  | ||||||
|  | #include "observer/table/ob_table_rpc_processor.h"              | ||||||
|  | #include "observer/table/ob_table_execute_processor.h"          | ||||||
|  | #include "observer/table/ob_table_batch_execute_processor.h"            | ||||||
|  | #include "observer/table/ob_table_query_processor.h" | ||||||
|  |  | ||||||
| using namespace oceanbase; | using namespace oceanbase; | ||||||
| using namespace oceanbase::observer; | using namespace oceanbase::observer; | ||||||
| using namespace oceanbase::lib; | using namespace oceanbase::lib; | ||||||
| @ -125,6 +130,12 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator* xlator) | |||||||
|   // SQL Estimate |   // SQL Estimate | ||||||
|   RPC_PROCESSOR(ObEstimatePartitionRowsP, gctx_); |   RPC_PROCESSOR(ObEstimatePartitionRowsP, gctx_); | ||||||
|  |  | ||||||
|  |   // table api          | ||||||
|  |   RPC_PROCESSOR(ObTableLoginP, gctx_);          | ||||||
|  |   RPC_PROCESSOR(ObTableApiExecuteP, gctx_);             | ||||||
|  |   RPC_PROCESSOR(ObTableBatchExecuteP, gctx_);           | ||||||
|  |   RPC_PROCESSOR(ObTableQueryP, gctx_);  | ||||||
|  |  | ||||||
|   // HA GTS |   // HA GTS | ||||||
|   RPC_PROCESSOR(ObHaGtsPingRequestP, gctx_); |   RPC_PROCESSOR(ObHaGtsPingRequestP, gctx_); | ||||||
|   RPC_PROCESSOR(ObHaGtsGetRequestP, gctx_); |   RPC_PROCESSOR(ObHaGtsGetRequestP, gctx_); | ||||||
|  | |||||||
							
								
								
									
										236
									
								
								src/observer/table/ob_rpc_async_response.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										236
									
								
								src/observer/table/ob_rpc_async_response.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,236 @@ | |||||||
|  | /** | ||||||
|  |  * 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_RPC_ASYNC_RESPONSE_H | ||||||
|  | #define _OB_RPC_ASYNC_RESPONSE_H 1 | ||||||
|  | #include "rpc/ob_request.h" | ||||||
|  | #include "rpc/obrpc/ob_rpc_packet.h" | ||||||
|  | #include "rpc/frame/ob_req_processor.h" | ||||||
|  | #include "rpc/obmysql/ob_mysql_request_utils.h" | ||||||
|  | #include "rpc/obrpc/ob_rpc_result_code.h" | ||||||
|  | #include "lib/oblog/ob_warning_buffer.h" | ||||||
|  | #include "ob_table_rpc_processor_util.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace obrpc | ||||||
|  | { | ||||||
|  | // this class is copied from ObRpcProcessor | ||||||
|  | template <class T> | ||||||
|  | class ObRpcAsyncResponse | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObRpcAsyncResponse(rpc::ObRequest *req, T &result) | ||||||
|  |       :req_(req), | ||||||
|  |        result_(result), | ||||||
|  |        using_buffer_(NULL) | ||||||
|  |   {} | ||||||
|  |   virtual ~ObRpcAsyncResponse() = default; | ||||||
|  |   int response(const int retcode); | ||||||
|  | private: | ||||||
|  |   int serialize(); | ||||||
|  |   int do_response(ObRpcPacket *response_pkt, bool bad_routing); | ||||||
|  |   char *easy_alloc(int64_t size) const; | ||||||
|  |   // disallow copy | ||||||
|  |   DISALLOW_COPY_AND_ASSIGN(ObRpcAsyncResponse); | ||||||
|  | private: | ||||||
|  |   rpc::ObRequest *req_; | ||||||
|  |   T &result_; | ||||||
|  |   common::ObDataBuffer *using_buffer_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | template <class T> | ||||||
|  | char *ObRpcAsyncResponse<T>::easy_alloc(int64_t size) const | ||||||
|  | { | ||||||
|  |   void *buf = NULL; | ||||||
|  |   if (OB_ISNULL(req_)) { | ||||||
|  |     RPC_OBRPC_LOG(ERROR, "request is invalid", KP(req_)); | ||||||
|  |   } else if (OB_ISNULL(req_->get_request()) | ||||||
|  |              || OB_ISNULL(req_->get_request()->ms) | ||||||
|  |              || OB_ISNULL(req_->get_request()->ms->pool)) { | ||||||
|  |     RPC_OBRPC_LOG(ERROR, "request is invalid", K(req_)); | ||||||
|  |   } else { | ||||||
|  |     buf = easy_pool_alloc( | ||||||
|  |         req_->get_request()->ms->pool, static_cast<uint32_t>(size)); | ||||||
|  |   } | ||||||
|  |   return static_cast<char*>(buf); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | template <class T> | ||||||
|  | int ObRpcAsyncResponse<T>::serialize() | ||||||
|  | { | ||||||
|  |   int ret = common::OB_SUCCESS; | ||||||
|  |   if (OB_ISNULL(using_buffer_)) { | ||||||
|  |     ret = common::OB_ERR_UNEXPECTED; | ||||||
|  |     RPC_OBRPC_LOG(ERROR, "using_buffer_ should not be NULL", K(ret)); | ||||||
|  |   } else if (OB_FAIL(common::serialization::encode( | ||||||
|  |         using_buffer_->get_data(), using_buffer_->get_capacity(), | ||||||
|  |         using_buffer_->get_position(), result_))) { | ||||||
|  |     RPC_OBRPC_LOG(WARN, "encode data error", K(ret)); | ||||||
|  |   } else { | ||||||
|  |     //do nothing | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | template <class T> | ||||||
|  | int ObRpcAsyncResponse<T>::do_response(ObRpcPacket *response_pkt, bool bad_routing) | ||||||
|  | { | ||||||
|  |   int ret = common::OB_SUCCESS; | ||||||
|  |   if (OB_ISNULL(req_)) { | ||||||
|  |     ret = common::OB_ERR_NULL_VALUE; | ||||||
|  |     RPC_OBRPC_LOG(WARN, "req is NULL", K(ret)); | ||||||
|  |   } else if (OB_ISNULL(req_->get_request())) { | ||||||
|  |     ret = common::OB_ERR_NULL_VALUE; | ||||||
|  |     RPC_OBRPC_LOG(WARN, "req is NULL", K(ret)); | ||||||
|  |   } else { | ||||||
|  |     const ObRpcPacket *rpc_pkt = &reinterpret_cast<const ObRpcPacket&>(req_->get_packet()); | ||||||
|  |     // TODO: fufeng, make force_destroy_second as a configure item | ||||||
|  |     // static const int64_t RESPONSE_RESERVED_US = 20 * 1000 * 1000; | ||||||
|  |     // int64_t rts = static_cast<int64_t>(req_->get_request()->start_time) * 1000 * 1000; | ||||||
|  |     // todo(fufeng): get 'force destroy second' from eio? | ||||||
|  |     // if (rts > 0 && eio_->force_destroy_second > 0 | ||||||
|  |     //     && ::oceanbase::common::ObTimeUtility::current_time() - rts + RESPONSE_RESERVED_US > eio_->force_destroy_second * 1000000) { | ||||||
|  |     //   _OB_LOG(ERROR, "pkt process too long time: pkt_receive_ts=%ld, pkt_code=%d", rts, pcode); | ||||||
|  |     // } | ||||||
|  |     //copy packet into req buffer | ||||||
|  |     ObRpcPacketCode pcode = rpc_pkt->get_pcode(); | ||||||
|  |     if (OB_SUCC(ret)) { | ||||||
|  |       ObRpcPacket *packet = response_pkt; | ||||||
|  |       packet->set_pcode(pcode); | ||||||
|  |       packet->set_chid(rpc_pkt->get_chid()); | ||||||
|  |       packet->set_session_id(0);  // not stream | ||||||
|  |       packet->set_trace_id(common::ObCurTraceId::get()); | ||||||
|  |       packet->set_resp(); | ||||||
|  |  | ||||||
|  |       packet->set_request_arrival_time(req_->get_request_arrival_time()); | ||||||
|  |       packet->set_arrival_push_diff(req_->get_arrival_push_diff()); | ||||||
|  |       packet->set_push_pop_diff(req_->get_push_pop_diff()); | ||||||
|  |       packet->set_pop_process_start_diff(req_->get_pop_process_start_diff()); | ||||||
|  |       packet->set_process_start_end_diff(req_->get_process_start_end_diff()); | ||||||
|  |       packet->set_process_end_response_diff(req_->get_process_end_response_diff()); | ||||||
|  |       if (bad_routing) { | ||||||
|  |         packet->set_bad_routing(); | ||||||
|  |       } | ||||||
|  |       packet->calc_checksum(); | ||||||
|  |       req_->get_request()->opacket = packet; | ||||||
|  |     } | ||||||
|  |     //just set request retcode, wakeup in ObSingleServer::handlePacketQueue() | ||||||
|  |     req_->set_request_rtcode(EASY_OK); | ||||||
|  |     obmysql::ObMySQLRequestUtils::wakeup_request(req_); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | template <class T> | ||||||
|  | int ObRpcAsyncResponse<T>::response(const int retcode) | ||||||
|  | { | ||||||
|  |   int ret = common::OB_SUCCESS; | ||||||
|  |   if (OB_ISNULL(req_)) { | ||||||
|  |     ret = common::OB_INVALID_ARGUMENT; | ||||||
|  |     RPC_OBRPC_LOG(WARN, "invalid req, maybe stream rpc timeout", K(ret), K(retcode), | ||||||
|  |                   KP_(req)); | ||||||
|  |   } else { | ||||||
|  |     obrpc::ObRpcResultCode rcode; | ||||||
|  |     rcode.rcode_ = retcode; | ||||||
|  |  | ||||||
|  |     // add warning buffer into result code buffer if rpc fails. | ||||||
|  |     common::ObWarningBuffer *wb = common::ob_get_tsi_warning_buffer(); | ||||||
|  |     if (wb) { | ||||||
|  |       if (retcode != common::OB_SUCCESS) { | ||||||
|  |         (void)snprintf(rcode.msg_, common::OB_MAX_ERROR_MSG_LEN, "%s", wb->get_err_msg()); | ||||||
|  |       } | ||||||
|  |       //always add warning buffer | ||||||
|  |       bool not_null = true; | ||||||
|  |       for (uint32_t idx = 0; OB_SUCC(ret) && not_null && idx < wb->get_readable_warning_count(); idx++) { | ||||||
|  |         const common::ObWarningBuffer::WarningItem *item = wb->get_warning_item(idx); | ||||||
|  |         if (item != NULL) { | ||||||
|  |           if (OB_FAIL(rcode.warnings_.push_back(*item))) { | ||||||
|  |             RPC_OBRPC_LOG(WARN, "Failed to add warning", K(ret)); | ||||||
|  |           } | ||||||
|  |         } else { | ||||||
|  |           not_null = false; | ||||||
|  |         } | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |  | ||||||
|  |     int64_t content_size = common::serialization::encoded_length(result_) + | ||||||
|  |         common::serialization::encoded_length(rcode); | ||||||
|  |  | ||||||
|  |     char *buf = NULL; | ||||||
|  |     if (OB_FAIL(ret)) { | ||||||
|  |       //do nothing | ||||||
|  |     } else if (content_size > common::OB_MAX_PACKET_LENGTH) { | ||||||
|  |       ret = common::OB_RPC_PACKET_TOO_LONG; | ||||||
|  |       RPC_OBRPC_LOG(WARN, "response content size bigger than OB_MAX_PACKET_LENGTH", K(ret)); | ||||||
|  |     } else { | ||||||
|  |       //allocate memory from easy | ||||||
|  |       //[ ObRpcPacket ... ObDatabuffer ... serilized content ...] | ||||||
|  |       int64_t size = (content_size) + sizeof (common::ObDataBuffer) + sizeof(ObRpcPacket); | ||||||
|  |       buf = static_cast<char*>(easy_alloc(size)); | ||||||
|  |       if (NULL == buf) { | ||||||
|  |         ret = common::OB_ALLOCATE_MEMORY_FAILED; | ||||||
|  |         RPC_OBRPC_LOG(WARN, "allocate rpc data buffer fail", K(ret), K(size)); | ||||||
|  |       } else { | ||||||
|  |         using_buffer_ = new (buf + sizeof(ObRpcPacket)) common::ObDataBuffer(); | ||||||
|  |         if (!(using_buffer_->set_data(buf + sizeof(ObRpcPacket) + sizeof (*using_buffer_), | ||||||
|  |             content_size))) { | ||||||
|  |           ret = common::OB_INVALID_ARGUMENT; | ||||||
|  |           RPC_OBRPC_LOG(WARN, "invalid parameters", K(ret)); | ||||||
|  |         } | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |  | ||||||
|  |     if (OB_FAIL(ret)) { | ||||||
|  |       //do nothing | ||||||
|  |     } else if (OB_ISNULL(using_buffer_)) { | ||||||
|  |       ret = common::OB_ERR_UNEXPECTED; | ||||||
|  |       RPC_OBRPC_LOG(ERROR, "using_buffer_ is NULL", K(ret)); | ||||||
|  |     } else if (OB_FAIL(rcode.serialize(using_buffer_->get_data(), | ||||||
|  |         using_buffer_->get_capacity(), | ||||||
|  |         using_buffer_->get_position()))) { | ||||||
|  |       RPC_OBRPC_LOG(WARN, "serialize result code fail", K(ret)); | ||||||
|  |     } else { | ||||||
|  |       // also send result if process successfully. | ||||||
|  |       if (common::OB_SUCCESS == retcode) { | ||||||
|  |         if (OB_FAIL(serialize())) { | ||||||
|  |           RPC_OBRPC_LOG(WARN, "serialize result fail", K(ret)); | ||||||
|  |         } | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |  | ||||||
|  |     // routing check : whether client should refresh location cache and retry | ||||||
|  |     // Now, following the same logic as in ../mysql/ob_query_retry_ctrl.cpp | ||||||
|  |     bool bad_routing = false; | ||||||
|  |     if (OB_SUCC(ret)) { | ||||||
|  |       if (common::OB_SUCCESS != retcode && observer::is_bad_routing_err(retcode)) { | ||||||
|  |         bad_routing = true; | ||||||
|  |         RPC_OBRPC_LOG(WARN, "bad routing", K(retcode), K(bad_routing)); | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |  | ||||||
|  |     if (OB_SUCC(ret)) { | ||||||
|  |       ObRpcPacket *pkt = new (buf) ObRpcPacket(); | ||||||
|  |       //Response rsp(sessid, is_stream_, is_last, pkt); | ||||||
|  |       pkt->set_content(using_buffer_->get_data(), using_buffer_->get_position()); | ||||||
|  |       if (OB_FAIL(do_response(pkt, bad_routing))) { | ||||||
|  |         RPC_OBRPC_LOG(WARN, "response data fail", K(ret)); | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |  | ||||||
|  |     using_buffer_ = NULL; | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  | } // end namespace obrpc | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_RPC_ASYNC_RESPONSE_H */ | ||||||
							
								
								
									
										1489
									
								
								src/observer/table/ob_table_api_row_iterator.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1489
									
								
								src/observer/table/ob_table_api_row_iterator.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										256
									
								
								src/observer/table/ob_table_api_row_iterator.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										256
									
								
								src/observer/table/ob_table_api_row_iterator.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,256 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_API_ROW_ITERATOR_H_ | ||||||
|  | #define OB_TABLE_API_ROW_ITERATOR_H_ | ||||||
|  |  | ||||||
|  | #include "ob_table_service.h" | ||||||
|  | #include "common/row/ob_row_iterator.h" | ||||||
|  |  | ||||||
|  | namespace oceanbase { | ||||||
|  | namespace observer { | ||||||
|  |  | ||||||
|  | class ObTableApiRowIterator : public common::ObNewRowIterator { | ||||||
|  | public: | ||||||
|  |   ObTableApiRowIterator(); | ||||||
|  |   virtual ~ObTableApiRowIterator(); | ||||||
|  |   int init( | ||||||
|  |       storage::ObPartitionService &partition_service, | ||||||
|  |       share::schema::ObMultiVersionSchemaService &schema_service, | ||||||
|  |       ObTableServiceCtx &ctx); | ||||||
|  |   virtual void reset(); | ||||||
|  |   OB_INLINE common::ObIArray<uint64_t> &get_column_ids() { return column_ids_; } | ||||||
|  |   OB_INLINE common::ObIArray<common::ObString> &get_properties() { return properties_; } | ||||||
|  |   OB_INLINE int64_t get_schema_version() { return schema_version_; } | ||||||
|  |   OB_INLINE int64_t get_rowkey_column_cnt() { return rowkey_column_cnt_; } | ||||||
|  | protected: | ||||||
|  |   int check_row(common::ObNewRow &row); | ||||||
|  |   int entity_to_row(const table::ObITableEntity &entity, common::ObIArray<ObObj> &row); | ||||||
|  |   int cons_all_columns(const table::ObITableEntity &entity, const bool ignore_missing_column = false); | ||||||
|  |   int cons_missing_columns(const table::ObITableEntity &entity); | ||||||
|  |   int fill_get_param( | ||||||
|  |       ObTableServiceCtx &ctx, | ||||||
|  |       const table::ObTableOperationType::Type op_type, | ||||||
|  |       ObRowkey &rowkey, | ||||||
|  |       storage::ObTableScanParam &scan_param, | ||||||
|  |       share::schema::ObTableParam &table_param); | ||||||
|  |   int fill_multi_get_param( | ||||||
|  |       ObTableServiceCtx &ctx, | ||||||
|  |       const ObTableBatchOperation &batch_operation, | ||||||
|  |       storage::ObTableScanParam &scan_param, | ||||||
|  |       share::schema::ObTableParam &table_param); | ||||||
|  |   int fill_generate_columns(common::ObNewRow &row); | ||||||
|  |   virtual bool is_read() const { return false; } | ||||||
|  | private: | ||||||
|  |   int check_table_supported(const share::schema::ObTableSchema *table_schema); | ||||||
|  |   int check_column_type(const sql::ObExprResType &column_type, common::ObObj &obj); | ||||||
|  |   int fill_range(const ObRowkey &rowkey, ObIArray<common::ObNewRange> &ranges); | ||||||
|  |   int fill_flag(ObTableServiceCtx &ctx, storage::ObTableScanParam &scan_param); | ||||||
|  |   int add_column_type(const share::schema::ObColumnSchemaV2 &column_schema); | ||||||
|  |   int cons_column_type(const share::schema::ObColumnSchemaV2 &column_schema, sql::ObExprResType &column_type); | ||||||
|  | protected: | ||||||
|  |   static const int64_t COMMON_COLUMN_NUM = 16; | ||||||
|  |   storage::ObPartitionService *part_service_; | ||||||
|  |   share::schema::ObMultiVersionSchemaService *schema_service_; | ||||||
|  |   ObTableServiceCtx *ctx_; | ||||||
|  |   share::schema::ObSchemaGetterGuard schema_guard_; | ||||||
|  |   const share::schema::ObTableSchema *table_schema_; | ||||||
|  |   int64_t table_id_; | ||||||
|  |   int64_t tenant_id_; | ||||||
|  |   int64_t schema_version_; | ||||||
|  |   int64_t rowkey_column_cnt_; | ||||||
|  |   common::ObSEArray<common::ObString, COMMON_COLUMN_NUM> properties_; | ||||||
|  |   common::ObSEArray<uint64_t, COMMON_COLUMN_NUM> column_ids_; | ||||||
|  |   common::ObSEArray<sql::ObExprResType, COMMON_COLUMN_NUM> columns_type_; | ||||||
|  |   common::ObSEArray<share::schema::ObColDesc, COMMON_COLUMN_NUM> column_descs_; | ||||||
|  |   common::ObSEArray<common::ObObj, COMMON_COLUMN_NUM> row_objs_; | ||||||
|  |   common::ObSEArray<common::ObObj, COMMON_COLUMN_NUM> missing_default_objs_; | ||||||
|  |   common::ObSEArray<common::ObISqlExpression*, COMMON_COLUMN_NUM> generate_column_exprs_; | ||||||
|  |   common::ObSEArray<int64_t, COMMON_COLUMN_NUM> generate_column_idxs_; | ||||||
|  |   common::ObExprCtx expr_ctx_; | ||||||
|  |   common::ObNewRow row_; | ||||||
|  |   common::ObArenaAllocator stmt_allocator_; | ||||||
|  |   common::ObArenaAllocator row_allocator_; | ||||||
|  |   const table::ObITableEntity *entity_; | ||||||
|  |   bool has_generate_column_; | ||||||
|  |   bool is_inited_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiInsertRowIterator : public ObTableApiRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiInsertRowIterator(); | ||||||
|  |   virtual ~ObTableApiInsertRowIterator(); | ||||||
|  |   int open(const ObTableOperation &table_operation); | ||||||
|  |   virtual int get_next_row(common::ObNewRow *&row); | ||||||
|  | protected: | ||||||
|  |   int cons_row(const table::ObITableEntity &entity, common::ObNewRow *&row); | ||||||
|  |   virtual bool is_read() const override { return false; } | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiMultiInsertRowIterator : public ObTableApiInsertRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiMultiInsertRowIterator(); | ||||||
|  |   virtual ~ObTableApiMultiInsertRowIterator(); | ||||||
|  |   virtual void reset(); | ||||||
|  |   int open(const ObTableBatchOperation &table_operation); | ||||||
|  |   virtual int get_next_row(common::ObNewRow *&row); | ||||||
|  |   OB_INLINE void continue_iter() { is_iter_pause_ = false; } | ||||||
|  | private: | ||||||
|  |   const ObTableBatchOperation *batch_operation_; | ||||||
|  |   int64_t row_idx_; | ||||||
|  |   int64_t batch_cnt_; | ||||||
|  |   bool is_iter_pause_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiUpdateRowIterator : public ObTableApiRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiUpdateRowIterator(); | ||||||
|  |   virtual ~ObTableApiUpdateRowIterator(); | ||||||
|  |   virtual void reset(); | ||||||
|  |   int open(const ObTableOperation &table_operation, | ||||||
|  |       const ObRowkey &rowkey, bool need_update_rowkey = false); | ||||||
|  |   virtual int get_next_row(common::ObNewRow *&row); | ||||||
|  |   OB_INLINE common::ObIArray<uint64_t> &get_update_column_ids() { return update_column_ids_; } | ||||||
|  |   OB_INLINE common::ObNewRow *get_cur_new_row() { return new_row_; } | ||||||
|  | protected: | ||||||
|  |   int cons_update_columns(bool need_update_rowkey); | ||||||
|  |   int cons_new_row(const ObTableOperation &table_operation, common::ObNewRow *&row); | ||||||
|  |   virtual bool is_read() const override { return false; } | ||||||
|  | private: | ||||||
|  |   int obj_increment( | ||||||
|  |       const common::ObObj &delta, | ||||||
|  |       const common::ObObj &src, | ||||||
|  |       const sql::ObExprResType target_type, | ||||||
|  |       common::ObObj &target); | ||||||
|  |   int obj_append( | ||||||
|  |       const common::ObObj &delta, | ||||||
|  |       const common::ObObj &src, | ||||||
|  |       const sql::ObExprResType target_type, | ||||||
|  |       common::ObObj &target); | ||||||
|  |   int int_add_int_with_check( | ||||||
|  |       int64_t old_int, | ||||||
|  |       int64_t delta_int, | ||||||
|  |       common::ObObjType result_type, | ||||||
|  |       common::ObObj &result); | ||||||
|  |   int uint_add_int_with_check( | ||||||
|  |       uint64_t old_uint, | ||||||
|  |       int64_t delta_int, | ||||||
|  |       common::ObObjType result_type, | ||||||
|  |       common::ObObj &result); | ||||||
|  | protected: | ||||||
|  |   storage::ObTableScanParam scan_param_; | ||||||
|  |   share::schema::ObTableParam table_param_; | ||||||
|  |   common::ObSEArray<uint64_t, COMMON_COLUMN_NUM> update_column_ids_; | ||||||
|  |   common::ObNewRowIterator *scan_iter_; | ||||||
|  |   common::ObNewRow *old_row_; | ||||||
|  |   common::ObNewRow *new_row_; | ||||||
|  |   int64_t row_idx_; | ||||||
|  |   bool need_update_rowkey_; | ||||||
|  | private: | ||||||
|  |   const ObTableOperation *table_operation_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiMultiUpdateRowIterator : public ObTableApiUpdateRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiMultiUpdateRowIterator(); | ||||||
|  |   virtual ~ObTableApiMultiUpdateRowIterator(); | ||||||
|  |   virtual void reset(); | ||||||
|  |   int open(const ObTableBatchOperation &batch_operation); | ||||||
|  |   virtual int get_next_row(common::ObNewRow *&row); | ||||||
|  |   OB_INLINE void continue_iter() { is_iter_pause_ = false; } | ||||||
|  |   OB_INLINE int64_t get_cur_update_idx() { return cur_update_idx_; } | ||||||
|  |   OB_INLINE bool has_finished() { return batch_idx_ >= batch_cnt_; } | ||||||
|  | private: | ||||||
|  |   const ObTableBatchOperation *batch_operation_; | ||||||
|  |   int64_t batch_cnt_; | ||||||
|  |   int64_t batch_idx_; | ||||||
|  |   int64_t cur_update_idx_; | ||||||
|  |   bool is_iter_pause_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiDeleteRowIterator : public ObTableApiRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiDeleteRowIterator(); | ||||||
|  |   virtual ~ObTableApiDeleteRowIterator(); | ||||||
|  |   virtual void reset(); | ||||||
|  |   int open(const ObTableOperation &table_operation); | ||||||
|  |   virtual int get_next_row(common::ObNewRow *&row); | ||||||
|  |   OB_INLINE common::ObIArray<uint64_t> &get_delete_column_ids() { return column_ids_; } | ||||||
|  | protected: | ||||||
|  |   virtual bool is_read() const override { return false; } | ||||||
|  | protected: | ||||||
|  |   storage::ObTableScanParam scan_param_; | ||||||
|  |   share::schema::ObTableParam table_param_; | ||||||
|  |   common::ObNewRowIterator *scan_iter_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiMultiDeleteRowIterator : public ObTableApiDeleteRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiMultiDeleteRowIterator(); | ||||||
|  |   virtual ~ObTableApiMultiDeleteRowIterator(); | ||||||
|  |   virtual void reset(); | ||||||
|  |   int open(const ObTableBatchOperation &table_operation); | ||||||
|  |   virtual int get_next_row(common::ObNewRow *&row); | ||||||
|  |   OB_INLINE void continue_iter() { is_iter_pause_ = false; } | ||||||
|  |   OB_INLINE int64_t get_cur_delete_idx() { return cur_delete_idx_; } | ||||||
|  |   OB_INLINE bool has_finished() { return batch_idx_ >= batch_cnt_; } | ||||||
|  | private: | ||||||
|  |   const ObTableBatchOperation *batch_operation_; | ||||||
|  |   int64_t batch_cnt_; | ||||||
|  |   int64_t batch_idx_; | ||||||
|  |   int64_t cur_delete_idx_; | ||||||
|  |   bool is_iter_pause_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiGetRowIterator : public ObTableApiRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiGetRowIterator(); | ||||||
|  |   virtual ~ObTableApiGetRowIterator(); | ||||||
|  |   virtual void reset(); | ||||||
|  |   int open(const ObTableOperation &table_operation); | ||||||
|  |   virtual int get_next_row(common::ObNewRow *&row); | ||||||
|  | protected: | ||||||
|  |   virtual bool is_read() const override { return true; } | ||||||
|  | protected: | ||||||
|  |   storage::ObTableScanParam scan_param_; | ||||||
|  |   share::schema::ObTableParam table_param_; | ||||||
|  |   common::ObNewRowIterator *scan_iter_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableApiMultiGetRowIterator : public ObTableApiGetRowIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableApiMultiGetRowIterator(); | ||||||
|  |   virtual ~ObTableApiMultiGetRowIterator(); | ||||||
|  |   int open(const ObTableBatchOperation &table_operation); | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | #endif /* OB_TABLE_API_ROW_ITERATOR_H_ */ | ||||||
							
								
								
									
										490
									
								
								src/observer/table/ob_table_batch_execute_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										490
									
								
								src/observer/table/ob_table_batch_execute_processor.cpp
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,490 @@ | |||||||
|  | /** | ||||||
|  |  * 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 SERVER | ||||||
|  | #include "ob_table_batch_execute_processor.h" | ||||||
|  | #include "ob_table_rpc_processor_util.h" | ||||||
|  | #include "observer/ob_service.h" | ||||||
|  | #include "storage/ob_partition_service.h" | ||||||
|  | #include "ob_table_end_trans_cb.h" | ||||||
|  | #include "sql/optimizer/ob_table_location.h"  // ObTableLocation | ||||||
|  | #include "lib/stat/ob_diagnose_info.h" | ||||||
|  | #include "lib/stat/ob_session_stat.h" | ||||||
|  |  | ||||||
|  | using namespace oceanbase::observer; | ||||||
|  | using namespace oceanbase::common; | ||||||
|  | using namespace oceanbase::table; | ||||||
|  | using namespace oceanbase::share; | ||||||
|  | using namespace oceanbase::sql; | ||||||
|  |  | ||||||
|  | ObTableBatchExecuteP::ObTableBatchExecuteP(const ObGlobalContext &gctx) | ||||||
|  |     :ObTableRpcProcessor(gctx), | ||||||
|  |      allocator_(ObModIds::TABLE_PROC), | ||||||
|  |      table_service_ctx_(allocator_), | ||||||
|  |      need_rollback_trans_(false) | ||||||
|  | { | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::deserialize() | ||||||
|  | { | ||||||
|  |   // we should set entity factory before deserialize | ||||||
|  |   arg_.batch_operation_.set_entity_factory(&default_entity_factory_); | ||||||
|  |   result_.set_entity_factory(&default_entity_factory_); | ||||||
|  |   int ret = ParentType::deserialize(); | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::check_arg() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (arg_.consistency_level_ != ObTableConsistencyLevel::STRONG) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("some options not supported yet", K(ret), | ||||||
|  |              "consistency_level", arg_.consistency_level_); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::check_arg2() const | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (arg_.returning_rowkey_ | ||||||
|  |       || arg_.returning_affected_entity_) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("some options not supported yet", K(ret), | ||||||
|  |              "returning_rowkey", arg_.returning_rowkey_, | ||||||
|  |              "returning_affected_entity", arg_.returning_affected_entity_); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | OB_INLINE bool is_errno_need_retry(int ret) | ||||||
|  | { | ||||||
|  |   return OB_TRY_LOCK_ROW_CONFLICT == ret | ||||||
|  |       || OB_TRANSACTION_SET_VIOLATION == ret | ||||||
|  |       || OB_SCHEMA_ERROR == ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableBatchExecuteP::audit_on_finish() | ||||||
|  | { | ||||||
|  |   audit_record_.consistency_level_ = ObTableConsistencyLevel::STRONG == arg_.consistency_level_ ? | ||||||
|  |       ObConsistencyLevel::STRONG : ObConsistencyLevel::WEAK; | ||||||
|  |   audit_record_.return_rows_ = arg_.returning_affected_rows_ ? result_.count() : 0; | ||||||
|  |   audit_record_.table_scan_ = false; | ||||||
|  |   audit_record_.affected_rows_ = result_.count(); | ||||||
|  |   audit_record_.try_cnt_ = retry_count_ + 1; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | uint64_t ObTableBatchExecuteP::get_request_checksum() | ||||||
|  | { | ||||||
|  |   uint64_t checksum = 0; | ||||||
|  |   checksum = ob_crc64(checksum, arg_.table_name_.ptr(), arg_.table_name_.length()); | ||||||
|  |   const uint64_t op_checksum = arg_.batch_operation_.get_checksum(); | ||||||
|  |   checksum = ob_crc64(checksum, &op_checksum, sizeof(op_checksum)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.consistency_level_, sizeof(arg_.consistency_level_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.returning_rowkey_, sizeof(arg_.returning_rowkey_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.returning_affected_entity_, sizeof(arg_.returning_affected_entity_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.returning_affected_rows_, sizeof(arg_.returning_affected_rows_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.binlog_row_image_type_, sizeof(arg_.binlog_row_image_type_)); | ||||||
|  |   return checksum; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::response(const int retcode) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (!need_retry_in_queue_ && !did_async_end_trans()) { | ||||||
|  |     ret = ObRpcProcessor::response(retcode); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableBatchExecuteP::reset_ctx() | ||||||
|  | { | ||||||
|  |   table_service_ctx_.reset_dml(); | ||||||
|  |   need_retry_in_queue_ = false; | ||||||
|  |   need_rollback_trans_ = false; | ||||||
|  |   result_.reset(); | ||||||
|  |   ObTableApiProcessorBase::reset_ctx(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::try_process() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   if (batch_operation.count() <= 0) { | ||||||
|  |     ret = OB_INVALID_ARGUMENT; | ||||||
|  |     LOG_WARN("no operation in the batch", K(ret)); | ||||||
|  |   } else { | ||||||
|  |     if (batch_operation.is_readonly()) { | ||||||
|  |       if (batch_operation.is_same_properties_names()) { | ||||||
|  |         stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_GET; | ||||||
|  |         ret = multi_get(); | ||||||
|  |       } else { | ||||||
|  |         stat_event_type_ = ObTableProccessType::TABLE_API_BATCH_RETRIVE; | ||||||
|  |         ret = batch_execute(true); | ||||||
|  |       } | ||||||
|  |     } else if (batch_operation.is_same_type()) { | ||||||
|  |       switch(batch_operation.at(0).type()) { | ||||||
|  |         case ObTableOperationType::INSERT: | ||||||
|  |           stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_INSERT; | ||||||
|  |           ret = multi_insert(); | ||||||
|  |           break; | ||||||
|  |         case ObTableOperationType::DEL: | ||||||
|  |           stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_DELETE; | ||||||
|  |           ret = multi_delete(); | ||||||
|  |           break; | ||||||
|  |         case ObTableOperationType::UPDATE: | ||||||
|  |           stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_UPDATE; | ||||||
|  |           ret = multi_update(); | ||||||
|  |           break; | ||||||
|  |         case ObTableOperationType::INSERT_OR_UPDATE: | ||||||
|  |           stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_INSERT_OR_UPDATE; | ||||||
|  |           ret = multi_insert_or_update(); | ||||||
|  |           break; | ||||||
|  |         case ObTableOperationType::REPLACE: | ||||||
|  |           stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_REPLACE; | ||||||
|  |           ret = multi_replace(); | ||||||
|  |           break; | ||||||
|  |         case ObTableOperationType::APPEND: | ||||||
|  |           stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_APPEND; | ||||||
|  |           ret = batch_execute(false); | ||||||
|  |           break; | ||||||
|  |         case ObTableOperationType::INCREMENT: | ||||||
|  |           stat_event_type_ = ObTableProccessType::TABLE_API_MULTI_INCREMENT; | ||||||
|  |           ret = batch_execute(false); | ||||||
|  |           break; | ||||||
|  |         default: | ||||||
|  |           ret = OB_ERR_UNEXPECTED; | ||||||
|  |           LOG_ERROR("unexpected operation type", "type", batch_operation.at(0).type(), K(stat_event_type_)); | ||||||
|  |           break; | ||||||
|  |       } | ||||||
|  |     } else { | ||||||
|  |       // complex batch hybrid operation | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_BATCH_HYBRID; | ||||||
|  |       ret = batch_execute(false); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |  | ||||||
|  |   // record events | ||||||
|  |   audit_row_count_ = arg_.batch_operation_.count(); | ||||||
|  |  | ||||||
|  | #ifndef NDEBUG | ||||||
|  |   // debug mode | ||||||
|  |   LOG_INFO("[TABLE] execute batch operation", K(ret), K_(arg), K_(result), "timeout", rpc_pkt_->get_timeout(), K_(retry_count)); | ||||||
|  | #else | ||||||
|  |   // release mode | ||||||
|  |   LOG_TRACE("[TABLE] execute batch operation", K(ret), K_(arg), K_(result), "timeout", rpc_pkt_->get_timeout(), K_(retry_count), | ||||||
|  |             "receive_ts", get_receive_timestamp()); | ||||||
|  | #endif | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | ObTableAPITransCb *ObTableBatchExecuteP::new_callback(rpc::ObRequest *req) | ||||||
|  | { | ||||||
|  |   ObTableBatchExecuteEndTransCb *cb = OB_NEW(ObTableBatchExecuteEndTransCb, ObModIds::TABLE_PROC, req, arg_.batch_operation_.at(0).type()); | ||||||
|  |   if (NULL != cb) { | ||||||
|  |     // @todo optimize to avoid this copy | ||||||
|  |     int ret = OB_SUCCESS; | ||||||
|  |     if (OB_FAIL(cb->assign_batch_execute_result(result_))) { | ||||||
|  |       LOG_WARN("failed to assign result", K(ret)); | ||||||
|  |       cb->~ObTableBatchExecuteEndTransCb(); | ||||||
|  |       cb = NULL; | ||||||
|  |     } else { | ||||||
|  |       LOG_DEBUG("[yzfdebug] copy result", K_(result)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return cb; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::get_rowkeys(ObIArray<ObRowkey> &rowkeys) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   const int64_t N = batch_operation.count(); | ||||||
|  |   for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) | ||||||
|  |   { | ||||||
|  |     const ObTableOperation &table_op = batch_operation.at(i); | ||||||
|  |     ObRowkey rowkey = const_cast<ObITableEntity&>(table_op.entity()).get_rowkey(); | ||||||
|  |     if (OB_FAIL(rowkeys.push_back(rowkey))) { | ||||||
|  |       LOG_WARN("failed to push back", K(ret)); | ||||||
|  |     } | ||||||
|  |   } // end for | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::get_partition_ids(uint64_t table_id, ObIArray<int64_t> &part_ids) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   uint64_t partition_id = arg_.partition_id_; | ||||||
|  |   if (OB_INVALID_ID == partition_id) { | ||||||
|  |     ObSEArray<sql::RowkeyArray, 3> rowkeys_per_part; | ||||||
|  |     ObSEArray<ObRowkey, 3> rowkeys; | ||||||
|  |     if (OB_FAIL(get_rowkeys(rowkeys))) { | ||||||
|  |       LOG_WARN("failed to get rowkeys", K(ret)); | ||||||
|  |     } else if (OB_FAIL(get_partition_by_rowkey(table_id, rowkeys, part_ids, rowkeys_per_part))) { | ||||||
|  |       LOG_WARN("failed to get partition", K(ret), K(rowkeys)); | ||||||
|  |     } | ||||||
|  |   } else { | ||||||
|  |     if (OB_FAIL(part_ids.push_back(partition_id))) { | ||||||
|  |       LOG_WARN("failed to push back", K(ret)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::multi_insert_or_update() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                                 arg_.returning_affected_rows_, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 arg_.binlog_row_image_type_); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_INSERT, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->multi_insert_or_update(table_service_ctx_, batch_operation, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to insert_or_update", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans"); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::multi_get() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   need_rollback_trans_ = false; | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                                 arg_.returning_affected_rows_, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 arg_.binlog_row_image_type_); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   const bool is_readonly = true; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_SELECT, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start readonly transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->multi_get(table_service_ctx_, arg_.batch_operation_, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to execute get", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } else {} | ||||||
|  |   need_rollback_trans_ = (OB_SUCCESS != ret); | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(need_rollback_trans_, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret), "rollback", need_rollback_trans_); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::multi_delete() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                                 arg_.returning_affected_rows_, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 arg_.binlog_row_image_type_); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_DELETE, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->multi_delete(table_service_ctx_, batch_operation, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to multi_delete", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans"); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::multi_insert() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                                 arg_.returning_affected_rows_, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 arg_.binlog_row_image_type_); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_INSERT, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->multi_insert(table_service_ctx_, batch_operation, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to multi_insert", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans"); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::multi_replace() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                                 arg_.returning_affected_rows_, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 arg_.binlog_row_image_type_); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_REPLACE, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->multi_replace(table_service_ctx_, batch_operation, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to multi_replace", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans"); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::multi_update() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                                 arg_.returning_affected_rows_, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 arg_.binlog_row_image_type_/*important*/); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_UPDATE, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->multi_update(table_service_ctx_, batch_operation, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to multi_update", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans"); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteP::batch_execute(bool is_readonly) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableBatchOperation &batch_operation = arg_.batch_operation_; | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                                 arg_.returning_affected_rows_, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 arg_.binlog_row_image_type_, | ||||||
|  |                                 arg_.returning_affected_entity_, | ||||||
|  |                                 arg_.returning_rowkey_); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, (is_readonly ? sql::stmt::T_SELECT : sql::stmt::T_UPDATE), | ||||||
|  |                                  table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->batch_execute(table_service_ctx_, batch_operation, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to execute batch", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans"); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
							
								
								
									
										65
									
								
								src/observer/table/ob_table_batch_execute_processor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										65
									
								
								src/observer/table/ob_table_batch_execute_processor.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 _OB_TABLE_BATCH_EXECUTE_PROCESSOR_H | ||||||
|  | #define _OB_TABLE_BATCH_EXECUTE_PROCESSOR_H 1 | ||||||
|  | #include "rpc/obrpc/ob_rpc_proxy.h" | ||||||
|  | #include "rpc/obrpc/ob_rpc_processor.h" | ||||||
|  | #include "share/table/ob_table_rpc_proxy.h" | ||||||
|  | #include "ob_table_rpc_processor.h" | ||||||
|  | #include "ob_table_service.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace observer | ||||||
|  | { | ||||||
|  | /// @see RPC_S(PR5 batch_execute, obrpc::OB_TABLE_API_BATCH_EXECUTE, (table::ObTableBatchOperationRequest), table::ObTableBatchOperationResult); | ||||||
|  | class ObTableBatchExecuteP: public ObTableRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_BATCH_EXECUTE> > | ||||||
|  | { | ||||||
|  |   typedef ObTableRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_BATCH_EXECUTE> > ParentType; | ||||||
|  | public: | ||||||
|  |   explicit ObTableBatchExecuteP(const ObGlobalContext &gctx); | ||||||
|  |   virtual ~ObTableBatchExecuteP() = default; | ||||||
|  |  | ||||||
|  |   virtual int deserialize() override; | ||||||
|  |   virtual int response(const int retcode) override; | ||||||
|  | protected: | ||||||
|  |   virtual int check_arg() override; | ||||||
|  |   virtual int try_process() override; | ||||||
|  |   virtual void reset_ctx() override; | ||||||
|  |   table::ObTableAPITransCb *new_callback(rpc::ObRequest *req) override; | ||||||
|  |   virtual void audit_on_finish() override; | ||||||
|  |   virtual uint64_t get_request_checksum() override; | ||||||
|  |  | ||||||
|  | private: | ||||||
|  |   int check_arg2() const; | ||||||
|  |   int get_rowkeys(common::ObIArray<common::ObRowkey> &rowkeys); | ||||||
|  |   int get_partition_ids(uint64_t table_id, common::ObIArray<int64_t> &part_ids); | ||||||
|  |   int multi_insert_or_update(); | ||||||
|  |   int multi_get(); | ||||||
|  |   int multi_delete(); | ||||||
|  |   int multi_insert(); | ||||||
|  |   int multi_replace(); | ||||||
|  |   int multi_update(); | ||||||
|  |   int batch_execute(bool is_readonly); | ||||||
|  | private: | ||||||
|  |   static const int64_t COMMON_COLUMN_NUM = 16; | ||||||
|  |   table::ObTableEntityFactory<table::ObTableEntity> default_entity_factory_; | ||||||
|  |   table::ObTableEntity result_entity_; | ||||||
|  |   common::ObArenaAllocator allocator_; | ||||||
|  |   ObTableServiceGetCtx table_service_ctx_; | ||||||
|  |   bool need_rollback_trans_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | } // end namespace observer | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_BATCH_EXECUTE_PROCESSOR_H */ | ||||||
							
								
								
									
										162
									
								
								src/observer/table/ob_table_end_trans_cb.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										162
									
								
								src/observer/table/ob_table_end_trans_cb.cpp
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,162 @@ | |||||||
|  | /** | ||||||
|  |  * 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 SERVER | ||||||
|  | #include "ob_table_end_trans_cb.h" | ||||||
|  | using namespace oceanbase::common; | ||||||
|  | using namespace oceanbase::table; | ||||||
|  | ObTableAPITransCb::ObTableAPITransCb() | ||||||
|  |     :ref_count_(2) | ||||||
|  | {} | ||||||
|  |  | ||||||
|  | ObTableAPITransCb::~ObTableAPITransCb() | ||||||
|  | { | ||||||
|  |   LOG_DEBUG("[yzfdebug] ObTableAPITransCb destruct", K_(ref_count)); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableAPITransCb::destroy_cb_if_no_ref() | ||||||
|  | { | ||||||
|  |   int32_t new_ref = ATOMIC_SAF(&ref_count_, 1); | ||||||
|  |   if (0 >= new_ref) { | ||||||
|  |     // @caution !!! | ||||||
|  |     this->~ObTableAPITransCb(); | ||||||
|  |     ob_free(this); | ||||||
|  |   } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | void ObTableExecuteEndTransCb::callback(int cb_param) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (OB_UNLIKELY(!has_set_need_rollback_)) { | ||||||
|  |     LOG_ERROR("is_need_rollback_ has not been set", | ||||||
|  |               K(has_set_need_rollback_), | ||||||
|  |               K(is_need_rollback_)); | ||||||
|  |   } else if (OB_UNLIKELY(ObExclusiveEndTransCallback::END_TRANS_TYPE_INVALID == end_trans_type_)) { | ||||||
|  |     LOG_ERROR("end trans type is invalid", K(cb_param), K(end_trans_type_)); | ||||||
|  |   } else if (!is_txs_end_trans_called()) { | ||||||
|  | 	//has NOT invoke the end trans interface | ||||||
|  |     LOG_WARN("fail before trans service end trans, disconnct", K(cb_param)); | ||||||
|  |     if (OB_UNLIKELY(OB_SUCCESS == cb_param)) { | ||||||
|  |       LOG_ERROR("callback before trans service end trans, but ret is OB_SUCCESS, it is BUG!!!", | ||||||
|  |                 K(cb_param), K_(end_trans_type)); | ||||||
|  |     } | ||||||
|  |   } else { | ||||||
|  |     //has invoke the end trans interface | ||||||
|  |   } | ||||||
|  |   this->handin(); | ||||||
|  |   CHECK_BALANCE("[table async callback]"); | ||||||
|  |   if (cb_param != OB_SUCCESS) { | ||||||
|  |     // commit failed | ||||||
|  |     result_.set_errno(cb_param); | ||||||
|  |     result_.set_affected_rows(0); | ||||||
|  |     result_entity_.reset(); | ||||||
|  |   } | ||||||
|  |   if (OB_FAIL(response_sender_.response(cb_param))) { | ||||||
|  |     LOG_WARN("failed to send response", K(ret), K(cb_param)); | ||||||
|  |   } else { | ||||||
|  |     LOG_DEBUG("yzfdebug async send execute response", K(cb_param)); | ||||||
|  |   } | ||||||
|  |   this->destroy_cb_if_no_ref(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableExecuteEndTransCb::callback(int cb_param, const transaction::ObTransID &trans_id) | ||||||
|  | { | ||||||
|  |   UNUSED(trans_id); | ||||||
|  |   this->callback(cb_param); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // when the operation is append/increment and returning_affected_entity is true, we will return the | ||||||
|  | // new values after append/increment to the client, so we need to deep copy the entity_result here. | ||||||
|  | int ObTableExecuteEndTransCb::assign_execute_result(ObTableOperationResult &result) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObITableEntity *src_entity = NULL; | ||||||
|  |   if (OB_FAIL(result.get_entity(src_entity))) { | ||||||
|  |     LOG_WARN("failed to get entity", K(ret)); | ||||||
|  |   } else if (OB_FAIL(result_entity_.deep_copy(allocator_, *src_entity))) { | ||||||
|  |     LOG_WARN("failed to copy entity", K(ret)); | ||||||
|  |   } else { | ||||||
|  |     result_ = result; | ||||||
|  |     result_.set_entity(result_entity_); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | void ObTableBatchExecuteEndTransCb::callback(int cb_param) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (OB_UNLIKELY(!has_set_need_rollback_)) { | ||||||
|  |     LOG_ERROR("is_need_rollback_ has not been set", | ||||||
|  |               K(has_set_need_rollback_), | ||||||
|  |               K(is_need_rollback_)); | ||||||
|  |   } else if (OB_UNLIKELY(ObExclusiveEndTransCallback::END_TRANS_TYPE_INVALID == end_trans_type_)) { | ||||||
|  |     LOG_ERROR("end trans type is invalid", K(cb_param), K(end_trans_type_)); | ||||||
|  |   } else if (!is_txs_end_trans_called()) { | ||||||
|  | 	//has NOT invoked the end trans interface | ||||||
|  |     LOG_WARN("fail before trans service end trans, disconnct", K(cb_param)); | ||||||
|  |     if (OB_UNLIKELY(OB_SUCCESS == cb_param)) { | ||||||
|  |       LOG_ERROR("callback before trans service end trans, but ret is OB_SUCCESS, it is BUG!!!", | ||||||
|  |                 K(cb_param), K_(end_trans_type)); | ||||||
|  |     } | ||||||
|  |   } else { | ||||||
|  |     //has invoked the end trans interface | ||||||
|  |   } | ||||||
|  |   this->handin(); | ||||||
|  |   CHECK_BALANCE("[table batch async callback]"); | ||||||
|  |   if (cb_param != OB_SUCCESS) { | ||||||
|  |     result_.reset(); | ||||||
|  |   } | ||||||
|  |   if (0 >= result_.count()) { | ||||||
|  |     // same result for all | ||||||
|  |     ObTableOperationResult single_op_result; | ||||||
|  |     single_op_result.set_entity(result_entity_); | ||||||
|  |     single_op_result.set_errno(cb_param); | ||||||
|  |     single_op_result.set_type(table_operation_type_); | ||||||
|  |     if (OB_FAIL(result_.push_back(single_op_result))) { | ||||||
|  |       LOG_WARN("failed to add result", K(ret));  // @todo reset the connection | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   if (OB_SUCC(ret)) { | ||||||
|  |     if (OB_FAIL(response_sender_.response(cb_param))) { | ||||||
|  |       LOG_WARN("failed to send response", K(ret), K(cb_param)); | ||||||
|  |     } else { | ||||||
|  |       LOG_DEBUG("yzfdebug async send batch_execute response", K(cb_param)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   this->destroy_cb_if_no_ref(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableBatchExecuteEndTransCb::callback(int cb_param, const transaction::ObTransID &trans_id) | ||||||
|  | { | ||||||
|  |   UNUSED(trans_id); | ||||||
|  |   this->callback(cb_param); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableBatchExecuteEndTransCb::assign_batch_execute_result(ObTableBatchOperationResult &result) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   result_.reset(); | ||||||
|  |   ObTableOperationResult dest_result; | ||||||
|  |   int64_t N = result.count(); | ||||||
|  |   for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) | ||||||
|  |   { | ||||||
|  |     const ObTableOperationResult &src_result = result.at(i); | ||||||
|  |     if (OB_FAIL(dest_result.deep_copy(allocator_, entity_factory_, src_result))) { | ||||||
|  |       LOG_WARN("failed to deep copy result", K(ret)); | ||||||
|  |     } else if (OB_FAIL(result_.push_back(dest_result))) { | ||||||
|  |       LOG_WARN("failed to push back", K(ret)); | ||||||
|  |     } | ||||||
|  |   } // end for | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
							
								
								
									
										90
									
								
								src/observer/table/ob_table_end_trans_cb.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										90
									
								
								src/observer/table/ob_table_end_trans_cb.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,90 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_END_TRANS_CB_H | ||||||
|  | #define _OB_TABLE_END_TRANS_CB_H 1 | ||||||
|  | #include "ob_rpc_async_response.h" | ||||||
|  | #include "sql/ob_end_trans_callback.h" | ||||||
|  | #include "share/table/ob_table.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace table | ||||||
|  | { | ||||||
|  | class ObTableAPITransCb: public sql::ObExclusiveEndTransCallback | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableAPITransCb(); | ||||||
|  |   virtual ~ObTableAPITransCb(); | ||||||
|  |   void destroy_cb_if_no_ref(); | ||||||
|  | private: | ||||||
|  |   int32_t ref_count_; | ||||||
|  |   // disallow copy | ||||||
|  |   DISALLOW_COPY_AND_ASSIGN(ObTableAPITransCb); | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | class ObTableExecuteEndTransCb: public ObTableAPITransCb | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableExecuteEndTransCb(rpc::ObRequest *req, ObTableOperationType::Type table_operation_type) | ||||||
|  |       :response_sender_(req, result_) | ||||||
|  |   { | ||||||
|  |     result_.set_type(table_operation_type); | ||||||
|  |   } | ||||||
|  |   virtual ~ObTableExecuteEndTransCb() = default; | ||||||
|  |  | ||||||
|  |   virtual void callback(int cb_param) override; | ||||||
|  |   virtual void callback(int cb_param, const transaction::ObTransID &trans_id) override; | ||||||
|  |   virtual const char *get_type() const override { return "ObTableEndTransCallback"; } | ||||||
|  |   virtual sql::ObEndTransCallbackType get_callback_type() const override { return sql::ASYNC_CALLBACK_TYPE; } | ||||||
|  |   int assign_execute_result(ObTableOperationResult &result); | ||||||
|  | private: | ||||||
|  |   // disallow copy | ||||||
|  |   DISALLOW_COPY_AND_ASSIGN(ObTableExecuteEndTransCb); | ||||||
|  | private: | ||||||
|  |   ObTableEntity result_entity_; | ||||||
|  |   common::ObArenaAllocator allocator_; | ||||||
|  |   ObTableOperationResult result_; | ||||||
|  |   obrpc::ObRpcAsyncResponse<ObTableOperationResult> response_sender_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObTableBatchExecuteEndTransCb: public ObTableAPITransCb | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableBatchExecuteEndTransCb(rpc::ObRequest *req, ObTableOperationType::Type table_operation_type) | ||||||
|  |       :response_sender_(req, result_), | ||||||
|  |       table_operation_type_(table_operation_type) | ||||||
|  |   { | ||||||
|  |   } | ||||||
|  |   virtual ~ObTableBatchExecuteEndTransCb() = default; | ||||||
|  |  | ||||||
|  |   virtual void callback(int cb_param) override; | ||||||
|  |   virtual void callback(int cb_param, const transaction::ObTransID &trans_id) override; | ||||||
|  |   virtual const char *get_type() const override { return "ObTableBatchEndTransCallback"; } | ||||||
|  |   virtual sql::ObEndTransCallbackType get_callback_type() const override { return sql::ASYNC_CALLBACK_TYPE; } | ||||||
|  |   int assign_batch_execute_result(ObTableBatchOperationResult &result); | ||||||
|  | private: | ||||||
|  |   // disallow copy | ||||||
|  |   DISALLOW_COPY_AND_ASSIGN(ObTableBatchExecuteEndTransCb); | ||||||
|  | private: | ||||||
|  |   ObTableEntity result_entity_; | ||||||
|  |   common::ObArenaAllocator allocator_; | ||||||
|  |   table::ObTableEntityFactory<table::ObTableEntity> entity_factory_; | ||||||
|  |   ObTableBatchOperationResult result_; | ||||||
|  |   obrpc::ObRpcAsyncResponse<ObTableBatchOperationResult> response_sender_; | ||||||
|  |   ObTableOperationType::Type table_operation_type_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | } // end namespace table | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_END_TRANS_CB_H */ | ||||||
							
								
								
									
										500
									
								
								src/observer/table/ob_table_execute_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										500
									
								
								src/observer/table/ob_table_execute_processor.cpp
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,500 @@ | |||||||
|  | /** | ||||||
|  |  * 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 SERVER | ||||||
|  | #include "ob_table_execute_processor.h" | ||||||
|  | #include "ob_table_rpc_processor_util.h" | ||||||
|  | #include "observer/ob_service.h" | ||||||
|  | #include "storage/ob_partition_service.h" | ||||||
|  | #include "ob_table_end_trans_cb.h" | ||||||
|  | #include "sql/optimizer/ob_table_location.h"  // ObTableLocation | ||||||
|  | #include "lib/stat/ob_session_stat.h" | ||||||
|  |  | ||||||
|  | using namespace oceanbase::observer; | ||||||
|  | using namespace oceanbase::common; | ||||||
|  | using namespace oceanbase::table; | ||||||
|  | using namespace oceanbase::share; | ||||||
|  | using namespace oceanbase::sql; | ||||||
|  |  | ||||||
|  | int ObTableRpcProcessorUtil::negate_htable_timestamp(table::ObITableEntity &entity) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   // negative the value of T | ||||||
|  |   ObObj T_val; | ||||||
|  |   int64_t val = 0; | ||||||
|  |   if (3 == entity.get_rowkey_size()) { | ||||||
|  |     if (OB_FAIL(entity.get_rowkey_value(2, T_val))) { | ||||||
|  |       LOG_WARN("failed to get T from entity", K(ret), K(entity)); | ||||||
|  |     } else if (OB_FAIL(T_val.get_int(val))) { | ||||||
|  |       LOG_WARN("invalid obj type for T", K(ret), K(T_val)); | ||||||
|  |     } else { | ||||||
|  |       T_val.set_int(-val); | ||||||
|  |       if (OB_FAIL(entity.set_rowkey_value(2, T_val))) { | ||||||
|  |         LOG_WARN("failed to negate T value", K(ret)); | ||||||
|  |       } else { | ||||||
|  |         LOG_DEBUG("[yzfdebug] nenative T value", K(ret), K(T_val)); | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | ObTableApiExecuteP::ObTableApiExecuteP(const ObGlobalContext &gctx) | ||||||
|  |     :ObTableRpcProcessor(gctx), | ||||||
|  |      allocator_(ObModIds::TABLE_PROC), | ||||||
|  |      get_ctx_(allocator_), | ||||||
|  |      need_rollback_trans_(false), | ||||||
|  |      query_timeout_ts_(0) | ||||||
|  | { | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::deserialize() | ||||||
|  | { | ||||||
|  |   // we should set entity before deserialize | ||||||
|  |   arg_.table_operation_.set_entity(request_entity_); | ||||||
|  |   result_.set_entity(result_entity_); | ||||||
|  |   int ret = ParentType::deserialize(); | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::check_arg() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (arg_.consistency_level_ != ObTableConsistencyLevel::STRONG) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("some options not supported yet", K(ret), | ||||||
|  |              "consistency_level", arg_.consistency_level_, | ||||||
|  |              "operation_type", arg_.table_operation_.type()); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::check_arg2() const | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (arg_.returning_rowkey_ | ||||||
|  |       || arg_.returning_affected_entity_) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("some options not supported yet", K(ret), | ||||||
|  |              "returning_rowkey", arg_.returning_rowkey_, | ||||||
|  |              "returning_affected_entity", arg_.returning_affected_entity_, | ||||||
|  |              "operation_type", arg_.table_operation_.type()); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::process() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   ret = ParentType::process(); | ||||||
|  |   int tmp_ret = revert_get_ctx(); | ||||||
|  |   if (OB_SUCCESS != tmp_ret) { | ||||||
|  |     LOG_WARN("fail to revert get ctx", K(tmp_ret)); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::try_process() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   const ObTableOperation &table_operation = arg_.table_operation_; | ||||||
|  |   switch (table_operation.type()) { | ||||||
|  |     case ObTableOperationType::INSERT: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_INSERT; | ||||||
|  |       ret = process_insert(); | ||||||
|  |       break; | ||||||
|  |     case ObTableOperationType::GET: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_GET; | ||||||
|  |       ret = process_get(); | ||||||
|  |       break; | ||||||
|  |     case ObTableOperationType::DEL: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_DELETE; | ||||||
|  |       ret = process_del(); | ||||||
|  |       break; | ||||||
|  |     case ObTableOperationType::UPDATE: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_UPDATE; | ||||||
|  |       ret = process_update(); | ||||||
|  |       break; | ||||||
|  |     case ObTableOperationType::INSERT_OR_UPDATE: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_INSERT_OR_UPDATE; | ||||||
|  |       ret = process_insert_or_update(); | ||||||
|  |       break; | ||||||
|  |     case ObTableOperationType::REPLACE: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_REPLACE; | ||||||
|  |       ret = process_replace(); | ||||||
|  |       break; | ||||||
|  |     case ObTableOperationType::INCREMENT: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_INCREMENT; | ||||||
|  |       ret = process_increment(); | ||||||
|  |       break; | ||||||
|  |     case ObTableOperationType::APPEND: | ||||||
|  |       stat_event_type_ = ObTableProccessType::TABLE_API_SINGLE_APPEND; | ||||||
|  |       // for both increment and append | ||||||
|  |       ret = process_increment(); | ||||||
|  |       break; | ||||||
|  |     default: | ||||||
|  |       ret = OB_INVALID_ARGUMENT; | ||||||
|  |       LOG_WARN("invalid table operation type", K(ret), K(table_operation)); | ||||||
|  |       break; | ||||||
|  |   } | ||||||
|  |   audit_row_count_ = 1; | ||||||
|  |  | ||||||
|  | #ifndef NDEBUG | ||||||
|  |   // debug mode | ||||||
|  |   LOG_INFO("[TABLE] execute operation", K(ret), K_(arg), K_(result), "timeout", rpc_pkt_->get_timeout(), K_(retry_count)); | ||||||
|  | #else | ||||||
|  |   // release mode | ||||||
|  |   LOG_TRACE("[TABLE] execute operation", K(ret), K_(arg), K_(result), | ||||||
|  |             "timeout", rpc_pkt_->get_timeout(), "receive_ts", get_receive_timestamp(), K_(retry_count)); | ||||||
|  | #endif | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::revert_get_ctx() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (ObTableOperationType::GET == arg_.table_operation_.type()) { | ||||||
|  |     if (NULL != get_ctx_.scan_result_) { | ||||||
|  |       part_service_->revert_scan_iter(get_ctx_.scan_result_); | ||||||
|  |       get_ctx_.scan_result_ = NULL; | ||||||
|  |     } | ||||||
|  |     if (query_timeout_ts_ <= 0) { | ||||||
|  |       // for robust purpose | ||||||
|  |       query_timeout_ts_ = ObTimeUtility::current_time() + 1000000; | ||||||
|  |     } | ||||||
|  |     if (OB_FAIL(end_trans(need_rollback_trans_, req_, query_timeout_ts_))) { | ||||||
|  |       LOG_WARN("failed to end trans", K(ret)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableApiExecuteP::audit_on_finish() | ||||||
|  | { | ||||||
|  |   audit_record_.consistency_level_ = ObTableConsistencyLevel::STRONG == arg_.consistency_level_ ? | ||||||
|  |       ObConsistencyLevel::STRONG : ObConsistencyLevel::WEAK; | ||||||
|  |   audit_record_.return_rows_ = arg_.returning_affected_rows_ ? 1 : 0; | ||||||
|  |   audit_record_.table_scan_ = false; | ||||||
|  |   audit_record_.affected_rows_ = result_.get_affected_rows(); | ||||||
|  |   audit_record_.try_cnt_ = retry_count_ + 1; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | uint64_t ObTableApiExecuteP::get_request_checksum() | ||||||
|  | { | ||||||
|  |   uint64_t checksum = 0; | ||||||
|  |   checksum = ob_crc64(checksum, arg_.table_name_.ptr(), arg_.table_name_.length()); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.consistency_level_, sizeof(arg_.consistency_level_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.returning_rowkey_, sizeof(arg_.returning_rowkey_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.returning_affected_entity_, sizeof(arg_.returning_affected_entity_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.returning_affected_rows_, sizeof(arg_.returning_affected_rows_)); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.binlog_row_image_type_, sizeof(arg_.binlog_row_image_type_)); | ||||||
|  |   const uint64_t op_checksum = arg_.table_operation_.get_checksum(); | ||||||
|  |   checksum = ob_crc64(checksum, &op_checksum, sizeof(op_checksum)); | ||||||
|  |   return checksum; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::response(const int retcode) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (!need_retry_in_queue_ && !did_async_end_trans()) { | ||||||
|  |     ret = ObRpcProcessor::response(retcode); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableApiExecuteP::reset_ctx() | ||||||
|  | { | ||||||
|  |   (void)revert_get_ctx(); | ||||||
|  |   get_ctx_.reset_dml(); | ||||||
|  |   ObTableApiProcessorBase::reset_ctx(); | ||||||
|  |   need_rollback_trans_ = false; | ||||||
|  |   need_retry_in_queue_ = false; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::get_partition_id(uint64_t table_id, const ObRowkey &rowkey, uint64_t &partition_id) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   partition_id = arg_.partition_id_; | ||||||
|  |   if (OB_INVALID_ID == partition_id) { | ||||||
|  |     ObSEArray<ObRowkey, 1> rowkeys; | ||||||
|  |     ObSEArray<int64_t, 1> part_ids; | ||||||
|  |     ObSEArray<sql::RowkeyArray, 1> rowkeys_per_part; | ||||||
|  |     if (OB_FAIL(rowkeys.push_back(rowkey))) { | ||||||
|  |       LOG_WARN("failed to push back", K(ret)); | ||||||
|  |     } else if (OB_FAIL(get_partition_by_rowkey(table_id, rowkeys, part_ids, rowkeys_per_part))) { | ||||||
|  |       LOG_WARN("failed to get partition", K(ret), K(table_id), K(rowkeys)); | ||||||
|  |     } else if (1 != part_ids.count()) { | ||||||
|  |       ret = OB_ERR_UNEXPECTED; | ||||||
|  |       LOG_ERROR("should have one partition", K(ret)); | ||||||
|  |     } else { | ||||||
|  |       partition_id = part_ids.at(0); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | // get | ||||||
|  | int ObTableApiExecuteP::process_get() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   need_rollback_trans_ = false; | ||||||
|  |   uint64_t &table_id = get_ctx_.param_table_id(); | ||||||
|  |   get_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                       arg_.returning_affected_rows_, | ||||||
|  |                       arg_.entity_type_, | ||||||
|  |                       arg_.binlog_row_image_type_); | ||||||
|  |   const bool is_readonly = true; | ||||||
|  |   ObRowkey rowkey = const_cast<ObITableEntity&>(arg_.table_operation_.entity()).get_rowkey(); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret), K(table_id)); | ||||||
|  |   } else if (OB_FAIL(get_partition_id(table_id, rowkey, get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to get partition id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(part_ids.push_back(get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to push back", K(ret)); | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_SELECT, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start readonly transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_get(get_ctx_, arg_.table_operation_, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to execute get", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } else {} | ||||||
|  |   // end trans in after_process() | ||||||
|  |   need_rollback_trans_ = (OB_SUCCESS != ret); | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | // insert_or_update | ||||||
|  | ObTableAPITransCb *ObTableApiExecuteP::new_callback(rpc::ObRequest *req) | ||||||
|  | { | ||||||
|  |   ObTableExecuteEndTransCb *cb = OB_NEW(ObTableExecuteEndTransCb, ObModIds::TABLE_PROC, req, arg_.table_operation_.type()); | ||||||
|  |   if (NULL != cb) { | ||||||
|  |     // @todo optimize to avoid this copy | ||||||
|  |     int ret = OB_SUCCESS; | ||||||
|  |     if (OB_FAIL(cb->assign_execute_result(result_))) { | ||||||
|  |       LOG_WARN("failed to assign result", K(ret)); | ||||||
|  |       cb->~ObTableExecuteEndTransCb(); | ||||||
|  |       cb = NULL; | ||||||
|  |     } else { | ||||||
|  |       LOG_DEBUG("yzfdebug copy result", K_(result)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return cb; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::process_insert_or_update() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   uint64_t &table_id = get_ctx_.param_table_id(); | ||||||
|  |   get_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                       arg_.returning_affected_rows_, | ||||||
|  |                       arg_.entity_type_, | ||||||
|  |                       arg_.binlog_row_image_type_); | ||||||
|  |  | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   ObRowkey rowkey = const_cast<ObITableEntity&>(arg_.table_operation_.entity()).get_rowkey(); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret), K(table_id)); | ||||||
|  |   } else if (OB_FAIL(get_partition_id(table_id, rowkey, get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to get partition id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(part_ids.push_back(get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to push back", K(ret)); | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_INSERT, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_insert_or_update(get_ctx_, arg_.table_operation_, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to insert_or_update", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret)); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::process_del() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   uint64_t &table_id = get_ctx_.param_table_id(); | ||||||
|  |   get_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                       arg_.returning_affected_rows_, | ||||||
|  |                       arg_.entity_type_, | ||||||
|  |                       arg_.binlog_row_image_type_); | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   ObRowkey rowkey = const_cast<ObITableEntity&>(arg_.table_operation_.entity()).get_rowkey(); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret), K(table_id)); | ||||||
|  |   } else if (OB_FAIL(get_partition_id(table_id, rowkey, get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to get partition id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(part_ids.push_back(get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to push back", K(ret)); | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_DELETE, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_delete(get_ctx_, arg_.table_operation_, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to delete", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret)); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::process_replace() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   uint64_t &table_id = get_ctx_.param_table_id(); | ||||||
|  |   get_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                       arg_.returning_affected_rows_, | ||||||
|  |                       arg_.entity_type_, | ||||||
|  |                       arg_.binlog_row_image_type_); | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   ObRowkey rowkey = const_cast<ObITableEntity&>(arg_.table_operation_.entity()).get_rowkey(); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret), K(table_id)); | ||||||
|  |   } else if (OB_FAIL(get_partition_id(table_id, rowkey, get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to get partition id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(part_ids.push_back(get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to push back", K(ret)); | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_REPLACE, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_replace(get_ctx_, arg_.table_operation_, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to replace", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret)); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::process_insert() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   ObNewRowIterator *duplicate_row_iter = nullptr; | ||||||
|  |   uint64_t &table_id = get_ctx_.param_table_id(); | ||||||
|  |   get_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                       arg_.returning_affected_rows_, | ||||||
|  |                       arg_.entity_type_, | ||||||
|  |                       arg_.binlog_row_image_type_); | ||||||
|  |  | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   ObRowkey rowkey = const_cast<ObITableEntity&>(arg_.table_operation_.entity()).get_rowkey(); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret), K(table_id)); | ||||||
|  |   } else if (OB_FAIL(get_partition_id(table_id, rowkey, get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to get partition id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(part_ids.push_back(get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to push back", K(ret)); | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_INSERT, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_insert(get_ctx_, | ||||||
|  |       arg_.table_operation_, result_, duplicate_row_iter))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to insert", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   const bool did_rollback = (OB_SUCCESS != ret || OB_SUCCESS != result_.get_errno()); | ||||||
|  |   if (OB_FAIL(end_trans(did_rollback, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret)); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::process_update() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   uint64_t &table_id = get_ctx_.param_table_id(); | ||||||
|  |   get_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                       arg_.returning_affected_rows_, | ||||||
|  |                       arg_.entity_type_, | ||||||
|  |                       arg_.binlog_row_image_type_); | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   ObRowkey rowkey = const_cast<ObITableEntity&>(arg_.table_operation_.entity()).get_rowkey(); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(check_arg2())) { | ||||||
|  |   } else if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret), K(table_id)); | ||||||
|  |   } else if (OB_FAIL(get_partition_id(table_id, rowkey, get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to get partition id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(part_ids.push_back(get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to push back", K(ret)); | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_UPDATE, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_update(get_ctx_, arg_.table_operation_, nullptr, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to update", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret)); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableApiExecuteP::process_increment() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   uint64_t &table_id = get_ctx_.param_table_id(); | ||||||
|  |   get_ctx_.init_param(get_timeout_ts(), this, &allocator_, | ||||||
|  |                       arg_.returning_affected_rows_, | ||||||
|  |                       arg_.entity_type_, | ||||||
|  |                       arg_.binlog_row_image_type_, | ||||||
|  |                       arg_.returning_affected_entity_, | ||||||
|  |                       arg_.returning_rowkey_); | ||||||
|  |   const bool is_readonly = false; | ||||||
|  |   ObRowkey rowkey = const_cast<ObITableEntity&>(arg_.table_operation_.entity()).get_rowkey(); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret), K(table_id)); | ||||||
|  |   } else if (OB_FAIL(get_partition_id(table_id, rowkey, get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to get partition id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(part_ids.push_back(get_ctx_.param_partition_id()))) { | ||||||
|  |     LOG_WARN("failed to push back", K(ret)); | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_UPDATE, table_id, part_ids, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to start transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_increment(get_ctx_, arg_.table_operation_, result_))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to update", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(OB_SUCCESS != ret, req_, get_timeout_ts()))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret)); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
							
								
								
									
										70
									
								
								src/observer/table/ob_table_execute_processor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										70
									
								
								src/observer/table/ob_table_execute_processor.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,70 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_EXECUTE_PROCESSOR_H | ||||||
|  | #define _OB_TABLE_EXECUTE_PROCESSOR_H 1 | ||||||
|  | #include "rpc/obrpc/ob_rpc_proxy.h" | ||||||
|  | #include "rpc/obrpc/ob_rpc_processor.h" | ||||||
|  | #include "share/table/ob_table_rpc_proxy.h" | ||||||
|  | #include "ob_table_rpc_processor.h" | ||||||
|  | #include "ob_table_service.h" | ||||||
|  |  | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace observer | ||||||
|  | { | ||||||
|  | /// @see RPC_S(PR5 execute, obrpc::OB_TABLE_API_EXECUTE, (table::ObTableOperationRequest), table::ObTableOperationResult); | ||||||
|  | class ObTableApiExecuteP: public ObTableRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_EXECUTE> > | ||||||
|  | { | ||||||
|  |   typedef ObTableRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_EXECUTE> > ParentType; | ||||||
|  | public: | ||||||
|  |   explicit ObTableApiExecuteP(const ObGlobalContext &gctx); | ||||||
|  |   virtual ~ObTableApiExecuteP() = default; | ||||||
|  |  | ||||||
|  |   virtual int deserialize() override; | ||||||
|  |   virtual int process() override; | ||||||
|  |   virtual int response(const int retcode) override; | ||||||
|  | protected: | ||||||
|  |   virtual int check_arg() override; | ||||||
|  |   virtual int try_process() override; | ||||||
|  |   virtual void reset_ctx() override; | ||||||
|  |   table::ObTableAPITransCb *new_callback(rpc::ObRequest *req) override; | ||||||
|  |   virtual void audit_on_finish() override; | ||||||
|  |   virtual uint64_t get_request_checksum() override; | ||||||
|  |  | ||||||
|  | private: | ||||||
|  |   int check_arg2() const; | ||||||
|  |   int revert_get_ctx(); | ||||||
|  |   int get_partition_id(uint64_t table_id, const ObRowkey &rowkey, uint64_t &partition_id); | ||||||
|  |   int process_get(); | ||||||
|  |   int process_insert(); | ||||||
|  |   int process_del(); | ||||||
|  |   int process_update(); | ||||||
|  |   int process_insert_or_update(); | ||||||
|  |   int process_replace(); | ||||||
|  |   int process_increment(); | ||||||
|  | private: | ||||||
|  |   table::ObTableEntity request_entity_; | ||||||
|  |   table::ObTableEntity result_entity_; | ||||||
|  |   common::ObArenaAllocator allocator_; | ||||||
|  |   table::ObTableEntityFactory<table::ObTableEntity> default_entity_factory_; | ||||||
|  |   // the life of scan_ctx_ should be longer than process() | ||||||
|  |   ObTableServiceGetCtx get_ctx_; | ||||||
|  |   bool need_rollback_trans_; | ||||||
|  |   int64_t query_timeout_ts_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | } // end namespace observer | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_EXECUTE_PROCESSOR_H */ | ||||||
							
								
								
									
										201
									
								
								src/observer/table/ob_table_query_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										201
									
								
								src/observer/table/ob_table_query_processor.cpp
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,201 @@ | |||||||
|  | /** | ||||||
|  |  * 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 SERVER | ||||||
|  | #include "ob_table_query_processor.h" | ||||||
|  | #include "ob_table_rpc_processor_util.h" | ||||||
|  | #include "observer/ob_service.h" | ||||||
|  | #include "storage/ob_partition_service.h" | ||||||
|  | #include "ob_table_end_trans_cb.h" | ||||||
|  | #include "sql/optimizer/ob_table_location.h"  // ObTableLocation | ||||||
|  | #include "lib/stat/ob_diagnose_info.h" | ||||||
|  | #include "lib/stat/ob_session_stat.h" | ||||||
|  |  | ||||||
|  | using namespace oceanbase::observer; | ||||||
|  | using namespace oceanbase::common; | ||||||
|  | using namespace oceanbase::table; | ||||||
|  | using namespace oceanbase::share; | ||||||
|  | using namespace oceanbase::sql; | ||||||
|  |  | ||||||
|  | ObTableQueryP::ObTableQueryP(const ObGlobalContext &gctx) | ||||||
|  |     :ObTableRpcProcessor(gctx), | ||||||
|  |      allocator_(ObModIds::TABLE_PROC), | ||||||
|  |      table_service_ctx_(allocator_), | ||||||
|  |      result_row_count_(0) | ||||||
|  | { | ||||||
|  |   // the streaming interface may return multi packet. The memory may be freed after the first packet has been sended. | ||||||
|  |   // the deserialization of arg_ is shallow copy, so we need deep copy data to processor | ||||||
|  |   set_preserve_recv_data(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableQueryP::deserialize() | ||||||
|  | { | ||||||
|  |   arg_.query_.set_deserialize_allocator(&allocator_); | ||||||
|  |   return ParentType::deserialize(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableQueryP::check_arg() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (!arg_.query_.is_valid()) { | ||||||
|  |     ret = OB_INVALID_ARGUMENT; | ||||||
|  |     LOG_WARN("invalid table query request", K(ret), "query", arg_.query_); | ||||||
|  |   } else if (arg_.consistency_level_ != ObTableConsistencyLevel::STRONG) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("some options not supported yet", K(ret), | ||||||
|  |              "consistency_level", arg_.consistency_level_); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableQueryP::audit_on_finish() | ||||||
|  | { | ||||||
|  |   audit_record_.consistency_level_ = ObTableConsistencyLevel::STRONG == arg_.consistency_level_ ? | ||||||
|  |       ObConsistencyLevel::STRONG : ObConsistencyLevel::WEAK; | ||||||
|  |   audit_record_.return_rows_ = result_.get_row_count(); | ||||||
|  |   audit_record_.table_scan_ = true; // todo: exact judgement | ||||||
|  |   audit_record_.affected_rows_ = result_.get_row_count(); | ||||||
|  |   audit_record_.try_cnt_ = retry_count_ + 1; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | uint64_t ObTableQueryP::get_request_checksum() | ||||||
|  | { | ||||||
|  |   uint64_t checksum = 0; | ||||||
|  |   checksum = ob_crc64(checksum, arg_.table_name_.ptr(), arg_.table_name_.length()); | ||||||
|  |   checksum = ob_crc64(checksum, &arg_.consistency_level_, sizeof(arg_.consistency_level_)); | ||||||
|  |   const uint64_t op_checksum = arg_.query_.get_checksum(); | ||||||
|  |   checksum = ob_crc64(checksum, &op_checksum, sizeof(op_checksum)); | ||||||
|  |   return checksum; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void ObTableQueryP::reset_ctx() | ||||||
|  | { | ||||||
|  |   table_service_ctx_.reset_query_ctx(part_service_); | ||||||
|  |   need_retry_in_queue_ = false; | ||||||
|  |   result_row_count_ = 0; | ||||||
|  |   ObTableApiProcessorBase::reset_ctx(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | ObTableAPITransCb *ObTableQueryP::new_callback(rpc::ObRequest *req) | ||||||
|  | { | ||||||
|  |   UNUSED(req); | ||||||
|  |   return nullptr; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableQueryP::get_partition_ids(uint64_t table_id, ObIArray<int64_t> &part_ids) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   uint64_t partition_id = arg_.partition_id_; | ||||||
|  |   if (OB_INVALID_ID == partition_id) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("partitioned table not supported", K(ret), K(table_id)); | ||||||
|  |   } else { | ||||||
|  |     if (OB_FAIL(part_ids.push_back(partition_id))) { | ||||||
|  |       LOG_WARN("failed to push back", K(ret)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int ObTableQueryP::try_process() | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   int64_t rpc_timeout = 0; | ||||||
|  |   if (NULL != rpc_pkt_) { | ||||||
|  |     rpc_timeout = rpc_pkt_->get_timeout(); | ||||||
|  |   } | ||||||
|  |   const int64_t timeout_ts = get_timeout_ts(); | ||||||
|  |   uint64_t &table_id = table_service_ctx_.param_table_id(); | ||||||
|  |   table_service_ctx_.init_param(timeout_ts, this, &allocator_, | ||||||
|  |                                 false/*ignored*/, | ||||||
|  |                                 arg_.entity_type_, | ||||||
|  |                                 table::ObBinlogRowImageType::MINIMAL/*ignored*/); | ||||||
|  |   ObSEArray<int64_t, 1> part_ids; | ||||||
|  |   const bool is_readonly = true; | ||||||
|  |   ObTableQueryResultIterator *result_iterator = nullptr; | ||||||
|  |   int32_t result_count = 0; | ||||||
|  |   if (OB_FAIL(get_table_id(arg_.table_name_, arg_.table_id_, table_id))) { | ||||||
|  |     LOG_WARN("failed to get table id", K(ret)); | ||||||
|  |   } else if (OB_FAIL(get_partition_ids(table_id, part_ids))) { | ||||||
|  |     LOG_WARN("failed to get part id", K(ret)); | ||||||
|  |   } else if (1 != part_ids.count()) { | ||||||
|  |     ret = OB_NOT_SUPPORTED; | ||||||
|  |     LOG_WARN("should have one partition", K(ret), K(part_ids)); | ||||||
|  |   } else if (FALSE_IT(table_service_ctx_.param_partition_id() = part_ids.at(0))) { | ||||||
|  |   } else if (OB_FAIL(start_trans(is_readonly, sql::stmt::T_SELECT, table_id, part_ids, timeout_ts))) { | ||||||
|  |     LOG_WARN("failed to start readonly transaction", K(ret)); | ||||||
|  |   } else if (OB_FAIL(table_service_->execute_query(table_service_ctx_, arg_.query_, | ||||||
|  |                                                    result_, result_iterator))) { | ||||||
|  |     if (OB_TRY_LOCK_ROW_CONFLICT != ret) { | ||||||
|  |       LOG_WARN("failed to execute query", K(ret), K(table_id)); | ||||||
|  |     } | ||||||
|  |   } else { | ||||||
|  |     // one_result references to result_ | ||||||
|  |     ObTableQueryResult *one_result = nullptr; | ||||||
|  |     while (OB_SUCC(ret)) { | ||||||
|  |       ++result_count; | ||||||
|  |       // the last result_ does not need flush, it will be send automatically | ||||||
|  |       if (ObTimeUtility::current_time() > timeout_ts) { | ||||||
|  |         ret = OB_TRANS_TIMEOUT; | ||||||
|  |         LOG_WARN("exceed operatiton timeout", K(ret)); | ||||||
|  |       } else if (OB_FAIL(result_iterator->get_next_result(one_result))) { | ||||||
|  |         if (OB_ITER_END != ret) { | ||||||
|  |           LOG_WARN("fail to get next result", K(ret)); | ||||||
|  |         } | ||||||
|  |       } else if (result_iterator->has_more_result()) { | ||||||
|  |         if (OB_FAIL(this->flush())) { | ||||||
|  |           if (OB_ITER_END != ret) { | ||||||
|  |             LOG_WARN("failed to flush result packet", K(ret)); | ||||||
|  |           } else { | ||||||
|  |             LOG_TRACE("user abort the stream rpc", K(ret)); | ||||||
|  |           } | ||||||
|  |         } else { | ||||||
|  |           LOG_DEBUG("[yzfdebug] flush one result", K(ret), "row_count", result_.get_row_count()); | ||||||
|  |           result_row_count_ += result_.get_row_count(); | ||||||
|  |           result_.reset_except_property(); | ||||||
|  |         } | ||||||
|  |       } else { | ||||||
|  |         // no more result | ||||||
|  |         result_row_count_ += result_.get_row_count(); | ||||||
|  |         break; | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |     if (OB_ITER_END == ret) { | ||||||
|  |       ret = OB_SUCCESS; | ||||||
|  |     } | ||||||
|  |     LOG_DEBUG("[yzfdebug] last result", K(ret), "row_count", result_.get_row_count()); | ||||||
|  |     NG_TRACE_EXT(tag1, OB_ID(return_rows), result_count, OB_ID(arg2), result_row_count_); | ||||||
|  |   } | ||||||
|  |   table_service_ctx_.destroy_result_iterator(part_service_); | ||||||
|  |   bool need_rollback_trans = (OB_SUCCESS != ret); | ||||||
|  |   int tmp_ret = ret; | ||||||
|  |   if (OB_FAIL(end_trans(need_rollback_trans, req_, timeout_ts))) { | ||||||
|  |     LOG_WARN("failed to end trans", K(ret), "rollback", need_rollback_trans); | ||||||
|  |   } | ||||||
|  |   ret = (OB_SUCCESS == tmp_ret) ? ret : tmp_ret; | ||||||
|  |  | ||||||
|  |   // record events | ||||||
|  |   stat_event_type_ = ObTableProccessType::TABLE_API_TABLE_QUERY;// table query | ||||||
|  |    | ||||||
|  |   audit_row_count_ = result_row_count_; | ||||||
|  |  | ||||||
|  | #ifndef NDEBUG | ||||||
|  |   // debug mode | ||||||
|  |   LOG_INFO("[TABLE] execute query", K(ret), K_(arg), K(rpc_timeout), | ||||||
|  |            K_(retry_count), K(result_count), K_(result_row_count)); | ||||||
|  | #else | ||||||
|  |   // release mode | ||||||
|  |   LOG_TRACE("[TABLE] execute query", K(ret), K_(arg), K(rpc_timeout), K_(retry_count), | ||||||
|  |             "receive_ts", get_receive_timestamp(), K(result_count), K_(result_row_count)); | ||||||
|  | #endif | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
							
								
								
									
										53
									
								
								src/observer/table/ob_table_query_processor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										53
									
								
								src/observer/table/ob_table_query_processor.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,53 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_QUERY_PROCESSOR_H | ||||||
|  | #define _OB_TABLE_QUERY_PROCESSOR_H 1 | ||||||
|  | #include "rpc/obrpc/ob_rpc_proxy.h" | ||||||
|  | #include "rpc/obrpc/ob_rpc_processor.h" | ||||||
|  | #include "share/table/ob_table_rpc_proxy.h" | ||||||
|  | #include "ob_table_rpc_processor.h" | ||||||
|  | #include "ob_table_service.h" | ||||||
|  |  | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace observer | ||||||
|  | { | ||||||
|  | class ObTableQueryP: public ObTableRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_EXECUTE_QUERY> > | ||||||
|  | { | ||||||
|  |   typedef ObTableRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_EXECUTE_QUERY> > ParentType; | ||||||
|  | public: | ||||||
|  |   explicit ObTableQueryP(const ObGlobalContext &gctx); | ||||||
|  |   virtual ~ObTableQueryP() {} | ||||||
|  |  | ||||||
|  |   virtual int deserialize() override; | ||||||
|  | protected: | ||||||
|  |   virtual int check_arg() override; | ||||||
|  |   virtual int try_process() override; | ||||||
|  |   virtual void reset_ctx() override; | ||||||
|  |   virtual table::ObTableAPITransCb *new_callback(rpc::ObRequest *req) override; | ||||||
|  |   virtual void audit_on_finish() override; | ||||||
|  |   virtual uint64_t get_request_checksum() override; | ||||||
|  |  | ||||||
|  | private: | ||||||
|  |   int get_partition_ids(uint64_t table_id, common::ObIArray<int64_t> &part_ids); | ||||||
|  |   DISALLOW_COPY_AND_ASSIGN(ObTableQueryP); | ||||||
|  | private: | ||||||
|  |   common::ObArenaAllocator allocator_; | ||||||
|  |   ObTableServiceQueryCtx table_service_ctx_; | ||||||
|  |   int64_t result_row_count_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | } // end namespace observer | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_QUERY_PROCESSOR_H */ | ||||||
							
								
								
									
										1014
									
								
								src/observer/table/ob_table_rpc_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1014
									
								
								src/observer/table/ob_table_rpc_processor.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										220
									
								
								src/observer/table/ob_table_rpc_processor.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										220
									
								
								src/observer/table/ob_table_rpc_processor.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,220 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_RPC_PROCESSOR_H | ||||||
|  | #define _OB_TABLE_RPC_PROCESSOR_H 1 | ||||||
|  |  | ||||||
|  | #include "rpc/obrpc/ob_rpc_proxy.h" | ||||||
|  | #include "rpc/obrpc/ob_rpc_processor.h" | ||||||
|  | #include "share/table/ob_table_rpc_proxy.h" | ||||||
|  | #include "sql/ob_sql_trans_control.h" | ||||||
|  | #include "sql/optimizer/ob_table_location.h"  // ObTableLocation | ||||||
|  | #include "ob_table_service.h" | ||||||
|  | #include "sql/monitor/ob_exec_stat.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace table | ||||||
|  | { | ||||||
|  | class ObTableAPITransCb; | ||||||
|  | } // end namespace table | ||||||
|  | namespace observer | ||||||
|  | { | ||||||
|  | class ObGlobalContext; | ||||||
|  | class ObTableService; | ||||||
|  |  | ||||||
|  | struct ObTableApiCredential final | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableApiCredential(); | ||||||
|  |   ~ObTableApiCredential(); | ||||||
|  | public: | ||||||
|  |   int64_t cluster_id_; | ||||||
|  |   uint64_t tenant_id_; | ||||||
|  |   uint64_t user_id_; | ||||||
|  |   uint64_t database_id_; | ||||||
|  |   int64_t expire_ts_; | ||||||
|  |   uint64_t hash_val_; | ||||||
|  | public: | ||||||
|  |   uint64_t hash(uint64_t seed = 0) const; | ||||||
|  |   TO_STRING_KV(K_(cluster_id), | ||||||
|  |                K_(tenant_id), | ||||||
|  |                K_(user_id), | ||||||
|  |                K_(database_id), | ||||||
|  |                K_(expire_ts), | ||||||
|  |                K_(hash_val)); | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// @see RPC_S(PR5 login, obrpc::OB_TABLE_API_LOGIN, (table::ObTableLoginRequest), table::ObTableLoginResult); | ||||||
|  | class ObTableLoginP: public obrpc::ObRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_LOGIN> > | ||||||
|  | { | ||||||
|  |   typedef obrpc::ObRpcProcessor<obrpc::ObTableRpcProxy::ObRpc<obrpc::OB_TABLE_API_LOGIN> > ParentType; | ||||||
|  | public: | ||||||
|  |   explicit ObTableLoginP(const ObGlobalContext &gctx) | ||||||
|  |       :gctx_(gctx) | ||||||
|  |   {} | ||||||
|  |   virtual ~ObTableLoginP() = default; | ||||||
|  |   virtual int process() override; | ||||||
|  | private: | ||||||
|  |   int get_ids(); | ||||||
|  |   int verify_password(const ObString &tenant, const ObString &user, const ObString &pass_secret, | ||||||
|  |                       const ObString &pass_scramble, const ObString &database, uint64_t &user_token); | ||||||
|  |   int generate_credential(uint64_t tenant_id, uint64_t user_id, uint64_t database, | ||||||
|  |                           int64_t ttl_us, uint64_t user_token, ObString &credential); | ||||||
|  | private: | ||||||
|  |   static const int64_t CREDENTIAL_BUF_SIZE = 256; | ||||||
|  | private: | ||||||
|  |   const ObGlobalContext &gctx_; | ||||||
|  |   char credential_buf_[CREDENTIAL_BUF_SIZE]; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObTableRetryPolicy | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableRetryPolicy() | ||||||
|  |   : allow_retry_(true), | ||||||
|  |     allow_rpc_retry_(true), | ||||||
|  |     local_retry_interval_us_(10), | ||||||
|  |     max_local_retry_count_(5) | ||||||
|  |   {} | ||||||
|  |   virtual ~ObTableRetryPolicy() {} | ||||||
|  |   bool allow_retry() const { return allow_retry_; } | ||||||
|  |   // rpc retry will receate the processor, | ||||||
|  |   // so there is no retry count limit for now. | ||||||
|  |   bool allow_rpc_retry() const { return allow_retry_ && allow_rpc_retry_; } | ||||||
|  | public: | ||||||
|  |   bool allow_retry_; | ||||||
|  |   bool allow_rpc_retry_; | ||||||
|  |   int64_t local_retry_interval_us_; | ||||||
|  |   int64_t max_local_retry_count_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /* | ||||||
|  |  * Normally, the rpc process flow is: | ||||||
|  |  * 1. deserialize | ||||||
|  |  * 2. before_process | ||||||
|  |  * 3. process | ||||||
|  |  * 4. before_response | ||||||
|  |  * 5. response | ||||||
|  |  * 6. after_process | ||||||
|  |  * 7. cleanup | ||||||
|  |  * | ||||||
|  |  * Attention: | ||||||
|  |  * After response or async_commit_trans, | ||||||
|  |  * all buffer related to the request (such as req_) may recycled by the network frame. | ||||||
|  |  * DO NOT access these memory in after_process() and cleanup(). | ||||||
|  |  */ | ||||||
|  |  | ||||||
|  |  | ||||||
|  | /// Base class of all table api processor | ||||||
|  | class ObTableApiProcessorBase | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   explicit ObTableApiProcessorBase(const ObGlobalContext &gctx); | ||||||
|  |   virtual ~ObTableApiProcessorBase() = default; | ||||||
|  | public: | ||||||
|  |   static int init_session(); | ||||||
|  |   int check_user_access(const ObString &credential_str); | ||||||
|  |   //@{  transaction control | ||||||
|  |   int start_trans(bool is_readonly, const sql::stmt::StmtType stmt_type, uint64_t table_id, | ||||||
|  |                   const common::ObIArray<int64_t> &part_ids, int64_t timeout_ts); | ||||||
|  |   int end_trans(bool is_rollback, rpc::ObRequest *req, int64_t timeout_ts, bool use_sync = false); | ||||||
|  |   inline bool did_async_end_trans() const { return did_async_end_trans_; } | ||||||
|  |   inline transaction::ObTransDesc& get_trans_desc() { return trans_desc_; } | ||||||
|  |   int get_partition_by_rowkey(uint64_t table_id, const ObIArray<common::ObRowkey> &rowkeys, | ||||||
|  |                               common::ObIArray<int64_t> &part_ids, | ||||||
|  |                               common::ObIArray<sql::RowkeyArray> &rowkeys_per_part); | ||||||
|  |  | ||||||
|  |   int get_table_id(const ObString &table_name, const uint64_t arg_table_id, uint64_t &real_table_id) const; | ||||||
|  | protected: | ||||||
|  |   virtual int check_arg() = 0; | ||||||
|  |   virtual int try_process() = 0; | ||||||
|  |   virtual table::ObTableAPITransCb *new_callback(rpc::ObRequest *req) = 0; | ||||||
|  |   virtual void set_req_has_wokenup() = 0; | ||||||
|  |   virtual void reset_ctx(); | ||||||
|  |   int process_with_retry(const ObString &credential, const int64_t timeout_ts); | ||||||
|  |  | ||||||
|  |   // audit | ||||||
|  |   bool need_audit() const; | ||||||
|  |   void start_audit(const rpc::ObRequest *req); | ||||||
|  |   void end_audit(); | ||||||
|  |   virtual void audit_on_finish() {} | ||||||
|  |   virtual void save_request_string() = 0; | ||||||
|  |   virtual void generate_sql_id() = 0; | ||||||
|  |  | ||||||
|  | private: | ||||||
|  |   int get_participants(uint64_t table_id, const common::ObIArray<int64_t> &part_ids, | ||||||
|  |                        common::ObPartitionLeaderArray &partition_leaders); | ||||||
|  |   int get_participants_from_lc(uint64_t table_id, const common::ObIArray<int64_t> &part_ids, | ||||||
|  |                                common::ObPartitionLeaderArray &partition_leaders); | ||||||
|  |   int get_participants_optimistic(uint64_t table_id, const common::ObIArray<int64_t> &part_ids, | ||||||
|  |                                   common::ObPartitionLeaderArray &partition_leaders); | ||||||
|  |  | ||||||
|  |   int async_commit_trans(rpc::ObRequest *req, int64_t timeout_ts); | ||||||
|  |   int sync_end_trans(bool is_rollback, int64_t timeout_ts); | ||||||
|  |   int generate_schema_info_arr(const uint64_t table_id, | ||||||
|  |                                const common::ObPartitionArray &participants, | ||||||
|  |                                transaction::ObPartitionSchemaInfoArray &schema_info_arr); | ||||||
|  |   //@} | ||||||
|  | protected: | ||||||
|  |   const ObGlobalContext &gctx_; | ||||||
|  |   storage::ObPartitionService *part_service_; | ||||||
|  |   ObTableService *table_service_; | ||||||
|  |   ObTableApiCredential credential_; | ||||||
|  |   int32_t stat_event_type_; | ||||||
|  |   int64_t audit_row_count_; | ||||||
|  |   bool need_audit_; | ||||||
|  |   const char *request_string_; | ||||||
|  |   int64_t request_string_len_; | ||||||
|  |   sql::ObAuditRecordData audit_record_; | ||||||
|  |   ObArenaAllocator audit_allocator_; | ||||||
|  |   ObTableRetryPolicy retry_policy_; | ||||||
|  |   bool need_retry_in_queue_; | ||||||
|  |   int32_t retry_count_; | ||||||
|  | private: | ||||||
|  |   // trans control | ||||||
|  |   ObPartitionLeaderArray participants_; | ||||||
|  |   sql::TransState trans_state_; | ||||||
|  |   transaction::ObTransDesc trans_desc_; | ||||||
|  |   //part_epoch_list_ record the epoch id of response_partitions_ | ||||||
|  |   //when start_participants executed in the leader replica | ||||||
|  |   transaction::ObPartitionEpochArray part_epoch_list_; | ||||||
|  |   bool did_async_end_trans_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | template<class T> | ||||||
|  | class ObTableRpcProcessor: public obrpc::ObRpcProcessor<T>, public ObTableApiProcessorBase | ||||||
|  | { | ||||||
|  |   typedef obrpc::ObRpcProcessor<T> RpcProcessor; | ||||||
|  | public: | ||||||
|  |   explicit ObTableRpcProcessor(const ObGlobalContext &gctx) : ObTableApiProcessorBase(gctx) {} | ||||||
|  |   virtual ~ObTableRpcProcessor() = default; | ||||||
|  |   virtual int deserialize() override; | ||||||
|  |   virtual int before_process() override; | ||||||
|  |   virtual int process() override; | ||||||
|  |   virtual int before_response() override; | ||||||
|  |   virtual int response(const int retcode) override; | ||||||
|  |   virtual int after_process() override; | ||||||
|  |  | ||||||
|  | protected: | ||||||
|  |   virtual void set_req_has_wokenup() override; | ||||||
|  |   int64_t get_timeout_ts() const; | ||||||
|  |   virtual void save_request_string() override; | ||||||
|  |   virtual void generate_sql_id() override; | ||||||
|  |   virtual uint64_t get_request_checksum() = 0; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | } // end namespace observer | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_RPC_PROCESSOR_H */ | ||||||
							
								
								
									
										202
									
								
								src/observer/table/ob_table_rpc_processor_util.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										202
									
								
								src/observer/table/ob_table_rpc_processor_util.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,202 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_RPC_PROCESSOR_UTIL_H | ||||||
|  | #define _OB_TABLE_RPC_PROCESSOR_UTIL_H 1 | ||||||
|  | #include "lib/stat/ob_diagnose_info.h" | ||||||
|  | #include "share/table/ob_table.h" | ||||||
|  | #include "sql/monitor/ob_exec_stat.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace observer | ||||||
|  | { | ||||||
|  |  | ||||||
|  | enum ObTableProccessType | ||||||
|  | { | ||||||
|  |   TABLE_API_PROCESS_TYPE_INVALID = 0, | ||||||
|  |   // table single mutate | ||||||
|  |   TABLE_API_SINGLE_INSERT, | ||||||
|  |   TABLE_API_SINGLE_GET, | ||||||
|  |   TABLE_API_SINGLE_DELETE, | ||||||
|  |   TABLE_API_SINGLE_UPDATE, | ||||||
|  |   TABLE_API_SINGLE_INSERT_OR_UPDATE, | ||||||
|  |   TABLE_API_SINGLE_REPLACE, | ||||||
|  |   TABLE_API_SINGLE_INCREMENT, | ||||||
|  |   TABLE_API_SINGLE_APPEND, | ||||||
|  |  | ||||||
|  |   // table batch mutate | ||||||
|  |   TABLE_API_MULTI_INSERT, | ||||||
|  |   TABLE_API_MULTI_GET, | ||||||
|  |   TABLE_API_MULTI_DELETE, | ||||||
|  |   TABLE_API_MULTI_UPDATE, | ||||||
|  |   TABLE_API_MULTI_INSERT_OR_UPDATE, | ||||||
|  |   TABLE_API_MULTI_REPLACE, | ||||||
|  |   TABLE_API_MULTI_INCREMENT, | ||||||
|  |   TABLE_API_MULTI_APPEND, | ||||||
|  |   TABLE_API_BATCH_RETRIVE, | ||||||
|  |   TABLE_API_BATCH_HYBRID, | ||||||
|  |  | ||||||
|  |   // query | ||||||
|  |   TABLE_API_TABLE_QUERY, | ||||||
|  |   TABLE_API_HBASE_QUERY, | ||||||
|  |  | ||||||
|  |   TABLE_API_PROCESS_TYPE_MAX | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | #define SET_AUDIT_SQL_STRING(op_type) \ | ||||||
|  | static const char op_type##_name[] = "table api: " #op_type; \ | ||||||
|  | audit_record.sql_ = const_cast<char *>(op_type##_name); \ | ||||||
|  | audit_record.sql_len_ = sizeof(op_type##_name) | ||||||
|  |  | ||||||
|  | class ObTableRpcProcessorUtil | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |  | ||||||
|  |   OB_INLINE static void record_stat( | ||||||
|  |       sql::ObAuditRecordData &audit_record, | ||||||
|  |       const int32_t process_type, | ||||||
|  |       int64_t elapsed_us, | ||||||
|  |       int64_t rows) | ||||||
|  |   { | ||||||
|  |     switch (process_type) { | ||||||
|  |       // table single mutate | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_INSERT: | ||||||
|  |       EVENT_INC(TABLEAPI_INSERT_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_INSERT_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_insert); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_GET: | ||||||
|  |       EVENT_INC(TABLEAPI_RETRIEVE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_RETRIEVE_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_get); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_DELETE: | ||||||
|  |       EVENT_INC(TABLEAPI_DELETE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_DELETE_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_delete); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_UPDATE: | ||||||
|  |       EVENT_INC(TABLEAPI_UPDATE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_UPDATE_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_update); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_INSERT_OR_UPDATE: | ||||||
|  |       EVENT_INC(TABLEAPI_INSERT_OR_UPDATE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_INSERT_OR_UPDATE_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_insert_or_update); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_REPLACE: | ||||||
|  |       EVENT_INC(TABLEAPI_REPLACE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_REPLACE_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_replace); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_INCREMENT: | ||||||
|  |       EVENT_INC(TABLEAPI_INCREMENT_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_INCREMENT_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_increment); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_SINGLE_APPEND: | ||||||
|  |       EVENT_INC(TABLEAPI_APPEND_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_APPEND_TIME, elapsed_us); | ||||||
|  |       SET_AUDIT_SQL_STRING(single_append); | ||||||
|  |       break; | ||||||
|  |  | ||||||
|  |       // table batch mutate | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_INSERT: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_INSERT_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_INSERT_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_INSERT_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_insert); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_GET: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_RETRIEVE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_RETRIEVE_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_RETRIEVE_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_get); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_DELETE: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_DELETE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_DELETE_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_DELETE_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_delete); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_UPDATE: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_UPDATE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_UPDATE_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_UPDATE_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_update); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_INSERT_OR_UPDATE: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_INSERT_OR_UPDATE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_INSERT_OR_UPDATE_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_INSERT_OR_UPDATE_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_insert_or_update); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_REPLACE: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_REPLACE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_REPLACE_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_REPLACE_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_replace); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_INCREMENT: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_INCREMENT_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_INCREMENT_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_INCREMENT_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_increment); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_MULTI_APPEND: | ||||||
|  |       EVENT_INC(TABLEAPI_MULTI_APPEND_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_APPEND_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_MULTI_APPEND_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(multi_append); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_BATCH_RETRIVE: | ||||||
|  |       EVENT_INC(TABLEAPI_BATCH_RETRIEVE_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_BATCH_RETRIEVE_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_BATCH_RETRIEVE_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(batch_retrieve); | ||||||
|  |       break; | ||||||
|  |     case ObTableProccessType::TABLE_API_BATCH_HYBRID: | ||||||
|  |       EVENT_INC(TABLEAPI_BATCH_HYBRID_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_BATCH_HYBRID_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_BATCH_HYBRID_INSERT_OR_UPDATE_ROW, rows); // @todo row count for each type | ||||||
|  |       SET_AUDIT_SQL_STRING(batch_hybrid); | ||||||
|  |       break; | ||||||
|  |     // table query | ||||||
|  |     case ObTableProccessType::TABLE_API_TABLE_QUERY: | ||||||
|  |       EVENT_INC(TABLEAPI_QUERY_COUNT); | ||||||
|  |       EVENT_ADD(TABLEAPI_QUERY_TIME, elapsed_us); | ||||||
|  |       EVENT_ADD(TABLEAPI_QUERY_ROW, rows); | ||||||
|  |       SET_AUDIT_SQL_STRING(table_query); | ||||||
|  |       break; | ||||||
|  |  | ||||||
|  |     default: | ||||||
|  |       SET_AUDIT_SQL_STRING(unknown); | ||||||
|  |       SERVER_LOG(WARN, "unknow process type", K(process_type), K(elapsed_us), K(rows)); | ||||||
|  |       break; | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |  | ||||||
|  |   static int negate_htable_timestamp(table::ObITableEntity &entity); | ||||||
|  | private: | ||||||
|  |   // disallow copy | ||||||
|  |   DISALLOW_COPY_AND_ASSIGN(ObTableRpcProcessorUtil); | ||||||
|  |   ObTableRpcProcessorUtil() = delete; | ||||||
|  |   ~ObTableRpcProcessorUtil() = delete; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | bool is_bad_routing_err(const int err); | ||||||
|  |  | ||||||
|  | } // end namespace observer | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_RPC_PROCESSOR_UTIL_H */ | ||||||
							
								
								
									
										2044
									
								
								src/observer/table/ob_table_service.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										2044
									
								
								src/observer/table/ob_table_service.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										320
									
								
								src/observer/table/ob_table_service.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										320
									
								
								src/observer/table/ob_table_service.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,320 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_SERVICE_H | ||||||
|  | #define _OB_TABLE_SERVICE_H 1 | ||||||
|  | #include "share/table/ob_table.h" | ||||||
|  | #include "observer/ob_server_struct.h" | ||||||
|  | #include "sql/engine/expr/ob_expr_res_type.h" | ||||||
|  | #include "share/table/ob_table_rpc_struct.h" | ||||||
|  | #include "storage/ob_dml_param.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace storage | ||||||
|  | { | ||||||
|  | class ObPartitionService; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | namespace observer | ||||||
|  | { | ||||||
|  | using table::ObTableOperation; | ||||||
|  | using table::ObTableOperationResult; | ||||||
|  | using table::ObTableBatchOperation; | ||||||
|  | using table::ObTableBatchOperationResult; | ||||||
|  | using table::ObITableBatchOperationResult; | ||||||
|  | using table::ObTableQuery; | ||||||
|  | using table::ObTableQueryResult; | ||||||
|  | class ObTableApiProcessorBase; | ||||||
|  | class ObTableService; | ||||||
|  | class ObTableApiRowIterator; | ||||||
|  |  | ||||||
|  | class ObTableServiceCtx | ||||||
|  | { | ||||||
|  |   static const int64_t COMMON_COLUMN_NUM = 16; | ||||||
|  | public: | ||||||
|  |   common::ObSEArray<sql::ObExprResType, COMMON_COLUMN_NUM> columns_type_; | ||||||
|  | protected: | ||||||
|  |   friend class ObTableService; | ||||||
|  |   friend class ObTableApiRowIterator; | ||||||
|  |   struct Param | ||||||
|  |   { | ||||||
|  |     uint64_t table_id_; | ||||||
|  |     uint64_t partition_id_; | ||||||
|  |     int64_t timeout_ts_; | ||||||
|  |     ObTableApiProcessorBase *processor_; | ||||||
|  |     common::ObArenaAllocator *allocator_; | ||||||
|  |     bool returning_affected_rows_; | ||||||
|  |     table::ObTableEntityType entity_type_; | ||||||
|  |     table::ObBinlogRowImageType binlog_row_image_type_; | ||||||
|  |     bool returning_affected_entity_; | ||||||
|  |     bool returning_rowkey_; | ||||||
|  |     Param() | ||||||
|  |         :table_id_(common::OB_INVALID_ID), | ||||||
|  |          partition_id_(common::OB_INVALID_ID), | ||||||
|  |          timeout_ts_(0), | ||||||
|  |          processor_(nullptr), | ||||||
|  |          allocator_(nullptr), | ||||||
|  |          returning_affected_rows_(false), | ||||||
|  |          entity_type_(table::ObTableEntityType::ET_DYNAMIC), | ||||||
|  |          binlog_row_image_type_(table::ObBinlogRowImageType::FULL), | ||||||
|  |          returning_affected_entity_(false), | ||||||
|  |          returning_rowkey_(false) | ||||||
|  |     {} | ||||||
|  |   } param_; | ||||||
|  | public: | ||||||
|  |   ObTableServiceCtx() | ||||||
|  |       :param_() | ||||||
|  |   {} | ||||||
|  |   void reset_dml() | ||||||
|  |   { | ||||||
|  |     columns_type_.reset(); | ||||||
|  |   } | ||||||
|  |   void init_param(int64_t timeout_ts, | ||||||
|  |                   ObTableApiProcessorBase *processor, | ||||||
|  |                   common::ObArenaAllocator *allocator, | ||||||
|  |                   bool returning_affected_rows, | ||||||
|  |                   table::ObTableEntityType entity_type, | ||||||
|  |                   table::ObBinlogRowImageType binlog_row_image_type, | ||||||
|  |                   bool returning_affected_entity = false, | ||||||
|  |                   bool returning_rowkey = false) | ||||||
|  |   { | ||||||
|  |     param_.timeout_ts_ = timeout_ts; | ||||||
|  |     param_.processor_ = processor; | ||||||
|  |     param_.allocator_ = allocator; | ||||||
|  |     param_.returning_affected_rows_ = returning_affected_rows; | ||||||
|  |     param_.entity_type_ = entity_type; | ||||||
|  |     param_.binlog_row_image_type_ = binlog_row_image_type; | ||||||
|  |     param_.returning_affected_entity_ = returning_affected_entity; | ||||||
|  |     param_.returning_rowkey_ = returning_rowkey; | ||||||
|  |   } | ||||||
|  |   uint64_t ¶m_table_id() { return param_.table_id_; } | ||||||
|  |   uint64_t ¶m_partition_id() { return param_.partition_id_; } | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObTableServiceGetCtx: public ObTableServiceCtx | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   share::schema::ObTableParam table_param_on_stack_; | ||||||
|  |   share::schema::ObTableParam *table_param_; | ||||||
|  |   storage::ObTableScanParam scan_param_; | ||||||
|  |   common::ObNewRowIterator *scan_result_; | ||||||
|  | public: | ||||||
|  |   ObTableServiceGetCtx(common::ObArenaAllocator &alloc); | ||||||
|  |   void reset_get_ctx() | ||||||
|  |   { | ||||||
|  |     ObTableServiceCtx::reset_dml(); | ||||||
|  |     table_param_->reset(); | ||||||
|  |   } | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObNormalTableQueryResultIterator: public table::ObTableQueryResultIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObNormalTableQueryResultIterator(const ObTableQuery &query, table::ObTableQueryResult &one_result) | ||||||
|  |       :one_result_(one_result), | ||||||
|  |        query_(&query), | ||||||
|  |        last_row_(NULL), | ||||||
|  |        batch_size_(query.get_batch()), | ||||||
|  |        max_result_size_(std::min(query.get_max_result_size(), | ||||||
|  |                                  static_cast<int64_t>(common::OB_MAX_PACKET_BUFFER_LENGTH-1024))), | ||||||
|  |        scan_result_(NULL), | ||||||
|  |        is_first_result_(true), | ||||||
|  |        has_more_rows_(true) | ||||||
|  |   { | ||||||
|  |   } | ||||||
|  |   virtual ~ObNormalTableQueryResultIterator() {} | ||||||
|  |   virtual int get_next_result(table::ObTableQueryResult *&one_result) override; | ||||||
|  |   virtual bool has_more_result() const override; | ||||||
|  |   void set_scan_result(common::ObNewRowIterator *scan_result) { scan_result_ = scan_result; } | ||||||
|  | private: | ||||||
|  |   table::ObTableQueryResult &one_result_; | ||||||
|  |   const ObTableQuery *query_; | ||||||
|  |   common::ObNewRow *last_row_; | ||||||
|  |   int32_t batch_size_; | ||||||
|  |   int64_t max_result_size_; | ||||||
|  |   common::ObNewRowIterator *scan_result_; | ||||||
|  |   bool is_first_result_; | ||||||
|  |   bool has_more_rows_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | struct ObTableServiceQueryCtx: public ObTableServiceGetCtx | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObNormalTableQueryResultIterator *normal_result_iterator_; | ||||||
|  | public: | ||||||
|  |   ObTableServiceQueryCtx(common::ObArenaAllocator &alloc) | ||||||
|  |       :ObTableServiceGetCtx(alloc), | ||||||
|  |        normal_result_iterator_(NULL) | ||||||
|  |   {} | ||||||
|  |   void reset_query_ctx(storage::ObPartitionService *part_service) | ||||||
|  |   { | ||||||
|  |     destroy_result_iterator(part_service); | ||||||
|  |     ObTableServiceGetCtx::reset_get_ctx(); | ||||||
|  |   } | ||||||
|  |   ObNormalTableQueryResultIterator *get_normal_result_iterator(const ObTableQuery &query, | ||||||
|  |                                                                table::ObTableQueryResult &one_result); | ||||||
|  |   void destroy_result_iterator(storage::ObPartitionService *part_service); | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// table service | ||||||
|  | class ObTableService | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   friend class TestBatchExecute_obj_increment_Test; | ||||||
|  | public: | ||||||
|  |   ObTableService() | ||||||
|  |       :part_service_(NULL), | ||||||
|  |        schema_service_(NULL) | ||||||
|  |   {} | ||||||
|  |   virtual ~ObTableService() = default; | ||||||
|  |   int init(ObGlobalContext &gctx); | ||||||
|  |  | ||||||
|  |   int execute_get(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result); | ||||||
|  |   int execute_insert_or_update(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result); | ||||||
|  |   int execute_delete(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result); | ||||||
|  |   int execute_insert(ObTableServiceCtx &ctx, const ObTableOperation &table_operation, | ||||||
|  |       ObTableOperationResult &result, ObNewRowIterator *&duplicate_row_iter); | ||||||
|  |   int execute_replace(ObTableServiceCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result); | ||||||
|  |   int execute_update(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, | ||||||
|  |       ObNewRow *target_row, ObTableOperationResult &result); | ||||||
|  |   int execute_increment(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result); | ||||||
|  |  | ||||||
|  |   int multi_get(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |   int multi_insert_or_update(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |   int multi_delete(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |   int multi_insert(ObTableServiceCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |   int multi_replace(ObTableServiceCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |   int multi_update(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |  | ||||||
|  |   int batch_execute(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |   int execute_query(ObTableServiceQueryCtx &ctx, const ObTableQuery &query, | ||||||
|  |                     table::ObTableQueryResult &one_result, table::ObTableQueryResultIterator *&query_result); | ||||||
|  | private: | ||||||
|  |   static int cons_rowkey_infos(const share::schema::ObTableSchema &table_schema, | ||||||
|  |                                common::ObIArray<uint64_t> *column_ids, | ||||||
|  |                                common::ObIArray<sql::ObExprResType> *columns_type); | ||||||
|  |   static int cons_properties_infos(const share::schema::ObTableSchema &table_schema, | ||||||
|  |                                    const common::ObIArray<common::ObString> &properties, | ||||||
|  |                                    common::ObIArray<uint64_t> &column_ids, | ||||||
|  |                                    common::ObIArray<sql::ObExprResType> *columns_type); | ||||||
|  |   static int cons_column_type(const share::schema::ObColumnSchemaV2 &column_schema, sql::ObExprResType &column_type); | ||||||
|  |   static int check_column_type(const sql::ObExprResType &column_type, common::ObObj &obj); | ||||||
|  |   static int add_index_columns_if_missing(share::schema::ObSchemaGetterGuard &schema_guard, | ||||||
|  |                                           uint64_t data_table_id, | ||||||
|  |                                           const share::schema::ObTableSchema *index_schema, | ||||||
|  |                                           common::ObIArray<uint64_t> &column_ids, | ||||||
|  |                                           common::ObIArray<sql::ObExprResType> *columns_type); | ||||||
|  |  | ||||||
|  |   int insert_or_update_can_use_put(uint64_t table_id, const table::ObITableEntity &entity, bool &use_put); | ||||||
|  |   int add_one_result(ObTableBatchOperationResult &result, | ||||||
|  |                      table::ObTableOperationType::Type op_type, | ||||||
|  |                      int32_t error_code, | ||||||
|  |                      int64_t affected_rows); | ||||||
|  |   int do_put(ObTableServiceCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result); | ||||||
|  |   int do_insert_or_update(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result); | ||||||
|  |   int multi_put(ObTableServiceCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result); | ||||||
|  |   int do_multi_insert_or_update(ObTableServiceGetCtx &ctx, | ||||||
|  |                                 const ObTableBatchOperation &batch_operation, | ||||||
|  |                                 ObTableBatchOperationResult &result); | ||||||
|  |   // for get | ||||||
|  |   int fill_scan_param(ObTableServiceCtx &ctx, | ||||||
|  |                       const ObIArray<uint64_t> &output_column_ids, | ||||||
|  |                       int64_t schema_version, | ||||||
|  |                       storage::ObTableScanParam &scan_param); | ||||||
|  |   int fill_get_result( | ||||||
|  |       ObTableServiceCtx &ctx, | ||||||
|  |       const ObIArray<ObString> &properties, | ||||||
|  |       ObTableApiRowIterator *scan_result, | ||||||
|  |       ObTableOperationResult &operation_result); | ||||||
|  |   // for multi-get | ||||||
|  |   int fill_multi_get_result( | ||||||
|  |       ObTableServiceGetCtx &ctx, | ||||||
|  |       const ObTableBatchOperation &batch_operation, | ||||||
|  |       ObTableApiRowIterator *scan_result, | ||||||
|  |       ObTableBatchOperationResult &result); | ||||||
|  |   int delete_can_use_put(uint64_t table_id, bool &use_put); | ||||||
|  |   static int cons_all_index_properties(share::schema::ObSchemaGetterGuard &schema_guard, | ||||||
|  |                                        const share::schema::ObTableSchema &table_schema, | ||||||
|  |                                        common::ObIArray<uint64_t> &column_ids, | ||||||
|  |                                        common::ObIArray<sql::ObExprResType> *columns_type); | ||||||
|  |  | ||||||
|  |   // for replace | ||||||
|  |   int do_replace( | ||||||
|  |       ObTableServiceCtx &ctx, | ||||||
|  |       common::ObPartitionKey &part_key, | ||||||
|  |       storage::ObDMLBaseParam &dml_param, | ||||||
|  |       common::ObIArray<uint64_t> &column_ids, | ||||||
|  |       common::ObIArray<uint64_t> &rowkey_column_ids, | ||||||
|  |       common::ObNewRow &row, | ||||||
|  |       int64_t &affected_rows); | ||||||
|  |   // for replace delete duplicate row | ||||||
|  |   int do_replace_delete( | ||||||
|  |       ObTableServiceCtx &ctx, | ||||||
|  |       common::ObPartitionKey &part_key, | ||||||
|  |       storage::ObDMLBaseParam &dml_param, | ||||||
|  |       common::ObIArray<uint64_t> &column_ids, | ||||||
|  |       common::ObNewRowIterator *duplicated_rows, | ||||||
|  |       int64_t &affected_rows); | ||||||
|  |   // for execute_query | ||||||
|  |   int cons_index_key_type(share::schema::ObSchemaGetterGuard &schema_guard, | ||||||
|  |                           const share::schema::ObTableSchema *index_schema, | ||||||
|  |                           uint64_t data_table_id, | ||||||
|  |                           common::ObIArray<sql::ObExprResType> &columns_type); | ||||||
|  |   int get_index_id_by_name(share::schema::ObSchemaGetterGuard &schema_guard, uint64_t base_table_id, | ||||||
|  |                            const ObString &index_name, uint64_t &index_id, | ||||||
|  |                            common::ObIArray<sql::ObExprResType> &columns_type, | ||||||
|  |                            const share::schema::ObTableSchema *&index_schema); | ||||||
|  |   int fill_query_table_param(uint64_t table_id, | ||||||
|  |                              const common::ObIArray<ObString> &properties, | ||||||
|  |                              const ObString &index_name, | ||||||
|  |                              share::schema::ObTableParam &table_param, | ||||||
|  |                              common::ObIArray<uint64_t> &output_column_ids, | ||||||
|  |                              common::ObIArray<sql::ObExprResType> &rowkey_columns_type, | ||||||
|  |                              int64_t &schema_version, | ||||||
|  |                              uint64_t &index_id, | ||||||
|  |                              int64_t &padding_num); | ||||||
|  |   int fill_query_scan_ranges(ObTableServiceCtx &ctx, | ||||||
|  |                              const ObTableQuery &query, | ||||||
|  |                              int64_t padding_num, | ||||||
|  |                              storage::ObTableScanParam &scan_param); | ||||||
|  |   int fill_query_scan_param(ObTableServiceCtx &ctx, | ||||||
|  |                             const common::ObIArray<uint64_t> &output_column_ids, | ||||||
|  |                             int64_t schema_version, | ||||||
|  |                             ObQueryFlag::ScanOrder scan_order, | ||||||
|  |                             uint64_t index_id, | ||||||
|  |                             int32_t limit, | ||||||
|  |                             int32_t offset, | ||||||
|  |                             storage::ObTableScanParam &scan_param); | ||||||
|  | private: | ||||||
|  |   int fill_new_entity( | ||||||
|  |       bool returning_rowkey, | ||||||
|  |       const common::ObNewRow &row, | ||||||
|  |       const int64_t primary_key_size, | ||||||
|  |       const common::ObIArray<common::ObString> &properties, | ||||||
|  |       common::ObIAllocator &alloc, | ||||||
|  |       table::ObITableEntity *new_entity); | ||||||
|  |   int execute_increment_by_update(ObTableServiceGetCtx &ctx, | ||||||
|  |                                   const ObTableOperation &table_operation, | ||||||
|  |                                   ObTableOperationResult &result); | ||||||
|  | private: | ||||||
|  |   // disallow copy | ||||||
|  |   DISALLOW_COPY_AND_ASSIGN(ObTableService); | ||||||
|  | private: | ||||||
|  |   static const int64_t COMMON_COLUMN_NUM = 16; | ||||||
|  |   storage::ObPartitionService *part_service_; | ||||||
|  |   share::schema::ObMultiVersionSchemaService *schema_service_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | } // end namespace observer | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_SERVICE_H */ | ||||||
| @ -34,6 +34,11 @@ ob_set_subtarget(ob_share config | |||||||
|   config/ob_system_config_key.cpp |   config/ob_system_config_key.cpp | ||||||
| ) | ) | ||||||
|  |  | ||||||
|  | ob_set_subtarget(ob_share table | ||||||
|  |   table/ob_table.cpp		 | ||||||
|  |   table/ob_table_rpc_struct.cpp		 | ||||||
|  | ) | ||||||
|  |  | ||||||
| file(GLOB SCHEMA_CPPS "inner_table/ob_inner_table_schema.*.cpp") | file(GLOB SCHEMA_CPPS "inner_table/ob_inner_table_schema.*.cpp") | ||||||
| ob_set_subtarget(ob_share inner_table | ob_set_subtarget(ob_share inner_table | ||||||
|   inner_table/ob_inner_table_schema_misc.ipp |   inner_table/ob_inner_table_schema_misc.ipp | ||||||
|  | |||||||
							
								
								
									
										1381
									
								
								src/share/table/ob_table.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										1381
									
								
								src/share/table/ob_table.cpp
									
									
									
									
									
										Normal file
									
								
							
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							
							
								
								
									
										658
									
								
								src/share/table/ob_table.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										658
									
								
								src/share/table/ob_table.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,658 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_TABLE_H | ||||||
|  | #define _OB_TABLE_TABLE_H 1 | ||||||
|  | #include "lib/ob_define.h" | ||||||
|  | #include "lib/ob_errno.h" | ||||||
|  | #include "lib/string/ob_string.h" | ||||||
|  | #include "common/object/ob_object.h" | ||||||
|  | #include "common/rowkey/ob_rowkey.h" | ||||||
|  | #include "lib/container/ob_iarray.h" | ||||||
|  | #include "lib/container/ob_se_array.h" | ||||||
|  | #include "lib/hash/ob_hashmap.h" | ||||||
|  | #include "lib/list/ob_dlist.h" | ||||||
|  | #include "common/ob_common_types.h" | ||||||
|  | #include "common/ob_range.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace common | ||||||
|  | { | ||||||
|  | class ObNewRow; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | namespace table | ||||||
|  | { | ||||||
|  | using common::ObString; | ||||||
|  | using common::ObRowkey; | ||||||
|  | using common::ObObj; | ||||||
|  | using common::ObIArray; | ||||||
|  | using common::ObSEArray; | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | // structs of a table storage interface | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  |  | ||||||
|  | /// A Table Entity | ||||||
|  | class ObITableEntity: public common::ObDLinkBase<ObITableEntity> | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION_V(1); | ||||||
|  | public: | ||||||
|  |   ObITableEntity() | ||||||
|  |       :alloc_(NULL) | ||||||
|  |   {} | ||||||
|  |   virtual ~ObITableEntity() = default; | ||||||
|  |   virtual void reset() = 0; | ||||||
|  |   virtual bool is_empty() const { return 0 == get_rowkey_size() && 0 == get_properties_count(); } | ||||||
|  |   //@{ primary key contains partition key. Note that all values are shallow copy. | ||||||
|  |   virtual int set_rowkey(const ObRowkey &rowkey) = 0; | ||||||
|  |   virtual int set_rowkey(const ObITableEntity &other) = 0; | ||||||
|  |   virtual int set_rowkey_value(int64_t idx, const ObObj &value) = 0; | ||||||
|  |   virtual int add_rowkey_value(const ObObj &value) = 0; | ||||||
|  |   virtual int64_t get_rowkey_size() const = 0; | ||||||
|  |   virtual int get_rowkey_value(int64_t idx, ObObj &value) const = 0; | ||||||
|  |   virtual ObRowkey get_rowkey() = 0; | ||||||
|  |   virtual int64_t hash_rowkey() const = 0; | ||||||
|  |   //@} | ||||||
|  |   //@{ property is a key-value pair. | ||||||
|  |   virtual int set_property(const ObString &prop_name, const ObObj &prop_value) = 0; | ||||||
|  |   virtual int get_property(const ObString &prop_name, ObObj &prop_value) const = 0; | ||||||
|  |   virtual int get_properties(ObIArray<std::pair<ObString, ObObj> > &properties) const = 0; // @todo property iterator | ||||||
|  |   virtual int get_properties_names(ObIArray<ObString> &properties) const = 0; | ||||||
|  |   virtual int get_properties_values(ObIArray<ObObj> &properties_values) const = 0; | ||||||
|  |   virtual int64_t get_properties_count() const = 0; | ||||||
|  |   //@} | ||||||
|  |   virtual int deep_copy(common::ObIAllocator &allocator, const ObITableEntity &other); | ||||||
|  |   int deep_copy_rowkey(common::ObIAllocator &allocator, const ObITableEntity &other); | ||||||
|  |   int deep_copy_properties(common::ObIAllocator &allocator, const ObITableEntity &other); | ||||||
|  |   virtual int add_retrieve_property(const ObString &prop_name); | ||||||
|  |   void set_allocator(common::ObIAllocator *alloc) { alloc_ = alloc; } | ||||||
|  |   common::ObIAllocator *get_allocator() { return alloc_; } | ||||||
|  |   VIRTUAL_TO_STRING_KV("ITableEntity", ""); | ||||||
|  | protected: | ||||||
|  |   common::ObIAllocator *alloc_;  // for deep copy in deserialize | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObITableEntityFactory | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   virtual ~ObITableEntityFactory() = default; | ||||||
|  |   virtual ObITableEntity *alloc() = 0; | ||||||
|  |   virtual void free(ObITableEntity *obj) = 0; | ||||||
|  |   virtual void free_and_reuse() = 0; | ||||||
|  |   virtual int64_t get_used_count() const = 0; | ||||||
|  |   virtual int64_t get_free_count() const = 0; | ||||||
|  |   virtual int64_t get_used_mem() const = 0; | ||||||
|  |   virtual int64_t get_total_mem() const = 0; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// An implementation for ObITableEntity | ||||||
|  | class ObTableEntity: public ObITableEntity | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableEntity(); | ||||||
|  |   ~ObTableEntity(); | ||||||
|  |   virtual int set_rowkey(const ObRowkey &rowkey) override; | ||||||
|  |   virtual int set_rowkey(const ObITableEntity &other) override; | ||||||
|  |   virtual int set_rowkey_value(int64_t idx, const ObObj &value) override; | ||||||
|  |   virtual int add_rowkey_value(const ObObj &value) override; | ||||||
|  |   virtual int64_t get_rowkey_size() const override { return rowkey_.count(); }; | ||||||
|  |   virtual int get_rowkey_value(int64_t idx, ObObj &value) const override; | ||||||
|  |   virtual int64_t hash_rowkey() const override; | ||||||
|  |   virtual int get_property(const ObString &prop_name, ObObj &prop_value) const override; | ||||||
|  |   virtual int set_property(const ObString &prop_name, const ObObj &prop_value) override; | ||||||
|  |   virtual int get_properties(ObIArray<std::pair<ObString, ObObj> > &properties) const override; | ||||||
|  |   virtual int get_properties_names(ObIArray<ObString> &properties_names) const override; | ||||||
|  |   virtual int get_properties_values(ObIArray<ObObj> &properties_values) const override; | ||||||
|  |   virtual int64_t get_properties_count() const override; | ||||||
|  |   virtual void reset() override { rowkey_.reset(); properties_.clear(); } | ||||||
|  |   virtual ObRowkey get_rowkey() override; | ||||||
|  |   DECLARE_TO_STRING; | ||||||
|  | private: | ||||||
|  |   int try_init(); | ||||||
|  |   class GetPropertyFn; | ||||||
|  |   class GetPropertyNameFn; | ||||||
|  |   class GetPropertyValueFn; | ||||||
|  |   typedef common::hash::ObHashMap<ObString, ObObj, common::hash::NoPthreadDefendMode> PropertiesMap; | ||||||
|  | private: | ||||||
|  |   ObSEArray<ObObj, 8> rowkey_; | ||||||
|  |   PropertiesMap properties_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | enum class ObTableEntityType | ||||||
|  | { | ||||||
|  |   ET_DYNAMIC = 0, | ||||||
|  |   ET_KV = 1 | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | // @note not thread-safe | ||||||
|  | template <typename T> | ||||||
|  | class ObTableEntityFactory: public ObITableEntityFactory | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableEntityFactory(const char *label = common::ObModIds::TABLE_PROC) | ||||||
|  |       :alloc_(label) | ||||||
|  |   {} | ||||||
|  |   virtual ~ObTableEntityFactory(); | ||||||
|  |   virtual ObITableEntity *alloc() override; | ||||||
|  |   virtual void free(ObITableEntity *obj) override; | ||||||
|  |   virtual void free_and_reuse() override; | ||||||
|  |   virtual int64_t get_free_count() const { return free_list_.get_size(); } | ||||||
|  |   virtual int64_t get_used_count() const { return used_list_.get_size(); } | ||||||
|  |   virtual int64_t get_used_mem() const { return alloc_.used(); } | ||||||
|  |   virtual int64_t get_total_mem() const { return alloc_.total(); } | ||||||
|  | private: | ||||||
|  |   void free_all(); | ||||||
|  | private: | ||||||
|  |   common::ObArenaAllocator alloc_; | ||||||
|  |   common::ObDList<ObITableEntity> used_list_; | ||||||
|  |   common::ObDList<ObITableEntity> free_list_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | template <typename T> | ||||||
|  | ObTableEntityFactory<T>::~ObTableEntityFactory() | ||||||
|  | { | ||||||
|  |   free_all(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | template <typename T> | ||||||
|  | ObITableEntity *ObTableEntityFactory<T>::alloc() | ||||||
|  | { | ||||||
|  |   ObITableEntity *entity = free_list_.remove_first(); | ||||||
|  |   if (NULL == entity) { | ||||||
|  |     void * ptr = alloc_.alloc(sizeof(T)); | ||||||
|  |     if (NULL == ptr) { | ||||||
|  |       CLIENT_LOG(WARN, "no memory for table entity"); | ||||||
|  |     } else { | ||||||
|  |       entity = new(ptr) T(); | ||||||
|  |       used_list_.add_last(entity); | ||||||
|  |     } | ||||||
|  |   } else { | ||||||
|  |     used_list_.add_last(entity); | ||||||
|  |   } | ||||||
|  |   return entity; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | template <typename T> | ||||||
|  | void ObTableEntityFactory<T>::free(ObITableEntity *entity) | ||||||
|  | { | ||||||
|  |   if (NULL != entity) { | ||||||
|  |     entity->reset(); | ||||||
|  |     entity->set_allocator(NULL); | ||||||
|  |     used_list_.remove(entity); | ||||||
|  |     free_list_.add_last(entity); | ||||||
|  |   } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | template <typename T> | ||||||
|  | void ObTableEntityFactory<T>::free_and_reuse() | ||||||
|  | { | ||||||
|  |   while (!used_list_.is_empty()) { | ||||||
|  |     this->free(used_list_.get_first()); | ||||||
|  |   } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | template <typename T> | ||||||
|  | void ObTableEntityFactory<T>::free_all() | ||||||
|  | { | ||||||
|  |   ObITableEntity *entity = NULL; | ||||||
|  |   while (NULL != (entity = used_list_.remove_first())) { | ||||||
|  |     entity->~ObITableEntity(); | ||||||
|  |   } | ||||||
|  |   while (NULL != (entity = free_list_.remove_first())) { | ||||||
|  |     entity->~ObITableEntity(); | ||||||
|  |   } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | /// Table Operation Type | ||||||
|  | struct ObTableOperationType | ||||||
|  | { | ||||||
|  |   enum Type | ||||||
|  |   { | ||||||
|  |     GET = 0, | ||||||
|  |     INSERT = 1, | ||||||
|  |     DEL = 2, | ||||||
|  |     UPDATE = 3, | ||||||
|  |     INSERT_OR_UPDATE = 4, | ||||||
|  |     REPLACE = 5, | ||||||
|  |     INCREMENT = 6, | ||||||
|  |     APPEND = 7 | ||||||
|  |   }; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// A table operation | ||||||
|  | class ObTableOperation | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   /** | ||||||
|  |    * insert the entity. | ||||||
|  |    * @return ObTableOperationResult | ||||||
|  |    * In the case of insert success, the return errno is OB_SUCCESS, affected_rows is 1 | ||||||
|  |    * In the case of insert failed, the affected_rows is 0 | ||||||
|  |    * In the case of insert failed caused by primary key duplicated, the errno is OB_ERR_PRIMARY_KEY_DUPLICATE. | ||||||
|  |    * If the option returning_affected_rows is false (default value), then the return value of affected_rows is undefined, but with better performance. | ||||||
|  |    * Other common error code: OB_TIMEOUT indicates time out; OB_TRY_LOCK_ROW_CONFLICT indicate row lock conflict | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation insert(const ObITableEntity &entity); | ||||||
|  |   /** | ||||||
|  |    * delete the entity. | ||||||
|  |    * @return ObTableOperationResult | ||||||
|  |    * In the case of delete success, the errno is OB_SUCCESS and the affeceted_row is 1. | ||||||
|  |    * In the case of the row is NOT EXIST, the errno is OB_SUCCESS and the affected_row is 0. | ||||||
|  |    * If the option returning_affected_rows is false (default value), then the return value of affected_rows is undefined, but with better performance. | ||||||
|  |    * Other common error code: OB_TIMEOUT indicates time out; OB_TRY_LOCK_ROW_CONFLICT indicate row lock conflict | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation del(const ObITableEntity &entity); | ||||||
|  |   /** | ||||||
|  |    * update the entity. | ||||||
|  |    * @return ObTableOperationResult | ||||||
|  |    * In the case of update success, the errno is OB_SUCCESS and the affeceted_row is 1. | ||||||
|  |    * In the case of the row is NOT EXIST, the errno is OB_SUCCESS and the affected_row is 0. | ||||||
|  |    * If the option returning_affected_rows is false (default value), then the return value of affected_rows is undefined, but with better performance. | ||||||
|  |    * Other common error code: OB_TIMEOUT indicates time out; OB_TRY_LOCK_ROW_CONFLICT indicate row lock conflict | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation update(const ObITableEntity &entity); | ||||||
|  |   /** | ||||||
|  |    * insert_or_update the entity. | ||||||
|  |    * @return ObTableOperationResult | ||||||
|  |    * If the row is NOT exist, then insert the row. In the case of success, the return errno is OB_SUCCESS and the affected_rows is 1. | ||||||
|  |    * If the row is exist, then update the row. In the case of success, the return errno is OB_SUCCESS and the affected_rows i 1. | ||||||
|  |    * If the option returning_affected_rows is false (default value), then the return value of affected_rows is undefined, but with better performance. | ||||||
|  |    * Other common error code: OB_TIMEOUT; OB_TRY_LOCK_ROW_CONFLICT | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation insert_or_update(const ObITableEntity &entity); | ||||||
|  |   /** | ||||||
|  |    * replace the entity. | ||||||
|  |    * @return ObTableOperationResult | ||||||
|  |    * If the row is NOT EXIST, then insert the row. In the case of success, | ||||||
|  |    * the errno is OB_SUCCESS and the affected_row is 1. | ||||||
|  |    * Otherwise the row is EXIST, then delete the old row and insert the new row. In the case of success, | ||||||
|  |    * the errno is OB_SUCCESS and the affected_row is 1. | ||||||
|  |    * Specially, if there is uniq index conflict, then delete all rows cause conflict and insert the new row. | ||||||
|  |    * In the case of success, the errno is OB_SUCCESS and the affected_row >= 1. | ||||||
|  |    * If the option returning_affected_rows is false (default value), then the return value of affected_rows is undefined, but with better performance. | ||||||
|  |    * Other common error code: OB_TIMEOUT; OB_TRY_LOCK_ROW_CONFLICT | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation replace(const ObITableEntity &entity); | ||||||
|  |   /** | ||||||
|  |    * retrieve the entity. | ||||||
|  |    * @param entity Only return the given property | ||||||
|  |    * @return ObTableOperationResult | ||||||
|  |    * affected_rows is always 0 | ||||||
|  |    * If the row is exist, then return the ObTableOperationResult.entity | ||||||
|  |    * Otherwise, entity is empty. | ||||||
|  |    * Other common error code: OB_TIMEOUT | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation retrieve(const ObITableEntity &entity); | ||||||
|  |   /** | ||||||
|  |    * Increase the value of given column. | ||||||
|  |    * The type of the column MUST be integer. | ||||||
|  |    * If the original value of given column is NULL, use the new value to replace it. | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation increment(const ObITableEntity &entity); | ||||||
|  |   /** | ||||||
|  |    * Append the given string to original string. | ||||||
|  |    * The type of the column MUST be string type, such as char, varchar, binary, varbinary or lob. | ||||||
|  |    * If the original value of given column is NULL, use the new value to replace it. | ||||||
|  |    */ | ||||||
|  |   static ObTableOperation append(const ObITableEntity &entity); | ||||||
|  | public: | ||||||
|  |   const ObITableEntity &entity() const { return *entity_; } | ||||||
|  |   ObTableOperationType::Type type() const { return operation_type_; } | ||||||
|  |   void set_entity(const ObITableEntity &entity) { entity_ = &entity; } | ||||||
|  |   void set_type(ObTableOperationType::Type op_type) { operation_type_ = op_type; } | ||||||
|  |   int get_entity(ObITableEntity *&entity); | ||||||
|  |   uint64_t get_checksum(); | ||||||
|  |   TO_STRING_KV(K_(operation_type), "entity", to_cstring(entity_)); | ||||||
|  | private: | ||||||
|  |   const ObITableEntity *entity_; | ||||||
|  |   ObTableOperationType::Type operation_type_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// common result for ObTable | ||||||
|  | class ObTableResult | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableResult() | ||||||
|  |       :errno_(common::OB_ERR_UNEXPECTED) | ||||||
|  |   { | ||||||
|  |     sqlstate_[0] = '\0'; | ||||||
|  |     msg_[0] = '\0'; | ||||||
|  |   } | ||||||
|  |   ~ObTableResult() = default; | ||||||
|  |   void set_errno(int err) { errno_ = err; } | ||||||
|  |   int get_errno() const { return errno_; } | ||||||
|  |   int assign(const ObTableResult &other); | ||||||
|  |   TO_STRING_KV(K_(errno)); | ||||||
|  | private: | ||||||
|  |   static const int64_t MAX_MSG_SIZE = common::OB_MAX_ERROR_MSG_LEN; | ||||||
|  | protected: | ||||||
|  |   int32_t errno_; | ||||||
|  |   char sqlstate_[6];  // terminate with '\0' | ||||||
|  |   char msg_[MAX_MSG_SIZE]; // terminate with '\0' | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// result for ObTableOperation | ||||||
|  | class ObTableOperationResult final: public ObTableResult | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableOperationResult(); | ||||||
|  |   ~ObTableOperationResult() = default; | ||||||
|  |  | ||||||
|  |   ObTableOperationType::Type type() const { return operation_type_; } | ||||||
|  |   int get_entity(const ObITableEntity *&entity) const; | ||||||
|  |   int get_entity(ObITableEntity *&entity); | ||||||
|  |   int64_t get_affected_rows() const { return affected_rows_; } | ||||||
|  |  | ||||||
|  |   void set_entity(ObITableEntity &entity) { entity_ = &entity; } | ||||||
|  |   void set_type(ObTableOperationType::Type op_type) { operation_type_ = op_type; } | ||||||
|  |   void set_affected_rows(int64_t affected_rows) { affected_rows_ = affected_rows; } | ||||||
|  |  | ||||||
|  |   int deep_copy(common::ObIAllocator &allocator, ObITableEntityFactory &entity_factory, const ObTableOperationResult &other); | ||||||
|  |   DECLARE_TO_STRING; | ||||||
|  | private: | ||||||
|  |   ObTableOperationType::Type operation_type_; | ||||||
|  |   ObITableEntity *entity_; | ||||||
|  |   int64_t affected_rows_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObIRetryPolicy | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   virtual bool need_retry(int32_t curr_retry_count, int last_errno, int64_t &retry_interval)  | ||||||
|  |   { | ||||||
|  |     UNUSEDx(curr_retry_count, last_errno, retry_interval); | ||||||
|  |     return false;  | ||||||
|  |   } | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObLinearRetry : public ObIRetryPolicy | ||||||
|  | {}; | ||||||
|  |  | ||||||
|  | class ObExponentialRetry : public ObIRetryPolicy | ||||||
|  | {}; | ||||||
|  |  | ||||||
|  | class ObNoRetry : public ObIRetryPolicy | ||||||
|  | {}; | ||||||
|  |  | ||||||
|  | /// consistency levels | ||||||
|  | /// @see https://www.atatech.org/articles/102030 | ||||||
|  | enum class ObTableConsistencyLevel | ||||||
|  | { | ||||||
|  |   STRONG = 0, | ||||||
|  |   EVENTUAL = 1, | ||||||
|  | }; | ||||||
|  | /// clog row image type | ||||||
|  | /// @see share::ObBinlogRowImage | ||||||
|  | enum class ObBinlogRowImageType | ||||||
|  | { | ||||||
|  |   MINIMAL = 0, | ||||||
|  |   NOBLOB = 1, | ||||||
|  |   FULL = 2, | ||||||
|  | }; | ||||||
|  | /// request options for all the table operations | ||||||
|  | class ObTableRequestOptions final | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableRequestOptions(); | ||||||
|  |   ~ObTableRequestOptions() = default; | ||||||
|  |  | ||||||
|  |   void set_consistency_level(ObTableConsistencyLevel consistency_level) { consistency_level_ = consistency_level; } | ||||||
|  |   ObTableConsistencyLevel consistency_level() const { return consistency_level_; } | ||||||
|  |   void set_server_timeout(int64_t server_timeout_us) { server_timeout_us_ = server_timeout_us; } | ||||||
|  |   int64_t server_timeout() const { return server_timeout_us_; } | ||||||
|  |   void set_execution_time(int64_t max_execution_time_us) { max_execution_time_us_ = max_execution_time_us; } | ||||||
|  |   int64_t max_execution_time() const { return max_execution_time_us_; } | ||||||
|  |   void set_retry_policy(ObIRetryPolicy *retry_policy) { retry_policy_ = retry_policy; } | ||||||
|  |   ObIRetryPolicy* retry_policy() { return retry_policy_; } | ||||||
|  |   void set_returning_affected_rows(bool returning) { returning_affected_rows_ = returning; } | ||||||
|  |   bool returning_affected_rows() const { return returning_affected_rows_; } | ||||||
|  |   void set_returning_rowkey(bool returning) { returning_rowkey_ = returning; } | ||||||
|  |   bool returning_rowkey() const { return returning_rowkey_; } | ||||||
|  |   void set_returning_affected_entity(bool returning) { returning_affected_entity_ = returning; } | ||||||
|  |   bool returning_affected_entity() const { return returning_affected_entity_; } | ||||||
|  |   void set_binlog_row_image_type(ObBinlogRowImageType type) { binlog_row_image_type_ = type; } | ||||||
|  |   ObBinlogRowImageType binlog_row_image_type() const { return binlog_row_image_type_; } | ||||||
|  | private: | ||||||
|  |   ObTableConsistencyLevel consistency_level_; | ||||||
|  |   int64_t server_timeout_us_; | ||||||
|  |   int64_t max_execution_time_us_; | ||||||
|  |   ObIRetryPolicy *retry_policy_; | ||||||
|  |   bool returning_affected_rows_;  // default: false | ||||||
|  |   bool returning_rowkey_;         // default: false | ||||||
|  |   bool returning_affected_entity_;  // default: false | ||||||
|  |   // bool batch_operation_as_atomic_;  // default: false | ||||||
|  |   // int route_policy | ||||||
|  |   ObBinlogRowImageType binlog_row_image_type_;  // default: FULL | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// A batch operation | ||||||
|  | class ObTableBatchOperation | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   static const int64_t MAX_BATCH_SIZE = 1000; | ||||||
|  |   static const int64_t COMMON_BATCH_SIZE = 8; | ||||||
|  | public: | ||||||
|  |   ObTableBatchOperation() | ||||||
|  |       :table_operations_(common::ObModIds::TABLE_BATCH_OPERATION, common::OB_MALLOC_NORMAL_BLOCK_SIZE), | ||||||
|  |        is_readonly_(true), | ||||||
|  |        is_same_type_(true), | ||||||
|  |        is_same_properties_names_(true), | ||||||
|  |        entity_factory_(NULL) | ||||||
|  |   {} | ||||||
|  |   ~ObTableBatchOperation() = default; | ||||||
|  |   void reset(); | ||||||
|  |   void set_entity_factory(ObITableEntityFactory *entity_factory) { entity_factory_ = entity_factory; } | ||||||
|  |   /// insert the entity if not exists | ||||||
|  |   int insert(const ObITableEntity &entity); | ||||||
|  |   /// delete the entity if exists | ||||||
|  |   int del(const ObITableEntity &entity); | ||||||
|  |   /// update the entity if exists | ||||||
|  |   int update(const ObITableEntity &entity); | ||||||
|  |   /// insert the entity if not exists, otherwise update it | ||||||
|  |   int insert_or_update(const ObITableEntity &entity); | ||||||
|  |   /// insert the entity if not exists, otherwise replace it | ||||||
|  |   int replace(const ObITableEntity &entity); | ||||||
|  |   /// get the entity if exists | ||||||
|  |   int retrieve(const ObITableEntity &entity); | ||||||
|  |   /// add one table operation | ||||||
|  |   int add(const ObTableOperation &table_operation); | ||||||
|  |   /// increment the value | ||||||
|  |   int increment(const ObITableEntity &entity); | ||||||
|  |   /// append to the value | ||||||
|  |   int append(const ObITableEntity &entity); | ||||||
|  |  | ||||||
|  |   int64_t count() const { return table_operations_.count(); } | ||||||
|  |   const ObTableOperation &at(int64_t idx) const { return table_operations_.at(idx); } | ||||||
|  |   bool is_readonly() const { return is_readonly_; } | ||||||
|  |   bool is_same_type() const { return is_same_type_; } | ||||||
|  |   bool is_same_properties_names() const { return is_same_properties_names_; } | ||||||
|  |   uint64_t get_checksum(); | ||||||
|  |   TO_STRING_KV(K_(is_readonly), | ||||||
|  |                K_(is_same_type), | ||||||
|  |                K_(is_same_properties_names), | ||||||
|  |                "operatiton_count", table_operations_.count(), | ||||||
|  |                K_(table_operations)); | ||||||
|  | private: | ||||||
|  |   ObSEArray<ObTableOperation, COMMON_BATCH_SIZE> table_operations_; | ||||||
|  |   bool is_readonly_; | ||||||
|  |   bool is_same_type_; | ||||||
|  |   bool is_same_properties_names_; | ||||||
|  |   // do not serialize | ||||||
|  |   ObITableEntityFactory *entity_factory_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// result for ObTableBatchOperation | ||||||
|  | typedef ObIArray<ObTableOperationResult> ObITableBatchOperationResult; | ||||||
|  | class ObTableBatchOperationResult: public common::ObSEArrayImpl<ObTableOperationResult, ObTableBatchOperation::COMMON_BATCH_SIZE> | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableBatchOperationResult() | ||||||
|  |       :BaseType(common::ObModIds::TABLE_BATCH_OPERATION_RESULT, common::OB_MALLOC_NORMAL_BLOCK_SIZE), | ||||||
|  |        entity_factory_(NULL), | ||||||
|  |        alloc_(NULL) | ||||||
|  |   {} | ||||||
|  |   virtual ~ObTableBatchOperationResult() = default; | ||||||
|  |   void set_entity_factory(ObITableEntityFactory *entity_factory) { entity_factory_ = entity_factory; } | ||||||
|  |   ObITableEntityFactory *get_entity_factory() { return entity_factory_; } | ||||||
|  |   void set_allocator(common::ObIAllocator *alloc) { alloc_ = alloc; } | ||||||
|  |   common::ObIAllocator *get_allocator() { return alloc_; } | ||||||
|  | private: | ||||||
|  |   typedef common::ObSEArrayImpl<ObTableOperationResult, ObTableBatchOperation::COMMON_BATCH_SIZE> BaseType; | ||||||
|  |   ObITableEntityFactory *entity_factory_; | ||||||
|  |   common::ObIAllocator *alloc_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// A table query | ||||||
|  | /// 1. support multi range scan | ||||||
|  | /// 2. support reverse scan | ||||||
|  | /// 3. support secondary index scan | ||||||
|  | class ObTableQuery final | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableQuery() | ||||||
|  |       :deserialize_allocator_(NULL), | ||||||
|  |       key_ranges_(), | ||||||
|  |       select_columns_(), | ||||||
|  |       filter_string_(), | ||||||
|  |       limit_(-1), | ||||||
|  |       offset_(0), | ||||||
|  |       scan_order_(common::ObQueryFlag::Forward), | ||||||
|  |       index_name_(), | ||||||
|  |       batch_size_(-1), | ||||||
|  |       max_result_size_(-1) | ||||||
|  |   {} | ||||||
|  |   ~ObTableQuery() = default; | ||||||
|  |   void reset(); | ||||||
|  |   bool is_valid() const; | ||||||
|  |  | ||||||
|  |   /// add a scan range, the number of scan ranges should be >=1. | ||||||
|  |   int add_scan_range(common::ObNewRange &scan_range); | ||||||
|  |   /// Scan order: Forward (By default) and Reverse. | ||||||
|  |   int set_scan_order(common::ObQueryFlag::ScanOrder scan_order); | ||||||
|  |   /// Set the index to scan, could be 'PRIMARY' (by default) or any other secondary index. | ||||||
|  |   int set_scan_index(const ObString &index_name); | ||||||
|  |   /// Add select columns. | ||||||
|  |   int add_select_column(const ObString &columns); | ||||||
|  |   /// Set the max rows to return. The value of -1 represents there is NO limit. The default value is -1. | ||||||
|  |   /// For htable, set the limit of htable rows for this scan. | ||||||
|  |   int set_limit(int32_t limit); | ||||||
|  |   /// Set the offset to return. The default value is 0. | ||||||
|  |   int set_offset(int32_t offset); | ||||||
|  |   /// Add filter, currently NOT supported. | ||||||
|  |   int set_filter(const ObString &filter); | ||||||
|  |   /// Set max row count of each batch. | ||||||
|  |   /// For htable, set the maximum number of cells to return for each call to next(). | ||||||
|  |   int set_batch(int32_t batch_size); | ||||||
|  |   /// Set the maximum result size. | ||||||
|  |   /// The default is -1; this means that no specific maximum result size will be set for this query. | ||||||
|  |   /// @param max_result_size - The maximum result size in bytes. | ||||||
|  |   int set_max_result_size(int64_t max_result_size); | ||||||
|  |  | ||||||
|  |   const ObIArray<ObString> &get_select_columns() const { return select_columns_; } | ||||||
|  |   const ObIArray<common::ObNewRange> &get_scan_ranges() const { return key_ranges_; } | ||||||
|  |   int32_t get_limit() const { return limit_; } | ||||||
|  |   int32_t get_offset() const { return offset_; } | ||||||
|  |   common::ObQueryFlag::ScanOrder get_scan_order() const { return scan_order_; } | ||||||
|  |   const ObString &get_index_name() const { return index_name_; } | ||||||
|  |   int32_t get_batch() const { return batch_size_; } | ||||||
|  |   int64_t get_max_result_size() const { return max_result_size_; } | ||||||
|  |   int64_t get_range_count() const { return key_ranges_.count(); } | ||||||
|  |   uint64_t get_checksum() const; | ||||||
|  |  | ||||||
|  |   void clear_scan_range() { key_ranges_.reset(); } | ||||||
|  |   void set_deserialize_allocator(common::ObIAllocator *allocator) { deserialize_allocator_ = allocator; } | ||||||
|  |   TO_STRING_KV(K_(key_ranges), | ||||||
|  |                K_(select_columns), | ||||||
|  |                K_(filter_string), | ||||||
|  |                K_(limit), | ||||||
|  |                K_(offset), | ||||||
|  |                K_(scan_order), | ||||||
|  |                K_(index_name), | ||||||
|  |                K_(batch_size), | ||||||
|  |                K_(max_result_size)); | ||||||
|  | public: | ||||||
|  |   static ObString generate_filter_condition(const ObString &column, const ObString &op, const ObObj &value); | ||||||
|  |   static ObString combile_filters(const ObString &filter1, const ObString &op, const ObString &filter2); | ||||||
|  |   static common::ObNewRange generate_prefix_scan_range(const ObRowkey &rowkey_prefix); | ||||||
|  | private: | ||||||
|  |   common::ObIAllocator *deserialize_allocator_; | ||||||
|  |   ObSEArray<common::ObNewRange, 16> key_ranges_; | ||||||
|  |   ObSEArray<ObString, 16> select_columns_; | ||||||
|  |   ObString filter_string_; | ||||||
|  |   int32_t limit_;  // default -1 means unlimited | ||||||
|  |   int32_t offset_; | ||||||
|  |   common::ObQueryFlag::ScanOrder scan_order_; | ||||||
|  |   ObString index_name_; | ||||||
|  |   int32_t batch_size_; | ||||||
|  |   int64_t max_result_size_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | /// result for ObTableQuery | ||||||
|  | class ObTableEntityIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableEntityIterator() = default; | ||||||
|  |   virtual ~ObTableEntityIterator(); | ||||||
|  |   /** | ||||||
|  |    * fetch the next entity | ||||||
|  |    * @return OB_ITER_END when finished | ||||||
|  |    */ | ||||||
|  |   virtual int get_next_entity(const ObITableEntity *&entity) = 0; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObTableQueryResult: public ObTableEntityIterator | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableQueryResult(); | ||||||
|  |   virtual ~ObTableQueryResult() {} | ||||||
|  |   void reset(); | ||||||
|  |   void reset_except_property(); | ||||||
|  |   void rewind(); | ||||||
|  |   virtual int get_next_entity(const ObITableEntity *&entity) override; | ||||||
|  |   int add_property_name(const ObString &name); | ||||||
|  |   int add_row(const common::ObNewRow &row); | ||||||
|  |   int add_all_property(const ObTableQueryResult &other); | ||||||
|  |   int add_all_row(const ObTableQueryResult &other); | ||||||
|  |   int64_t get_row_count() const { return row_count_; } | ||||||
|  |   int64_t get_property_count() const { return properties_names_.count(); } | ||||||
|  |   int64_t get_result_size(); | ||||||
|  |   int get_first_row(common::ObNewRow &row) const; | ||||||
|  |   bool reach_batch_size_or_result_size(const int32_t batch_count, | ||||||
|  |                                        const int64_t max_result_size); | ||||||
|  | private: | ||||||
|  |   static const int64_t MAX_BUF_BLOCK_SIZE = common::OB_MAX_PACKET_BUFFER_LENGTH - (1024*1024LL); | ||||||
|  |   static const int64_t DEFAULT_BUF_BLOCK_SIZE = common::OB_MALLOC_BIG_BLOCK_SIZE - (1024*1024LL); | ||||||
|  |   int alloc_buf_if_need(const int64_t size); | ||||||
|  | private: | ||||||
|  |   common::ObSEArray<ObString, 16> properties_names_;  // serialize | ||||||
|  |   int64_t row_count_;                                 // serialize | ||||||
|  |   common::ObDataBuffer buf_;                          // serialize | ||||||
|  |   common::ObArenaAllocator allocator_; | ||||||
|  |   int64_t fixed_result_size_; | ||||||
|  |   // for deserialize and read | ||||||
|  |   int64_t curr_idx_; | ||||||
|  |   ObTableEntity curr_entity_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | } // end namespace table | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_TABLE_H */ | ||||||
							
								
								
									
										37
									
								
								src/share/table/ob_table_rpc_proxy.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										37
									
								
								src/share/table/ob_table_rpc_proxy.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,37 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_RPC_PROXY_H | ||||||
|  | #define _OB_TABLE_RPC_PROXY_H 1 | ||||||
|  | #include "rpc/obrpc/ob_rpc_proxy.h" | ||||||
|  | #include "share/table/ob_table_rpc_struct.h" | ||||||
|  | #include "share/config/ob_server_config.h" | ||||||
|  | #include "observer/ob_server_struct.h" | ||||||
|  |  | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace obrpc | ||||||
|  | { | ||||||
|  | class ObTableRpcProxy: public obrpc::ObRpcProxy | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   DEFINE_TO(ObTableRpcProxy); | ||||||
|  |   RPC_S(PR5 login, obrpc::OB_TABLE_API_LOGIN, (table::ObTableLoginRequest), table::ObTableLoginResult); | ||||||
|  |   RPC_S(PR5 execute, obrpc::OB_TABLE_API_EXECUTE, (table::ObTableOperationRequest), table::ObTableOperationResult); | ||||||
|  |   RPC_S(PR5 batch_execute, obrpc::OB_TABLE_API_BATCH_EXECUTE, (table::ObTableBatchOperationRequest), table::ObTableBatchOperationResult); | ||||||
|  |   RPC_SS(PR5 execute_query, obrpc::OB_TABLE_API_EXECUTE_QUERY, (table::ObTableQueryRequest), table::ObTableQueryResult); | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | }; // end namespace obrpc | ||||||
|  | }; // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_RPC_PROXY_H */ | ||||||
							
								
								
									
										78
									
								
								src/share/table/ob_table_rpc_struct.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										78
									
								
								src/share/table/ob_table_rpc_struct.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 CLIENT | ||||||
|  | #include "ob_table_rpc_struct.h" | ||||||
|  | using namespace oceanbase::common; | ||||||
|  | using namespace oceanbase::table; | ||||||
|  |  | ||||||
|  | OB_SERIALIZE_MEMBER(ObTableLoginRequest, | ||||||
|  |                     auth_method_, | ||||||
|  |                     client_type_, | ||||||
|  |                     client_version_, | ||||||
|  |                     reserved1_, | ||||||
|  |                     client_capabilities_, | ||||||
|  |                     max_packet_size_, | ||||||
|  |                     reserved2_, | ||||||
|  |                     reserved3_, | ||||||
|  |                     tenant_name_, | ||||||
|  |                     user_name_, | ||||||
|  |                     pass_secret_, | ||||||
|  |                     pass_scramble_, | ||||||
|  |                     database_name_, | ||||||
|  |                     ttl_us_); | ||||||
|  |  | ||||||
|  | OB_SERIALIZE_MEMBER(ObTableLoginResult, | ||||||
|  |                     server_capabilities_, | ||||||
|  |                     reserved1_, | ||||||
|  |                     reserved2_, | ||||||
|  |                     server_version_, | ||||||
|  |                     credential_, | ||||||
|  |                     tenant_id_, | ||||||
|  |                     user_id_, | ||||||
|  |                     database_id_); | ||||||
|  |  | ||||||
|  | OB_SERIALIZE_MEMBER(ObTableOperationRequest, | ||||||
|  |                     credential_, | ||||||
|  |                     table_name_, | ||||||
|  |                     table_id_, | ||||||
|  |                     partition_id_, | ||||||
|  |                     entity_type_, | ||||||
|  |                     table_operation_, | ||||||
|  |                     consistency_level_, | ||||||
|  |                     returning_rowkey_, | ||||||
|  |                     returning_affected_entity_, | ||||||
|  |                     returning_affected_rows_ | ||||||
|  |                     ); | ||||||
|  |  | ||||||
|  | OB_SERIALIZE_MEMBER(ObTableBatchOperationRequest, | ||||||
|  |                     credential_, | ||||||
|  |                     table_name_, | ||||||
|  |                     table_id_, | ||||||
|  |                     entity_type_, | ||||||
|  |                     batch_operation_, | ||||||
|  |                     consistency_level_, | ||||||
|  |                     returning_rowkey_, | ||||||
|  |                     returning_affected_entity_, | ||||||
|  |                     returning_affected_rows_, | ||||||
|  |                     partition_id_ | ||||||
|  |                     ); | ||||||
|  |  | ||||||
|  | OB_SERIALIZE_MEMBER(ObTableQueryRequest, | ||||||
|  |                     credential_, | ||||||
|  |                     table_name_, | ||||||
|  |                     table_id_, | ||||||
|  |                     partition_id_, | ||||||
|  |                     entity_type_, | ||||||
|  |                     consistency_level_, | ||||||
|  |                     query_ | ||||||
|  |                     ); | ||||||
							
								
								
									
										222
									
								
								src/share/table/ob_table_rpc_struct.h
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										222
									
								
								src/share/table/ob_table_rpc_struct.h
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,222 @@ | |||||||
|  | /** | ||||||
|  |  * 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_TABLE_RPC_STRUCT_H | ||||||
|  | #define _OB_TABLE_RPC_STRUCT_H 1 | ||||||
|  | #include "ob_table.h" | ||||||
|  | #include "common/data_buffer.h" | ||||||
|  | namespace oceanbase | ||||||
|  | { | ||||||
|  | namespace common | ||||||
|  | { | ||||||
|  | class ObNewRow; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | namespace table | ||||||
|  | { | ||||||
|  | /// @see PCODE_DEF(OB_TABLE_API_LOGIN, 0x1101) | ||||||
|  | class ObTableLoginRequest final | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   uint8_t auth_method_;  // always 1 for now | ||||||
|  |   uint8_t client_type_;  // 1: libobtable; 2: java client | ||||||
|  |   uint8_t client_version_;  // always 1 for now | ||||||
|  |   uint8_t reserved1_; | ||||||
|  |   uint32_t client_capabilities_; | ||||||
|  |   uint32_t max_packet_size_;  // for stream result | ||||||
|  |   uint32_t reserved2_;  // always 0 for now | ||||||
|  |   uint64_t reserved3_;  // always 0 for now | ||||||
|  |   ObString tenant_name_; | ||||||
|  |   ObString user_name_; | ||||||
|  |   ObString pass_secret_; | ||||||
|  |   ObString pass_scramble_;  // 20 bytes random string | ||||||
|  |   ObString database_name_; | ||||||
|  |   int64_t ttl_us_;  // 0 means no TTL | ||||||
|  | public: | ||||||
|  |   TO_STRING_KV(K_(auth_method), | ||||||
|  |                K_(client_type), | ||||||
|  |                K_(client_version), | ||||||
|  |                K_(reserved1), | ||||||
|  |                K_(client_capabilities), | ||||||
|  |                K_(max_packet_size), | ||||||
|  |                K_(reserved2), | ||||||
|  |                K_(reserved3), | ||||||
|  |                K_(tenant_name), | ||||||
|  |                K_(user_name), | ||||||
|  |                K_(database_name), | ||||||
|  |                K_(ttl_us)); | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObTableLoginResult final | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   uint32_t server_capabilities_; | ||||||
|  |   uint32_t reserved1_;  // always 0 for now | ||||||
|  |   uint64_t reserved2_;  // always 0 for now | ||||||
|  |   ObString server_version_; | ||||||
|  |   ObString credential_; | ||||||
|  |   uint64_t tenant_id_; | ||||||
|  |   uint64_t user_id_; | ||||||
|  |   uint64_t database_id_; | ||||||
|  | public: | ||||||
|  |   TO_STRING_KV(K_(server_capabilities), | ||||||
|  |                K_(reserved1), | ||||||
|  |                K_(reserved2), | ||||||
|  |                K_(server_version), | ||||||
|  |                "credential", common::ObHexStringWrap(credential_), | ||||||
|  |                K_(tenant_id), | ||||||
|  |                K_(user_id), | ||||||
|  |                K_(database_id)); | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | /// @see PCODE_DEF(OB_TABLE_API_EXECUTE, 0x1102) | ||||||
|  | class ObTableOperationRequest final | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableOperationRequest() : credential_(), table_name_(), table_id_(common::OB_INVALID_ID), | ||||||
|  |       partition_id_(common::OB_INVALID_ID), entity_type_(), table_operation_(), | ||||||
|  |       consistency_level_(), returning_rowkey_(false), returning_affected_entity_(false), | ||||||
|  |       returning_affected_rows_(false), | ||||||
|  |       binlog_row_image_type_(ObBinlogRowImageType::FULL) | ||||||
|  |       {} | ||||||
|  |   ~ObTableOperationRequest() {} | ||||||
|  |  | ||||||
|  |   TO_STRING_KV("credential", common::ObHexStringWrap(credential_), | ||||||
|  |                K_(table_name), | ||||||
|  |                K_(table_id), | ||||||
|  |                K_(partition_id), | ||||||
|  |                K_(entity_type), | ||||||
|  |                K_(table_operation), | ||||||
|  |                K_(consistency_level), | ||||||
|  |                K_(returning_rowkey), | ||||||
|  |                K_(returning_affected_entity), | ||||||
|  |                K_(returning_affected_rows)); | ||||||
|  | public: | ||||||
|  |   /// the credential returned when login. | ||||||
|  |   ObString credential_; | ||||||
|  |   /// table name. | ||||||
|  |   ObString table_name_; | ||||||
|  |   /// table id. Set it to gain better performance. If unknown, set it to be OB_INVALID_ID | ||||||
|  |   uint64_t table_id_;  // for optimize purpose | ||||||
|  |   /// partition id. Set it to gain better performance. If unknown, set it to be OB_INVALID_ID | ||||||
|  |   uint64_t partition_id_;  // for optimize purpose | ||||||
|  |   /// entity type. Set it to gain better performance. If unknown, set it to be ObTableEntityType::DYNAMIC. | ||||||
|  |   ObTableEntityType entity_type_;  // for optimize purpose | ||||||
|  |   /// table operation. | ||||||
|  |   ObTableOperation table_operation_; | ||||||
|  |   /// read consistency level. currently only support STRONG. | ||||||
|  |   ObTableConsistencyLevel consistency_level_; | ||||||
|  |   /// Whether return the rowkey, currently the value MUST be false (In the case of Append/Increment the value could be true). | ||||||
|  |   bool returning_rowkey_; | ||||||
|  |   /// Whether return the row which has been modified, currently the value MUST be false (In the case of Append/Increment, the value could be true) | ||||||
|  |   bool returning_affected_entity_; | ||||||
|  |   /// Whether return affected_rows | ||||||
|  |   bool returning_affected_rows_; | ||||||
|  |   /// Whether record the full row in binlog of modification | ||||||
|  |   ObBinlogRowImageType binlog_row_image_type_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | /// batch operation of ONE partition | ||||||
|  | /// @see PCODE_DEF(OB_TABLE_API_BATCH_EXECUTE, 0x1103) | ||||||
|  | class ObTableBatchOperationRequest final | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableBatchOperationRequest() : credential_(), table_name_(), table_id_(common::OB_INVALID_ID), | ||||||
|  |       partition_id_(common::OB_INVALID_ID), entity_type_(), batch_operation_(), | ||||||
|  |       consistency_level_(), returning_rowkey_(false), returning_affected_entity_(false), | ||||||
|  |       returning_affected_rows_(false), | ||||||
|  |       binlog_row_image_type_(ObBinlogRowImageType::FULL) | ||||||
|  |       {} | ||||||
|  |   ~ObTableBatchOperationRequest() {} | ||||||
|  |  | ||||||
|  |   TO_STRING_KV("credential", common::ObHexStringWrap(credential_), | ||||||
|  |                K_(table_name), | ||||||
|  |                K_(table_id), | ||||||
|  |                K_(partition_id), | ||||||
|  |                K_(entity_type), | ||||||
|  |                K_(batch_operation), | ||||||
|  |                K_(consistency_level), | ||||||
|  |                K_(returning_rowkey), | ||||||
|  |                K_(returning_affected_entity), | ||||||
|  |                K_(returning_affected_rows)); | ||||||
|  | public: | ||||||
|  |   ObString credential_; | ||||||
|  |   ObString table_name_; | ||||||
|  |   uint64_t table_id_;  // for optimize purpose | ||||||
|  |   /// partition id. Set it to gain better performance. If unknown, set it to be OB_INVALID_ID | ||||||
|  |   uint64_t partition_id_;  // for optimize purpose | ||||||
|  |   ObTableEntityType entity_type_;  // for optimize purpose | ||||||
|  |   ObTableBatchOperation batch_operation_; | ||||||
|  |   // Only support STRONG | ||||||
|  |   ObTableConsistencyLevel consistency_level_; | ||||||
|  |   // Only support false (Support true for only Append/Increment) | ||||||
|  |   bool returning_rowkey_; | ||||||
|  |   // Only support false (Support true for only Append/Increment) | ||||||
|  |   bool returning_affected_entity_; | ||||||
|  |   /// whether return affected_rows | ||||||
|  |   bool returning_affected_rows_; | ||||||
|  |   /// Whether record the full row in binlog of modification | ||||||
|  |   ObBinlogRowImageType binlog_row_image_type_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | //////////////////////////////////////////////////////////////// | ||||||
|  | // @see PCODE_DEF(OB_TABLE_API_EXECUTE_QUERY, 0x1104) | ||||||
|  | class ObTableQueryRequest final | ||||||
|  | { | ||||||
|  |   OB_UNIS_VERSION(1); | ||||||
|  | public: | ||||||
|  |   ObTableQueryRequest() | ||||||
|  |       :table_id_(common::OB_INVALID_ID), | ||||||
|  |        partition_id_(common::OB_INVALID_ID), | ||||||
|  |        entity_type_(ObTableEntityType::ET_DYNAMIC), | ||||||
|  |        consistency_level_(ObTableConsistencyLevel::STRONG) | ||||||
|  |   {} | ||||||
|  |  | ||||||
|  |   TO_STRING_KV("credential", common::ObHexStringWrap(credential_), | ||||||
|  |                K_(table_name), | ||||||
|  |                K_(table_id), | ||||||
|  |                K_(partition_id), | ||||||
|  |                K_(entity_type), | ||||||
|  |                K_(consistency_level), | ||||||
|  |                K_(query)); | ||||||
|  | public: | ||||||
|  |   ObString credential_; | ||||||
|  |   ObString table_name_; | ||||||
|  |   uint64_t table_id_;  // for optimize purpose | ||||||
|  |   /// partition id. Set it to gain better performance. If unknown, set it to be OB_INVALID_ID | ||||||
|  |   uint64_t partition_id_;  // for optimize purpose | ||||||
|  |   ObTableEntityType entity_type_;  // for optimize purpose | ||||||
|  |   // only support STRONG | ||||||
|  |   ObTableConsistencyLevel consistency_level_; | ||||||
|  |   ObTableQuery query_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class ObTableQueryResultIterator | ||||||
|  | { | ||||||
|  | public: | ||||||
|  |   ObTableQueryResultIterator() {} | ||||||
|  |   virtual ~ObTableQueryResultIterator() {} | ||||||
|  |   virtual int get_next_result(ObTableQueryResult *&one_result) = 0; | ||||||
|  |   virtual bool has_more_result() const = 0; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  |  | ||||||
|  | } // end namespace table | ||||||
|  | } // end namespace oceanbase | ||||||
|  |  | ||||||
|  | #endif /* _OB_TABLE_RPC_STRUCT_H */ | ||||||
| @ -4,3 +4,4 @@ ob_unittest(test_omt_worker omt/test_worker.cpp) | |||||||
| ob_unittest(test_worker_pool omt/test_worker_pool.cpp) | ob_unittest(test_worker_pool omt/test_worker_pool.cpp) | ||||||
| ob_unittest(test_token_calcer omt/test_token_calcer.cpp) | ob_unittest(test_token_calcer omt/test_token_calcer.cpp) | ||||||
| ob_unittest(test_information_schema) | ob_unittest(test_information_schema) | ||||||
|  | ob_unittest(test_tableapi tableapi/test_tableapi.cpp) | ||||||
|  | |||||||
							
								
								
									
										372
									
								
								unittest/observer/tableapi/test_tableapi.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										372
									
								
								unittest/observer/tableapi/test_tableapi.cpp
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,372 @@ | |||||||
|  | /** | ||||||
|  |  * 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 <gtest/gtest.h> | ||||||
|  | #include <gmock/gmock.h> | ||||||
|  | #define private public | ||||||
|  |  | ||||||
|  | #include "share/schema/ob_schema_getter_guard.h" | ||||||
|  | #include "observer/ob_server.h" | ||||||
|  | #include "observer/table/ob_table_api_row_iterator.h" | ||||||
|  | #include "observer/table/ob_table_service.h" | ||||||
|  |  | ||||||
|  | namespace oceanbase { | ||||||
|  |  | ||||||
|  | namespace observer { | ||||||
|  |  | ||||||
|  | // #define UNUSED(x) (x) | ||||||
|  | static const int64_t TEST_COLUMN_CNT = 3; | ||||||
|  | static const int64_t TEST_ROWKEY_COLUMN_CNT = 1; | ||||||
|  |  | ||||||
|  | class TestTableApi : public::testing::Test { | ||||||
|  | public: | ||||||
|  |   TestTableApi(); | ||||||
|  |   virtual ~TestTableApi() | ||||||
|  |   {} | ||||||
|  |   virtual void SetUp(); | ||||||
|  |   virtual void TearDown(); | ||||||
|  |  | ||||||
|  | private: | ||||||
|  |   void prepare_schema(); | ||||||
|  |  | ||||||
|  | protected: | ||||||
|  |   ObArenaAllocator allocator_; | ||||||
|  |   ObTableSchema table_schema_; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | class TestObTableApiRowIterator : public ObTableApiRowIterator { | ||||||
|  | public: | ||||||
|  |   void set_table_schema(const ObTableSchema *table_schema) { table_schema_ = table_schema; } | ||||||
|  |   void set_is_init(bool is_init) { is_inited_ = is_init; } | ||||||
|  |   void set_has_gen_column(bool is_has) { has_generate_column_ = is_has; } | ||||||
|  |   void set_entity(table::ObITableEntity *entity) { _entity = entity; } | ||||||
|  |   int open() { return cons_all_columns(*_entity, true); } | ||||||
|  |   virtual int get_next_row(ObNewRow*& row); | ||||||
|  |   int cons_row(const table::ObITableEntity &entity, common::ObNewRow *&row); | ||||||
|  |  | ||||||
|  | private:  | ||||||
|  |   table::ObITableEntity *_entity; | ||||||
|  | }; | ||||||
|  |  | ||||||
|  | int TestObTableApiRowIterator::get_next_row(ObNewRow *&row) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   row_allocator_.reuse(); | ||||||
|  |   if (OB_ISNULL(_entity)) { | ||||||
|  |     ret = OB_NOT_INIT; | ||||||
|  |     COMMON_LOG(INFO, "The entity is null, ", K(ret)); | ||||||
|  |   } else if (OB_FAIL(cons_row(*_entity, row))) { | ||||||
|  |     COMMON_LOG(INFO, "Fail to construct insert row, ", K(ret)); | ||||||
|  |   } else { | ||||||
|  |     //success | ||||||
|  |     COMMON_LOG(INFO, "Api insert row iter, ", K(*row)); | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | int TestObTableApiRowIterator::cons_row(const table::ObITableEntity &entity, common::ObNewRow *&row) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (OB_SUCCESS != entity_to_row(entity, row_objs_)) { | ||||||
|  |     COMMON_LOG(INFO, "Fail to generate row from entity", K(ret)); | ||||||
|  |   } else { | ||||||
|  |     const int64_t N = missing_default_objs_.count(); | ||||||
|  |     for (int64_t i = 0; OB_SUCC(ret) && i < N; i++) { | ||||||
|  |       if (OB_FAIL(row_objs_.push_back(missing_default_objs_.at(i)))) { | ||||||
|  |         COMMON_LOG(INFO, "Fail to push default value to row, ", K(ret)); | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |     if (OB_SUCC(ret)) { | ||||||
|  |       row_.assign(&row_objs_.at(0), row_objs_.count()); | ||||||
|  |       if (has_generate_column_ && OB_FAIL(fill_generate_columns(row_))) { | ||||||
|  |         COMMON_LOG(INFO, "Fail to fill generate columns, ", K(ret)); | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |     if (OB_SUCC(ret)) { | ||||||
|  |       if (OB_FAIL(check_row(row_))) { | ||||||
|  |         COMMON_LOG(INFO, "Fail to check row, ", K(ret), K_(row)); | ||||||
|  |       } else { | ||||||
|  |         row = &row_; | ||||||
|  |       } | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
|  | TestTableApi::TestTableApi() : allocator_(ObModIds::TEST) | ||||||
|  | {} | ||||||
|  |  | ||||||
|  | void TestTableApi::SetUp() | ||||||
|  | { | ||||||
|  |   prepare_schema(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void TestTableApi::TearDown() | ||||||
|  | { | ||||||
|  |   table_schema_.reset(); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | void TestTableApi::prepare_schema() | ||||||
|  | { | ||||||
|  |   ObColumnSchemaV2 column; | ||||||
|  |   int64_t table_id = 3001; | ||||||
|  |   int64_t micro_block_size = 16 * 1024; | ||||||
|  |   //init table schema | ||||||
|  |   table_schema_.reset(); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, table_schema_.set_table_name("test_tableapi")); | ||||||
|  |   table_schema_.set_tenant_id(1); | ||||||
|  |   table_schema_.set_tablegroup_id(1); | ||||||
|  |   table_schema_.set_database_id(1); | ||||||
|  |   table_schema_.set_table_id(table_id); | ||||||
|  |   table_schema_.set_rowkey_column_num(TEST_ROWKEY_COLUMN_CNT); | ||||||
|  |   table_schema_.set_max_used_column_id(TEST_COLUMN_CNT); | ||||||
|  |   table_schema_.set_block_size(micro_block_size); | ||||||
|  |   table_schema_.set_compress_func_name("none"); | ||||||
|  |   //init column | ||||||
|  |   char name[OB_MAX_FILE_NAME_LENGTH]; | ||||||
|  |   memset(name, 0, sizeof(name)); | ||||||
|  |  | ||||||
|  |   for(int32_t i = 0; i < TEST_COLUMN_CNT; ++i) { | ||||||
|  |     ObObjType obj_type = static_cast<ObObjType>(ObIntType); | ||||||
|  |     column.reset(); | ||||||
|  |     column.set_table_id(table_id); | ||||||
|  |     column.set_column_id(i + OB_APP_MIN_COLUMN_ID); | ||||||
|  |     sprintf(name, "c%d", i); | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, column.set_column_name(name)); | ||||||
|  |     column.set_data_type(obj_type); | ||||||
|  |     column.set_collation_type(CS_TYPE_UTF8MB4_GENERAL_CI); | ||||||
|  |     column.set_data_length(1); | ||||||
|  |     if (i < TEST_ROWKEY_COLUMN_CNT) { | ||||||
|  |       column.set_rowkey_position(i + 1); | ||||||
|  |     } else { | ||||||
|  |       column.set_rowkey_position(0); | ||||||
|  |     } | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, table_schema_.add_column(column)); | ||||||
|  |   } | ||||||
|  |   // check rowkey column | ||||||
|  |   const ObRowkeyInfo& rowkey_info = table_schema_.get_rowkey_info(); | ||||||
|  |   for (int64_t i = 0; i < rowkey_info.get_size(); ++i) { | ||||||
|  |     uint64_t column_id = OB_INVALID_ID; | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, rowkey_info.get_column_id(i, column_id)); | ||||||
|  |   } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | TEST_F(TestTableApi, entity_factory) | ||||||
|  | { | ||||||
|  |   table::ObTableEntityFactory<table::ObTableEntity> entity_factory; | ||||||
|  |   static const int64_t N = 100; | ||||||
|  |   static const int64_t R = 3; | ||||||
|  |   for (int round = 0; round < R; ++round) { | ||||||
|  |     for (int i = 0; i < N; ++i) { | ||||||
|  |       table::ObITableEntity *entity = entity_factory.alloc(); | ||||||
|  |       ASSERT_TRUE(NULL != entity); | ||||||
|  |     } // end for | ||||||
|  |     fprintf(stderr, "used=%ld free=%ld mem_total=%ld mem_used=%ld\n", | ||||||
|  |             entity_factory.get_used_count(), entity_factory.get_free_count(), | ||||||
|  |             entity_factory.get_total_mem(), entity_factory.get_used_mem()); | ||||||
|  |     entity_factory.free_and_reuse(); | ||||||
|  |     fprintf(stderr, "used=%ld free=%ld mem_total=%ld mem_used=%ld\n", | ||||||
|  |             entity_factory.get_used_count(), entity_factory.get_free_count(), | ||||||
|  |             entity_factory.get_total_mem(), entity_factory.get_used_mem()); | ||||||
|  |   } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | TEST_F(TestTableApi, serialize_batch_result) | ||||||
|  | { | ||||||
|  |   ObTableBatchOperationResult result; | ||||||
|  |   table::ObTableEntity result_entity; | ||||||
|  |   ObTableOperationResult single_op_result; | ||||||
|  |   single_op_result.set_entity(result_entity); | ||||||
|  |   single_op_result.set_errno(1234); | ||||||
|  |   single_op_result.set_type(table::ObTableOperationType::INSERT_OR_UPDATE); | ||||||
|  |   single_op_result.set_affected_rows(4321); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, result.push_back(single_op_result)); | ||||||
|  |   int64_t expected_len = result.get_serialize_size(); | ||||||
|  |   char buf[1024]; | ||||||
|  |   int64_t pos = 0; | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, result.serialize(buf, 1024, pos)); | ||||||
|  |   ASSERT_EQ(expected_len, pos); | ||||||
|  |  | ||||||
|  |   ObTableBatchOperationResult result2; | ||||||
|  |   table::ObTableEntityFactory<table::ObTableEntity> entity_factory; | ||||||
|  |   result2.set_entity_factory(&entity_factory); | ||||||
|  |   int64_t data_len = pos; | ||||||
|  |   pos = 0; | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, result2.deserialize(buf, data_len, pos)); | ||||||
|  |   ASSERT_EQ(1, result2.count()); | ||||||
|  |   ASSERT_EQ(1234, result2.at(0).get_errno()); | ||||||
|  |   ASSERT_EQ(4321, result2.at(0).get_affected_rows()); | ||||||
|  |   ASSERT_EQ(table::ObTableOperationType::INSERT_OR_UPDATE, result2.at(0).type()); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | TEST_F(TestTableApi, serialize_table_query) | ||||||
|  | { | ||||||
|  |   ObTableQuery query; | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query.add_select_column("c1")); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query.add_select_column("c2")); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query.add_select_column("c3")); | ||||||
|  |  | ||||||
|  |   ObObj pk_objs_start[2]; | ||||||
|  |   pk_objs_start[0].set_int(0); | ||||||
|  |   pk_objs_start[1].set_min_value(); | ||||||
|  |   ObObj pk_objs_end[2]; | ||||||
|  |   pk_objs_end[0].set_int(0); | ||||||
|  |   pk_objs_end[1].set_max_value(); | ||||||
|  |   ObNewRange range; | ||||||
|  |   range.start_key_.assign(pk_objs_start, 2); | ||||||
|  |   range.end_key_.assign(pk_objs_end, 2); | ||||||
|  |   range.border_flag_.set_inclusive_start(); | ||||||
|  |   range.border_flag_.set_inclusive_end(); | ||||||
|  |  | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query.add_scan_range(range)); | ||||||
|  |   int64_t serialize_len = query.get_serialize_size(); | ||||||
|  |   fprintf(stderr, "serialize_size=%ld\n", serialize_len); | ||||||
|  |   char buf[1024]; | ||||||
|  |   int64_t pos = 0; | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query.serialize(buf, 1024, pos)); | ||||||
|  |   ASSERT_EQ(pos, serialize_len); | ||||||
|  |  | ||||||
|  |   ObTableQuery query2; | ||||||
|  |   ObArenaAllocator alloc; | ||||||
|  |   query2.set_deserialize_allocator(&alloc); | ||||||
|  |   pos = 0; | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query2.deserialize(buf, serialize_len, pos)); | ||||||
|  |   const ObIArray<ObString> &select_columns = query2.get_select_columns(); | ||||||
|  |   const ObIArray<ObNewRange> &scan_ranges = query2.get_scan_ranges(); | ||||||
|  |   ASSERT_EQ(3, select_columns.count()); | ||||||
|  |   ASSERT_EQ(1, scan_ranges.count()); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | TEST_F(TestTableApi, serialize_query_result) | ||||||
|  | { | ||||||
|  |   ObTableQueryResult query_result; | ||||||
|  |   ObObj objs[3]; | ||||||
|  |   objs[0].set_int(123); | ||||||
|  |   objs[1].set_null(); | ||||||
|  |   objs[2].set_varchar(ObString::make_string("serialize_query_result")); | ||||||
|  |   ObNewRow row; | ||||||
|  |   row.assign(objs, 3); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query_result.add_property_name("c1")); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query_result.add_property_name("c2")); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query_result.add_property_name("c3")); | ||||||
|  |   for (int64_t i = 0; i < 1024; ++i) { | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, query_result.add_row(row)); | ||||||
|  |   } | ||||||
|  |   ASSERT_EQ(1024, query_result.get_row_count()); | ||||||
|  |   ASSERT_EQ(3, query_result.get_property_count()); | ||||||
|  |   // serialize | ||||||
|  |   char *buf = static_cast<char*>(ob_malloc(OB_MALLOC_BIG_BLOCK_SIZE, ObModIds::TEST)); | ||||||
|  |   ASSERT_TRUE(nullptr != buf); | ||||||
|  |   int64_t pos = 0; | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query_result.serialize(buf, OB_MALLOC_BIG_BLOCK_SIZE, pos)); | ||||||
|  |   ASSERT_EQ(pos, query_result.get_serialize_size()); | ||||||
|  |   fprintf(stderr, "serialize_size=%ld\n", pos); | ||||||
|  |   // deserialize & check | ||||||
|  |   ObTableQueryResult query_result2; | ||||||
|  |   int64_t data_len = pos; | ||||||
|  |   pos = 0; | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, query_result2.deserialize(buf, data_len, pos)); | ||||||
|  |   ASSERT_EQ(1024, query_result2.get_row_count()); | ||||||
|  |   ASSERT_EQ(3, query_result2.get_property_count()); | ||||||
|  |   const table::ObITableEntity *entity = NULL; | ||||||
|  |   for (int64_t i = 0; i < 1024; ++i) { | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, query_result2.get_next_entity(entity)); | ||||||
|  |     ASSERT_TRUE(NULL != entity); | ||||||
|  |     ASSERT_EQ(3, entity->get_properties_count()); | ||||||
|  |     ObObj value; | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, entity->get_property("c1", value)); | ||||||
|  |     ASSERT_EQ(123, value.get_int()); | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, entity->get_property("c2", value)); | ||||||
|  |     ASSERT_TRUE(value.is_null()); | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, entity->get_property("c3", value)); | ||||||
|  |     ObString str; | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, value.get_varchar(str)); | ||||||
|  |     ASSERT_TRUE(str == ObString::make_string("serialize_query_result")); | ||||||
|  |   } | ||||||
|  |   ASSERT_EQ(OB_ITER_END, query_result2.get_next_entity(entity)); | ||||||
|  |   // cleanup | ||||||
|  |   if (NULL != buf) { | ||||||
|  |     ob_free(buf); | ||||||
|  |     buf = NULL; | ||||||
|  |   } | ||||||
|  | } | ||||||
|  |  | ||||||
|  | TEST_F(TestTableApi, table_entity) | ||||||
|  | { | ||||||
|  |   int ret; | ||||||
|  |   ObSEArray<ObString, 1> ppts; | ||||||
|  |   // set row key | ||||||
|  |   ObObj key_objs[3]; | ||||||
|  |   key_objs[0].set_varbinary("table_entity"); | ||||||
|  |   key_objs[1].set_varchar("hi"); | ||||||
|  |   key_objs[2].set_int(1); | ||||||
|  |   ObRowkey rk(key_objs, 3); | ||||||
|  |   // cons entity | ||||||
|  |   table::ObTableEntity entity; | ||||||
|  |   ObObj value; | ||||||
|  |   entity.set_rowkey(rk); | ||||||
|  |   ASSERT_EQ(3, entity.get_rowkey_size()); | ||||||
|  |   ASSERT_EQ(0, entity.get_rowkey_value(2, value)); | ||||||
|  |   ASSERT_EQ(1, value.get_int()); | ||||||
|  |   // properaties | ||||||
|  |   value.set_varchar("value"); | ||||||
|  |   value.set_collation_type(CS_TYPE_UTF8MB4_GENERAL_CI); | ||||||
|  |   ASSERT_EQ(0, entity.set_property("c1", value)); | ||||||
|  |   ASSERT_EQ(0, entity.get_property("c1", value)); | ||||||
|  |   ASSERT_EQ(ObString::make_string("value"), value.get_varchar()); | ||||||
|  |   ASSERT_EQ(0, entity.get_properties_names(ppts)); | ||||||
|  |   ASSERT_EQ(1, ppts.count()); | ||||||
|  |   ASSERT_EQ(1, entity.get_properties_count()); | ||||||
|  |   // reset entity | ||||||
|  |   entity.reset(); | ||||||
|  |   ASSERT_TRUE(entity.is_empty()); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | TEST_F(TestTableApi, open_and_get_next_row) | ||||||
|  | { | ||||||
|  |   ObTableOperation table_operation; | ||||||
|  |   TestObTableApiRowIterator row_iterator; | ||||||
|  |  | ||||||
|  |   row_iterator.set_is_init(true); | ||||||
|  |   row_iterator.set_has_gen_column(false); | ||||||
|  |   row_iterator.set_table_schema(&table_schema_); | ||||||
|  |  | ||||||
|  |   table::ObTableEntity entity; | ||||||
|  |   // set rowkey | ||||||
|  |   ObObj key_objs[1]; | ||||||
|  |   key_objs[0].set_int(1); | ||||||
|  |   ObRowkey rk(key_objs, 1); | ||||||
|  |   entity.set_rowkey(rk); | ||||||
|  |   // set properties | ||||||
|  |   ObObj value; | ||||||
|  |   value.set_int(111); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, entity.set_property("c1", value)); | ||||||
|  |   value.set_int(222); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, entity.set_property("c2", value)); | ||||||
|  |    | ||||||
|  |   ObNewRow *row = nullptr; | ||||||
|  |   row_iterator.set_entity(&entity); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, row_iterator.open()); | ||||||
|  |   ASSERT_EQ(OB_SUCCESS, row_iterator.get_next_row(row)); | ||||||
|  | } | ||||||
|  |  | ||||||
|  | }  // namespace observer | ||||||
|  | }  // namespace oceanbase | ||||||
|  |  | ||||||
|  | int main(int argc, char** argv) | ||||||
|  | { | ||||||
|  |   oceanbase::common::ObLogger::get_logger().set_log_level("INFO"); | ||||||
|  |   OB_LOGGER.set_log_level("INFO"); | ||||||
|  |   OB_LOGGER.set_file_name("test_observer.log", true); | ||||||
|  |   ::testing::InitGoogleTest(&argc, argv); | ||||||
|  |   return RUN_ALL_TESTS(); | ||||||
|  | } | ||||||
		Reference in New Issue
	
	Block a user
	 xj0
					xj0