From 895f700d99533787ffa2a036038e0f0c6cae56df Mon Sep 17 00:00:00 2001 From: xj0 Date: Wed, 11 Aug 2021 19:24:11 +0800 Subject: [PATCH] add tableapi to opensource release --- deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h | 1 - src/observer/CMakeLists.txt | 10 + src/observer/ob_server.cpp | 11 +- src/observer/ob_server.h | 4 + src/observer/ob_server_struct.cpp | 1 + src/observer/ob_server_struct.h | 2 + src/observer/ob_srv_xlator_partition.cpp | 11 + src/observer/table/ob_rpc_async_response.h | 236 ++ .../table/ob_table_api_row_iterator.cpp | 1489 ++++++++++++ .../table/ob_table_api_row_iterator.h | 256 +++ .../ob_table_batch_execute_processor.cpp | 490 ++++ .../table/ob_table_batch_execute_processor.h | 65 + src/observer/table/ob_table_end_trans_cb.cpp | 162 ++ src/observer/table/ob_table_end_trans_cb.h | 90 + .../table/ob_table_execute_processor.cpp | 500 ++++ .../table/ob_table_execute_processor.h | 70 + .../table/ob_table_query_processor.cpp | 201 ++ src/observer/table/ob_table_query_processor.h | 53 + src/observer/table/ob_table_rpc_processor.cpp | 1014 ++++++++ src/observer/table/ob_table_rpc_processor.h | 220 ++ .../table/ob_table_rpc_processor_util.h | 202 ++ src/observer/table/ob_table_service.cpp | 2044 +++++++++++++++++ src/observer/table/ob_table_service.h | 320 +++ src/share/CMakeLists.txt | 5 + src/share/table/ob_table.cpp | 1381 +++++++++++ src/share/table/ob_table.h | 658 ++++++ src/share/table/ob_table_rpc_proxy.h | 37 + src/share/table/ob_table_rpc_struct.cpp | 78 + src/share/table/ob_table_rpc_struct.h | 222 ++ unittest/observer/CMakeLists.txt | 1 + unittest/observer/tableapi/test_tableapi.cpp | 372 +++ 31 files changed, 10203 insertions(+), 3 deletions(-) create mode 100644 src/observer/table/ob_rpc_async_response.h create mode 100644 src/observer/table/ob_table_api_row_iterator.cpp create mode 100644 src/observer/table/ob_table_api_row_iterator.h create mode 100644 src/observer/table/ob_table_batch_execute_processor.cpp create mode 100644 src/observer/table/ob_table_batch_execute_processor.h create mode 100644 src/observer/table/ob_table_end_trans_cb.cpp create mode 100644 src/observer/table/ob_table_end_trans_cb.h create mode 100644 src/observer/table/ob_table_execute_processor.cpp create mode 100644 src/observer/table/ob_table_execute_processor.h create mode 100644 src/observer/table/ob_table_query_processor.cpp create mode 100644 src/observer/table/ob_table_query_processor.h create mode 100644 src/observer/table/ob_table_rpc_processor.cpp create mode 100644 src/observer/table/ob_table_rpc_processor.h create mode 100644 src/observer/table/ob_table_rpc_processor_util.h create mode 100644 src/observer/table/ob_table_service.cpp create mode 100644 src/observer/table/ob_table_service.h create mode 100644 src/share/table/ob_table.cpp create mode 100644 src/share/table/ob_table.h create mode 100644 src/share/table/ob_table_rpc_proxy.h create mode 100644 src/share/table/ob_table_rpc_struct.cpp create mode 100644 src/share/table/ob_table_rpc_struct.h create mode 100644 unittest/observer/tableapi/test_tableapi.cpp diff --git a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h index 60e4485733..9de069d8ae 100644 --- a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h +++ b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h @@ -578,7 +578,6 @@ PCODE_DEF(OB_TABLE_API_LOGIN, 0x1101) PCODE_DEF(OB_TABLE_API_EXECUTE, 0x1102) PCODE_DEF(OB_TABLE_API_BATCH_EXECUTE, 0x1103) PCODE_DEF(OB_TABLE_API_EXECUTE_QUERY, 0x1104) -PCODE_DEF(OB_TABLE_API_QUERY_AND_MUTATE, 0x1105) // Event Job API PCODE_DEF(OB_RUN_EVENT_JOB, 0x1201) diff --git a/src/observer/CMakeLists.txt b/src/observer/CMakeLists.txt index e726e29f1d..6f7e45cfff 100644 --- a/src/observer/CMakeLists.txt +++ b/src/observer/CMakeLists.txt @@ -240,6 +240,16 @@ ob_set_subtarget(ob_server vt 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_struct.h ob_uniq_task_queue.h diff --git a/src/observer/ob_server.cpp b/src/observer/ob_server.cpp index 53038741a5..f00121e24b 100644 --- a/src/observer/ob_server.cpp +++ b/src/observer/ob_server.cpp @@ -45,6 +45,7 @@ #include "sql/ob_sql_init.h" #include "sql/ob_sql_task.h" #include "observer/ob_server.h" +#include "observer/table/ob_table_rpc_processor.h" #include "sql/ob_sql_init.h" #include "sql/dtl/ob_dtl.h" #include "sql/ob_sql_init.h" @@ -129,6 +130,7 @@ ObServer::ObServer() vt_data_service_(root_service_, self_addr_, &config_), cache_size_calculator_(), weak_read_service_(), + table_service_(), cgroup_ctrl_(), start_time_(ObTimeUtility::current_time()), 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_FAIL(init_loaddata_global_stat())) { - LOG_WARN("fail to init global load data stat map", K(ret)); + if (OB_FAIL(ObTableApiProcessorBase::init_session())) { + 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)); } else if (OB_FAIL(ObCompatModeGetter::instance().init(&sql_proxy_))) { 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())) { LOG_WARN("failed to init timer monitor", K(ret)); } 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_v2(OB_INVALID_VERSION); gctx_.weak_read_service_ = &weak_read_service_; + gctx_.table_service_ = &table_service_; gctx_.cgroup_ctrl_ = &cgroup_ctrl_; gctx_.schema_status_proxy_ = &schema_status_proxy_; (void)gctx_.set_upgrade_stage(obrpc::OB_UPGRADE_STAGE_INVALID); diff --git a/src/observer/ob_server.h b/src/observer/ob_server.h index af61e545ae..16996ebb1c 100644 --- a/src/observer/ob_server.h +++ b/src/observer/ob_server.h @@ -19,6 +19,7 @@ #include "share/stat/ob_user_tab_col_statistics.h" #include "share/stat/ob_opt_stat_service.h" +#include "observer/table/ob_table_service.h" #include "sql/ob_sql.h" #include "sql/engine/cmd/ob_load_data_rpc.h" #include "sql/ob_query_exec_ctx_mgr.h" @@ -388,6 +389,9 @@ private: // Weakly Consistent Read Service transaction::ObWeakReadService weak_read_service_; + // table service + ObTableService table_service_; + // Tenant isolation resource management omt::ObCgroupCtrl cgroup_ctrl_; diff --git a/src/observer/ob_server_struct.cpp b/src/observer/ob_server_struct.cpp index 5212d1edef..a3a0325a16 100644 --- a/src/observer/ob_server_struct.cpp +++ b/src/observer/ob_server_struct.cpp @@ -266,6 +266,7 @@ ObGlobalContext& ObGlobalContext::operator=(const ObGlobalContext& other) sort_dir_ = other.sort_dir_; diag_ = other.diag_; scramble_rand_ = other.scramble_rand_; + table_service_ = other.table_service_; cgroup_ctrl_ = other.cgroup_ctrl_; inited_ = other.inited_; split_schema_version_ = other.split_schema_version_; diff --git a/src/observer/ob_server_struct.h b/src/observer/ob_server_struct.h index 53ecf259d4..68fc194d63 100644 --- a/src/observer/ob_server_struct.h +++ b/src/observer/ob_server_struct.h @@ -84,6 +84,7 @@ class ObCgroupCtrl; namespace observer { class ObService; class ObVTIterCreator; +class ObTableService; class ObServerOptions { public: @@ -197,6 +198,7 @@ struct ObGlobalContext { common::ObString* sort_dir_; obmysql::ObDiag* diag_; common::ObMysqlRandom* scramble_rand_; + ObTableService* table_service_; omt::ObCgroupCtrl* cgroup_ctrl_; bool inited_; int64_t split_schema_version_; diff --git a/src/observer/ob_srv_xlator_partition.cpp b/src/observer/ob_srv_xlator_partition.cpp index ad73165a6d..c9d21765e0 100644 --- a/src/observer/ob_srv_xlator_partition.cpp +++ b/src/observer/ob_srv_xlator_partition.cpp @@ -37,6 +37,11 @@ #include "observer/ob_rpc_processor_simple.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::observer; using namespace oceanbase::lib; @@ -125,6 +130,12 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator* xlator) // SQL Estimate RPC_PROCESSOR(ObEstimatePartitionRowsP, gctx_); + // table api + RPC_PROCESSOR(ObTableLoginP, gctx_); + RPC_PROCESSOR(ObTableApiExecuteP, gctx_); + RPC_PROCESSOR(ObTableBatchExecuteP, gctx_); + RPC_PROCESSOR(ObTableQueryP, gctx_); + // HA GTS RPC_PROCESSOR(ObHaGtsPingRequestP, gctx_); RPC_PROCESSOR(ObHaGtsGetRequestP, gctx_); diff --git a/src/observer/table/ob_rpc_async_response.h b/src/observer/table/ob_rpc_async_response.h new file mode 100644 index 0000000000..0760e6c99e --- /dev/null +++ b/src/observer/table/ob_rpc_async_response.h @@ -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 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 +char *ObRpcAsyncResponse::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(size)); + } + return static_cast(buf); +} + +template +int ObRpcAsyncResponse::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 +int ObRpcAsyncResponse::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(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(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 +int ObRpcAsyncResponse::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(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 */ diff --git a/src/observer/table/ob_table_api_row_iterator.cpp b/src/observer/table/ob_table_api_row_iterator.cpp new file mode 100644 index 0000000000..84afb455cc --- /dev/null +++ b/src/observer/table/ob_table_api_row_iterator.cpp @@ -0,0 +1,1489 @@ +/** + * 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_api_row_iterator.h" +#include "ob_table_rpc_processor.h" +#include "observer/ob_service.h" +#include "sql/ob_sql_utils.h" +#include "sql/engine/expr/ob_expr_add.h" +#include "sql/resolver/expr/ob_raw_expr_util.h" +#include "storage/ob_partition_service.h" + +using namespace oceanbase::common; +using namespace oceanbase::table; +using namespace oceanbase::share; +using namespace oceanbase::share::schema; +using namespace oceanbase::sql; + + +namespace oceanbase { +namespace observer { + +/** + * -------------------------------------------------------ObTableApiRowIterator-------------------------------------------------------- + */ +ObTableApiRowIterator::ObTableApiRowIterator() + : part_service_(NULL), + schema_service_(NULL), + ctx_(NULL), + schema_guard_(), + table_schema_(NULL), + table_id_(0), + tenant_id_(0), + schema_version_(0), + rowkey_column_cnt_(0), + properties_(), + column_ids_(), + columns_type_(), + column_descs_(), + row_objs_(), + missing_default_objs_(), + generate_column_exprs_(), + generate_column_idxs_(), + expr_ctx_(), + row_(), + stmt_allocator_(ObModIds::TABLE_BATCH_OPERATION), + row_allocator_(ObModIds::TABLE_BATCH_OPERATION), + entity_(NULL), + has_generate_column_(false), + is_inited_(false) +{ +} + +ObTableApiRowIterator::~ObTableApiRowIterator() +{ + if (has_generate_column_) { + sql::ObSQLUtils::destruct_default_expr_context(expr_ctx_); + has_generate_column_ = false; + } +} + +int ObTableApiRowIterator::init( + storage::ObPartitionService &partition_service, + share::schema::ObMultiVersionSchemaService &schema_service, + ObTableServiceCtx &ctx) +{ + int ret = OB_SUCCESS; + + if (is_inited_) { + ret = OB_INIT_TWICE; + LOG_WARN("The table api row iterator has been inited, ", K(ret)); + } else if (OB_FAIL(schema_service.get_schema_guard(schema_guard_))) { + LOG_WARN("failed to get schema guard", K(ret)); + } else if (OB_FAIL(schema_guard_.get_table_schema(ctx.param_.table_id_, table_schema_))) { + LOG_WARN("get table schema failed", K(ctx.param_.table_id_), K(ret)); + } else if (OB_ISNULL(table_schema_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("NULL ptr", K(ret), K(table_schema_)); + } else if (OB_FAIL(check_table_supported(table_schema_))) { + LOG_WARN("check table support failed", K(ret)); + } else { + table_id_ = ctx.param_.table_id_; + tenant_id_ = extract_tenant_id(table_id_); + schema_version_ = table_schema_->get_schema_version(); + rowkey_column_cnt_ = table_schema_->get_rowkey_column_num(); + part_service_ = &partition_service; + ctx_ = &ctx; + stmt_allocator_.set_tenant_id(tenant_id_); + row_allocator_.set_tenant_id(tenant_id_); + is_inited_ = true; + } + return ret; +} + +void ObTableApiRowIterator::reset() +{ + if (has_generate_column_) { + sql::ObSQLUtils::destruct_default_expr_context(expr_ctx_); + } + part_service_ = NULL; + schema_service_ = NULL; + ctx_ = NULL; + table_schema_ = NULL; + table_id_ = 0; + tenant_id_ = 0; + schema_version_ = 0; + rowkey_column_cnt_ = 0; + properties_.reset(); + column_ids_.reset(); + columns_type_.reset(); + column_descs_.reset(); + row_objs_.reset(); + missing_default_objs_.reset(); + generate_column_exprs_.reset(); + generate_column_idxs_.reset(); + stmt_allocator_.reset(); + row_allocator_.reset(); + entity_ = NULL; + has_generate_column_ = false; + is_inited_ = false; +} + +int ObTableApiRowIterator::check_row(ObNewRow &row) +{ + int ret = OB_SUCCESS; + const int64_t N = columns_type_.count(); + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { + if (OB_FAIL(check_column_type(columns_type_.at(i), row.get_cell(i)))) { + LOG_WARN("Fail to check column type, ", K(ret), K(i), K(columns_type_.at(i)), K(row.get_cell(i))); + } + } // end for + return ret; +} + +int ObTableApiRowIterator::entity_to_row(const ObITableEntity &entity, ObIArray &row) +{ + int ret = OB_SUCCESS; + ObObj obj; + const int64_t N = entity.get_rowkey_size(); + row.reuse(); + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { + if (OB_FAIL(entity.get_rowkey_value(i, obj))) { + LOG_WARN("failed to get rowkey value", K(ret), K(i)); + } else if (OB_FAIL(row.push_back(obj))) { + LOG_WARN("failed to push", K(ret), K(i), K(obj)); + } + } // end for + if (OB_SUCC(ret)) { + if (OB_FAIL(entity.get_properties_values(row))) { + LOG_WARN("failed to get properties values", K(ret)); + } + } + return ret; +} + +int ObTableApiRowIterator::fill_get_param( + ObTableServiceCtx &ctx, + const ObTableOperationType::Type op_type, + ObRowkey &rowkey, + storage::ObTableScanParam &scan_param, + share::schema::ObTableParam &table_param) +{ + int ret = OB_SUCCESS; + scan_param.key_ranges_.reset(); + if (OB_FAIL(fill_range(rowkey, scan_param.key_ranges_))) { + LOG_WARN("Fail to fill range, ", K(ret)); + } else if (OB_FAIL(fill_flag(ctx, scan_param))) { + LOG_WARN("Fail to fill param flag, ", K(ret)); + } else if (OB_FAIL(table_param.convert(*table_schema_, *table_schema_, column_ids_, false /*index back*/))) { + LOG_WARN("failed to convert table param", K(ret)); + } else { + if (ObTableOperationType::DEL == op_type + || ObTableOperationType::UPDATE == op_type + || ObTableOperationType::INSERT_OR_UPDATE == op_type + || ObTableOperationType::REPLACE == op_type + || ObTableOperationType::INCREMENT == op_type + || ObTableOperationType::APPEND == op_type) { + scan_param.for_update_ = true; + } + scan_param.table_param_ = &table_param; + LOG_DEBUG("Success to fill get param."); + } + return ret; +} + +int ObTableApiRowIterator::fill_multi_get_param( + ObTableServiceCtx &ctx, + const ObTableBatchOperation &batch_operation, + storage::ObTableScanParam &scan_param, + share::schema::ObTableParam &table_param) +{ + int ret = OB_SUCCESS; + scan_param.key_ranges_.reset(); + const int64_t N = batch_operation.count(); + if (N <= 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid argument, ", K(ret), K(N)); + } + + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { + ObRowkey rowkey = const_cast(batch_operation.at(i).entity()).get_rowkey(); + if (OB_FAIL(fill_range(rowkey, scan_param.key_ranges_))) { + LOG_WARN("Fail to fill range, ", K(ret), K(i)); + } + } // end for + + if (OB_SUCC(ret)) { + if (OB_FAIL(fill_flag(ctx, scan_param))) { + LOG_WARN("Fail to fill param flag, ", K(ret)); + } else if (OB_FAIL(table_param.convert(*table_schema_, *table_schema_, column_ids_, false /*index back*/))) { + LOG_WARN("failed to convert table param", K(ret)); + } else { + const ObTableOperation &table_operation = batch_operation.at(0); + if (ObTableOperationType::DEL == table_operation.type() + || ObTableOperationType::UPDATE == table_operation.type() + || ObTableOperationType::INSERT_OR_UPDATE == table_operation.type() + || ObTableOperationType::REPLACE == table_operation.type() + || ObTableOperationType::INCREMENT == table_operation.type() + || ObTableOperationType::APPEND == table_operation.type()) { + scan_param.for_update_ = true; + } + scan_param.table_param_ = &table_param; + } + } + return ret; +} + +int ObTableApiRowIterator::fill_generate_columns(common::ObNewRow &row) +{ + int ret = OB_SUCCESS; + ObISqlExpression *expr = NULL; + const int64_t N = generate_column_exprs_.count(); + int64_t col_idx = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { + if (OB_FAIL(generate_column_exprs_.at(i, expr))) { + LOG_WARN("Fail to get generate column expr, ", K(ret), K(i)); + } else if (OB_FAIL(generate_column_idxs_.at(i, col_idx))) { + LOG_WARN("Fail to get generate column idx, ", K(ret), K(i)); + } else if (NULL != expr) { + if (OB_FAIL(sql::ObSQLUtils::calc_sql_expression( + expr, + *table_schema_, + column_descs_, + row, + row_allocator_, + expr_ctx_, + row.cells_[col_idx]))) { + LOG_WARN("failed to calc expr from str", K(column_descs_), K(ret)); + } else { + LOG_DEBUG("Success to calc expr, ", K(col_idx), K(row.cells_[col_idx])); + } + } + } + return ret; +} + + +int ObTableApiRowIterator::cons_all_columns(const ObITableEntity &entity, const bool ignore_missing_column) +{ + int ret = OB_SUCCESS; + const ObRowkeyInfo &rowkey_info = table_schema_->get_rowkey_info(); + if (OB_FAIL(entity.get_properties_names(properties_))) { + LOG_WARN("failed to get properties, ", K(ret)); + } else if (OB_FAIL(rowkey_info.get_column_ids(column_ids_))) { + LOG_WARN("failed to get rowkey column ids", K(ret), K(rowkey_info)); + } else { + const schema::ObColumnSchemaV2 *column_schema = NULL; + uint64_t column_id = OB_INVALID_ID; + + //add rowkey column + const int64_t rowkey_cnt = rowkey_info.get_size(); + for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_cnt; ++i) { + if (OB_FAIL(rowkey_info.get_column_id(i, column_id))) { + LOG_WARN("failed to get column id", K(ret), K(i)); + } else if (NULL == (column_schema = table_schema_->get_column_schema(column_id))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rowkey column not exists", K(ret), K(column_id)); + } else if (OB_FAIL(add_column_type(*column_schema))) { + LOG_WARN("Fail to add column type, ", K(ret)); + } + } // end for + + //add property column + const int64_t property_cnt = properties_.count(); + for (int64_t i = 0; OB_SUCC(ret) && i < property_cnt; ++i) { + const ObString &cname = properties_.at(i); + if (NULL == (column_schema = table_schema_->get_column_schema(cname))) { + ret = OB_ERR_COLUMN_NOT_FOUND; + LOG_WARN("column not exists", K(ret), K(cname)); + } else if (!ignore_missing_column && column_schema->is_generated_column()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Should not have generate columns, ", K(ret), K(cname)); + } else if (OB_FAIL(column_ids_.push_back(column_schema->get_column_id()))) { + LOG_WARN("failed to add column id", K(ret)); + } else if (OB_FAIL(add_column_type(*column_schema))) { + LOG_WARN("Fail to add column type, ", K(ret)); + } + } // end for + + //add missing column + if (OB_SUCC(ret) && !ignore_missing_column) { + if (OB_FAIL(cons_missing_columns(entity))) { + LOG_WARN("Fail to cons missing column, ", K(ret)); + } + } + } + return ret; +} + +int ObTableApiRowIterator::cons_missing_columns(const ObITableEntity &entity) +{ + int ret = OB_SUCCESS; + has_generate_column_ = false; + if (table_schema_->get_column_count() - table_schema_->get_rowkey_column_num() <= entity.get_properties_count()) { + // no missing columns, do nothing + } else { + // some columns are missing + ObTableSchema::const_column_iterator iter = NULL; + ObTableSchema::const_column_iterator begin = table_schema_->column_begin(); + ObTableSchema::const_column_iterator end = table_schema_->column_end(); + ObObj obj; + ObExprResType column_type; + const schema::ObColumnSchemaV2 *column = NULL; + for (iter = begin; OB_SUCC(ret) && iter != end; ++iter) { + column = *iter; + // skip all rowkeys + if (OB_ISNULL(column)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid column schema", K(column)); + } else if (!column->is_rowkey_column()) { + if (OB_FAIL(entity.get_property(column->get_column_name_str(), obj))) { + ret = OB_SUCCESS; + // missing column + obj = column->get_cur_default_value(); + if (OB_FAIL(missing_default_objs_.push_back(obj))) { + LOG_WARN("Failed to push default obj, ", K(ret)); + } else if (OB_FAIL(column_ids_.push_back(column->get_column_id()))) { + LOG_WARN("failed to add column id", K(ret)); + } else if (OB_FAIL(add_column_type(*column))) { + LOG_WARN("Fail to add column type, ", K(ret)); + } else { + if (column->is_generated_column()) { + if (OB_FAIL(generate_column_idxs_.push_back(column_ids_.count() - 1))) { + LOG_WARN("Failed to push generate column idx, ", K(ret)); + } + has_generate_column_ = true; + } + LOG_DEBUG("add missing column", K(obj), K(column_type)); + } + } + } else {} + } //end for + + //fill generate expression + if (has_generate_column_) { + generate_column_exprs_.reuse(); + uint64_t tenant_id = extract_tenant_id(table_schema_->get_table_id()); + if (OB_FAIL(sql::ObSQLUtils::make_default_expr_context(tenant_id, stmt_allocator_, expr_ctx_))) { + LOG_WARN("failed to make default expr context ", K(ret)); + } + + for (iter = begin; OB_SUCC(ret) && iter != end; ++iter) { + column = *iter; + if (column->is_generated_column()) { + ObISqlExpression *expr = NULL; + obj = column->get_orig_default_value(); + if (OB_FAIL(sql::ObSQLUtils::make_generated_expression_from_str( + obj.get_string(), + *table_schema_, + *column, + column_descs_, + stmt_allocator_, + expr))) { + STORAGE_LOG(WARN, + "failed to make sql expression", + K(obj.get_string()), + K(*table_schema_), + K(*column), + K(column_descs_), + K(ret)); + } else if (OB_FAIL(generate_column_exprs_.push_back(expr))) { + STORAGE_LOG(WARN, "push back error", K(ret)); + } else { + /*do nothing*/ + } + + } + } + } + } + return ret; +} + + +int ObTableApiRowIterator::add_column_type(const share::schema::ObColumnSchemaV2 &column_schema) +{ + int ret = OB_SUCCESS; + ObExprResType column_type; + ObColDesc column_desc; + if (OB_FAIL(cons_column_type(column_schema, column_type))) { + LOG_WARN("failed to cons column type", K(ret)); + } else if (OB_FAIL(columns_type_.push_back(column_type))) { + LOG_WARN("failed to push back column type, ", K(ret)); + } else { + column_desc.col_id_ = column_schema.get_column_id(); + column_desc.col_type_ = column_schema.get_meta_type(); + if (OB_FAIL(column_descs_.push_back(column_desc))) { + LOG_WARN("failed to push back column desc, ", K(ret)); + } + } + return ret; +} + +int ObTableApiRowIterator::cons_column_type(const schema::ObColumnSchemaV2 &column_schema, ObExprResType &column_type) +{ + int ret = OB_SUCCESS; + column_type.set_type(column_schema.get_data_type()); + column_type.set_result_flag(ObRawExprUtils::calc_column_result_flag(column_schema)); + if (ob_is_string_type(column_schema.get_data_type())) { + column_type.set_collation_type(column_schema.get_collation_type()); + column_type.set_collation_level(CS_LEVEL_IMPLICIT); + } else { + column_type.set_collation_type(CS_TYPE_BINARY); + column_type.set_collation_level(CS_LEVEL_NUMERIC); + } + const ObAccuracy &accuracy = column_schema.get_accuracy(); + column_type.set_accuracy(accuracy); + const bool is_zerofill = column_type.has_result_flag(ZEROFILL_FLAG); + if (is_zerofill) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("modifing column with ZEROFILL flag is not supported", K(ret), K(column_schema)); + } + return ret; +} + + +int ObTableApiRowIterator::check_table_supported(const ObTableSchema *table_schema) +{ + int ret = OB_SUCCESS; + if (!table_schema->is_user_table()) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("table type not supported", K(ret), + "table_name", table_schema->get_table_name_str(), + "type", table_schema->get_table_type()); + } + return ret; +} + +int ObTableApiRowIterator::check_column_type(const ObExprResType &column_type, ObObj &obj) +{ + int ret = OB_SUCCESS; + const bool is_not_nullable = column_type.has_result_flag(NOT_NULL_FLAG); + const ObCollationType cs_type = column_type.get_collation_type(); + // 1. check nullable + if (is_not_nullable && obj.is_null()) { + ret = OB_BAD_NULL_ERROR; + } else if (obj.is_null()) { + // continue + } else if (column_type.get_type() != obj.get_type() + && !(ob_is_string_type(column_type.get_type()) && ob_is_string_type(obj.get_type()))) { + // 2. data type mismatch + ret = OB_OBJ_TYPE_ERROR; + LOG_WARN("object type mismatch with column type", K(ret), K(column_type), K(obj)); + } else { + // 3. check collation + if (!ob_is_string_type(obj.get_type())) { + // not string type, continue + } else { + if (cs_type == obj.get_collation_type()) { + // same collation type + } else if (cs_type == CS_TYPE_BINARY) { + // any collation type can be compatible with cs_type_binary + obj.set_collation_type(cs_type); + } else if (ObCharset::charset_type_by_coll(cs_type) == ObCharset::charset_type_by_coll(obj.get_collation_type())) { + // same charset, convert it + obj.set_collation_type(cs_type); + } else { + ret = OB_ERR_COLLATION_MISMATCH; + LOG_WARN("collation type mismatch with column", K(ret), K(column_type), K(obj)); + } + if (OB_SUCC(ret)) { + // convert obj type to the column type (char, varchar or text) + obj.set_type(column_type.get_type()); + } + } + // 4. check accuracy + if (OB_SUCC(ret)) { + if (OB_FAIL(ob_obj_accuracy_check_only(column_type.get_accuracy(), cs_type, obj))) { + LOG_WARN("accuracy check failed", K(ret), K(obj), K(column_type)); + } + } + } + return ret; +} + + +int ObTableApiRowIterator::fill_range(const ObRowkey &key, ObIArray &ranges) +{ + int ret = OB_SUCCESS; + ObNewRange range; + + if (key.get_obj_cnt() > rowkey_column_cnt_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("wrong rowkey size", K(ret), K(key), K(columns_type_)); + } else { + const int64_t N = key.get_obj_cnt(); + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { + if (OB_FAIL(check_column_type(columns_type_.at(i), const_cast (key.get_obj_ptr()[i])))) { + LOG_WARN("Invalid column type, ", K(ret), K(i), K(columns_type_.at(i)), K(key.get_obj_ptr()[i])); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(range.build_range(table_id_, key))) { + LOG_WARN("fail to build key range", K(ret), K_(table_id), K(key)); + } else if (OB_FAIL(ranges.push_back(range))) { + LOG_WARN("fail to push back key range", K(ret), K(range)); + } else { + } + } + } + + return ret; +} + +int ObTableApiRowIterator::fill_flag(ObTableServiceCtx &ctx, storage::ObTableScanParam &scan_param) +{ + int ret = OB_SUCCESS; + const uint64_t table_id = ctx.param_.table_id_; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + scan_param.timeout_ = ctx.param_.timeout_ts_; + ObQueryFlag query_flag(ObQueryFlag::KeepOrder, // scan_order KeepOrder! + false, // daily_merge + false, // optimize + false, // whole_macro_scan + false, // full_row + false, // index_back + false, // query_stat + ObQueryFlag::MysqlMode, // sql_mode + true // read_latest + ); + scan_param.scan_flag_.flag_ = query_flag.flag_; + scan_param.reserved_cell_count_ = column_ids_.count() + 10; + scan_param.for_update_ = false; + scan_param.column_ids_.reset(); + scan_param.pkey_ = part_key; + scan_param.schema_version_ = schema_version_; + if (OB_FAIL(scan_param.column_ids_.assign(column_ids_))) { + LOG_WARN("fail to assign column id", K(ret)); + } else { + scan_param.expr_ctx_.calc_buf_ = NULL; + scan_param.expr_ctx_.my_session_ = NULL; + scan_param.expr_ctx_.phy_plan_ctx_ = NULL; + scan_param.limit_param_.limit_ = -1; + scan_param.limit_param_.offset_ = 0; + scan_param.trans_desc_ = &(ctx.param_.processor_->get_trans_desc()); + scan_param.index_id_ = table_id_; + scan_param.sql_mode_ = SMO_DEFAULT; + scan_param.allocator_->set_tenant_id(scan_param.pkey_.get_tenant_id()); + } + return ret; +} + + + +/** + * ------------------------------------------------------------------ObTableApiInsertRowIterator--------------------------------------------------------- + */ +ObTableApiInsertRowIterator::ObTableApiInsertRowIterator() +{ +} + +ObTableApiInsertRowIterator::~ObTableApiInsertRowIterator() +{ +} + +int ObTableApiInsertRowIterator::open(const ObTableOperation &table_operation) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api insert row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(cons_all_columns(table_operation.entity()))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else { + entity_ = &(table_operation.entity()); + } + return ret; +} + +int ObTableApiInsertRowIterator::get_next_row(ObNewRow *&row) +{ + int ret = OB_SUCCESS; + row_allocator_.reuse(); + if (OB_ISNULL(entity_)) { + ret = OB_NOT_INIT; + LOG_WARN("The entity is null, ", K(ret)); + } else if (OB_FAIL(cons_row(*entity_, row))) { + LOG_WARN("Fail to construct insert row, ", K(ret)); + } else { + //success + LOG_DEBUG("Api insert row iter, ", K(*row)); + } + return ret; +} + +int ObTableApiInsertRowIterator::cons_row(const table::ObITableEntity &entity, common::ObNewRow *&row) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(entity_to_row(entity, row_objs_))) { + LOG_WARN("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)))) { + LOG_WARN("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_))) { + LOG_WARN("Fail to fill generate columns, ", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(check_row(row_))) { + LOG_WARN("Fail to check row, ", K(ret), K_(row)); + } else { + row = &row_; + } + } + } + return ret; +} + + +/** + * ------------------------------------------------------------------ObTableApiMultiInsertRowIterator--------------------------------------------------------- + */ +ObTableApiMultiInsertRowIterator::ObTableApiMultiInsertRowIterator() + : batch_operation_(NULL), + row_idx_(0), + batch_cnt_(0), + is_iter_pause_(false) +{ +} + +ObTableApiMultiInsertRowIterator::~ObTableApiMultiInsertRowIterator() +{ +} + +void ObTableApiMultiInsertRowIterator::reset() +{ + batch_operation_ = NULL; + row_idx_ = 0; + batch_cnt_ = 0; + is_iter_pause_ = false; + ObTableApiInsertRowIterator::reset(); +} + +int ObTableApiMultiInsertRowIterator::open(const ObTableBatchOperation &batch_operation) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api multi insert iterator has not been inited, ", K(ret)); + } else if (OB_UNLIKELY(batch_operation.count() <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid argument, ", K(ret), K(batch_operation.count())); + } else if (OB_FAIL(cons_all_columns(batch_operation.at(0).entity()))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else { + batch_operation_ = &batch_operation; + row_idx_ = 0; + batch_cnt_ = batch_operation.count(); + } + return ret; +} + +int ObTableApiMultiInsertRowIterator::get_next_row(common::ObNewRow *&row) +{ + int ret = OB_SUCCESS; + row_allocator_.reuse(); + if (OB_ISNULL(batch_operation_)) { + ret = OB_NOT_INIT; + LOG_WARN("The table api multi insert row iterator has not been inited, ", K(ret)); + } else if (row_idx_ >= batch_cnt_ || is_iter_pause_) { + ret = OB_ITER_END; + } else if (OB_FAIL(cons_row(batch_operation_->at(row_idx_).entity(), row))) { + LOG_WARN("Fail to construct row, ", K(ret), K(row_idx_)); + } else { + row_idx_++; + is_iter_pause_ = true; + LOG_DEBUG("Api insert row iter, ", K(*row), K_(row_idx)); + } + return ret; +} + + +/** + * ------------------------------------------------------------------ObTableApiUpdateRowIterator--------------------------------------------------------- + */ +ObTableApiUpdateRowIterator::ObTableApiUpdateRowIterator() + : scan_param_(), + table_param_(stmt_allocator_), + update_column_ids_(), + scan_iter_(NULL), + old_row_(NULL), + new_row_(NULL), + row_idx_(0), + need_update_rowkey_(false), + table_operation_(NULL) +{ +} + +ObTableApiUpdateRowIterator::~ObTableApiUpdateRowIterator() +{ + if (NULL != scan_iter_) { + if (NULL != part_service_) { + part_service_->revert_scan_iter(scan_iter_); + scan_iter_ = NULL; + } else { + LOG_ERROR("The part service is NULL, but the scan iter is NOT NULL!"); + } + } +} + +void ObTableApiUpdateRowIterator::reset() +{ + if (NULL != scan_iter_) { + if (NULL != part_service_) { + part_service_->revert_scan_iter(scan_iter_); + scan_iter_ = NULL; + } else { + LOG_ERROR("The part service is NULL, but the scan iter is NOT NULL!"); + } + } + scan_param_.destroy(); + table_param_.reset(); + update_column_ids_.reset(); + old_row_ = NULL; + new_row_ = NULL; + row_idx_ = 0; + ObTableApiRowIterator::reset(); +} + +int ObTableApiUpdateRowIterator::open(const ObTableOperation &table_operation, + const ObRowkey &rowkey, bool need_update_rowkey/* = false */) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(cons_all_columns(table_operation.entity()))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else if (OB_FAIL(cons_update_columns(need_update_rowkey))) { + LOG_WARN("Fail to construct update columns, ", K(ret)); + } else if (OB_FAIL(fill_get_param(*ctx_, + table_operation.type(), const_cast(rowkey), scan_param_, table_param_))) { + LOG_WARN("Fail to fill get param, ", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(scan_param_, scan_iter_))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("fail to scan table", K(ret)); + } + } else { + table_operation_ = &table_operation; + old_row_ = NULL; + new_row_ = NULL; + row_idx_ = 0; + need_update_rowkey_ = need_update_rowkey; + } + return ret; +} + +int ObTableApiUpdateRowIterator::get_next_row(ObNewRow *&row) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been inited, ", K(ret)); + } else if (OB_ISNULL(scan_iter_)) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been opened, ", K(ret)); + } else { + if (0 == row_idx_ % 2) { + if (OB_FAIL(scan_iter_->get_next_row(row))) { + if (OB_ITER_END != ret) { + LOG_WARN("Fail to get next row, ", K(ret)); + } else { + LOG_DEBUG("Update row iter end, ", K_(row_idx), K(scan_param_)); + } + } else { + old_row_ = row; + LOG_DEBUG("Update old row, ", K_(row_idx), K(*old_row_)); + } + } else if (OB_FAIL(cons_new_row(*table_operation_, row))) { + LOG_WARN("Fail to construct new row, ", K(ret)); + } else { + new_row_ = row; + LOG_DEBUG("Update iter cons new row, ", K_(row_idx), K(*new_row_)); + } + + if (OB_SUCC(ret)) { + row_idx_++; + } + } + return ret; +} + +int ObTableApiUpdateRowIterator::cons_update_columns(bool need_update_rowkey) +{ + int ret = OB_SUCCESS; + update_column_ids_.reuse(); + if (OB_UNLIKELY(column_ids_.count() <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("The column id array is empty, ", K(ret), K(column_ids_.count())); + } else { + const int64_t N = column_ids_.count() - missing_default_objs_.count(); + int64_t begin_column_idx = need_update_rowkey ? 0 : rowkey_column_cnt_; + for (int64_t i = begin_column_idx; OB_SUCC(ret) && i < N; i++) { + if (OB_FAIL(update_column_ids_.push_back(column_ids_.at(i)))) { + LOG_WARN("Fail to push column id to update column ids array, ", K(ret)); + } + } + + if (OB_SUCC(ret) && has_generate_column_) { + const int64_t gen_col_cnt = generate_column_idxs_.count(); + uint64_t column_id = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < gen_col_cnt; ++i) { + if (OB_FAIL(column_ids_.at(generate_column_idxs_.at(i), column_id))) { + LOG_WARN("Fail to get ith col id, ", K(ret), K(i)); + } else if (OB_FAIL(update_column_ids_.push_back(column_id))) { + LOG_WARN("Fail to push generate column id to update column ids array, ", K(ret), K(i), K(column_id)); + } + } + } + } + return ret; +} + +int ObTableApiUpdateRowIterator::cons_new_row(const ObTableOperation &table_operation, common::ObNewRow *&row) +{ + int ret = OB_SUCCESS; + row_objs_.reuse(); + if (OB_FAIL(entity_to_row(table_operation.entity(), row_objs_))) { + LOG_WARN("Fail to generate row from entity, ", K(ret)); + } else { + const int64_t rowkey_col_cnt = table_operation.entity().get_rowkey_size(); + const int64_t base_col_cnt = row_objs_.count(); + ObTableOperationType::Type type = table_operation.type(); + + switch(type) { + case ObTableOperationType::GET: + case ObTableOperationType::INSERT: + case ObTableOperationType::DEL: + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid operation type", K(ret), K(type)); + break; + case ObTableOperationType::UPDATE: + case ObTableOperationType::INSERT_OR_UPDATE: + case ObTableOperationType::REPLACE: + //nothing to do + break; + case ObTableOperationType::INCREMENT: { + for (int64_t i = rowkey_col_cnt; OB_SUCC(ret) && i < base_col_cnt; ++i) { + if (OB_FAIL(obj_increment(row_objs_.at(i), old_row_->cells_[i], columns_type_.at(i), row_objs_.at(i)))) { + LOG_WARN("failed to increment", K(ret), K(i), K(row_objs_.at(i)), K(old_row_->cells_[i])); + } + } + break; + } + case ObTableOperationType::APPEND: { + for (int64_t i = rowkey_col_cnt; OB_SUCC(ret) && i < base_col_cnt; ++i) { + if (OB_FAIL(obj_append(row_objs_.at(i), old_row_->cells_[i], columns_type_.at(i), row_objs_.at(i)))) { + LOG_WARN("failed to append", K(ret), K(i), K(row_objs_.at(i)), K(old_row_->cells_[i])); + } + } + break; + } + default: + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid operation type", K(ret), K(type)); + break; + } + + if (OB_SUCC(ret) && !need_update_rowkey_) { + const int64_t N = rowkey_col_cnt; + for (int64_t i = 0; i < N; ++i) { + row_objs_[i] = old_row_->cells_[i]; + } + } + + if (OB_SUCC(ret)) { + //fill default columns + 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(old_row_->cells_[base_col_cnt + i]))) { + LOG_WARN("Fail to push back old row obj to new row, ", K(ret)); + } + } + } + + if (OB_SUCC(ret)) { + //fill generate columns + row_.assign(&row_objs_.at(0), row_objs_.count()); + if (has_generate_column_ && OB_FAIL(fill_generate_columns(row_))) { + LOG_WARN("Fail to fill generate columns, ", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(check_row(row_))) { + LOG_WARN("Fail to check new row, ", K(ret)); + } + } + } + + if (OB_SUCC(ret)) { + row = &row_; + } + return ret; +} + + +int ObTableApiUpdateRowIterator::obj_increment( + const common::ObObj &delta, + const common::ObObj &src, + const sql::ObExprResType target_type, + common::ObObj &target) +{ + int ret = OB_SUCCESS; + if (OB_LIKELY(ob_is_int_tc(delta.get_type()))) { + int64_t delta_int = delta.get_int(); + ObObjType src_type = src.get_type(); + switch (src_type) { + case ObNullType: { + // the result is set to delta when src is NULL + ObObjType result_type = target_type.get_type(); + if (ob_is_int_tc(result_type)) { + ret = int_add_int_with_check(0, delta_int, result_type, target); + } else if (ob_is_uint_tc(result_type)) { + ret = uint_add_int_with_check(0, delta_int, result_type, target); + } else { + ret = OB_OBJ_TYPE_ERROR; + } + break; + } + case ObTinyIntType: + case ObSmallIntType: + case ObMediumIntType: + case ObInt32Type: + case ObIntType: { + // signed := signed + signed + ret = int_add_int_with_check(src.get_int(), delta_int, src_type, target); + break; + } + case ObUTinyIntType: + case ObUSmallIntType: + case ObUMediumIntType: + case ObUInt32Type: + case ObUInt64Type: { + // unsigned := usigned + signed + ret = uint_add_int_with_check(src.get_uint64(), delta_int, src_type, target); + break; + } + default: + ret = OB_OBJ_TYPE_ERROR; + break; + } // end switch + } else { + ret = OB_OBJ_TYPE_ERROR; + LOG_WARN("delta should only be signed integer type", K(ret), K(delta)); + } + return ret; +} + +int ObTableApiUpdateRowIterator::obj_append( + const common::ObObj &delta, + const common::ObObj &src, + const sql::ObExprResType target_type, + common::ObObj &target) +{ + int ret = OB_SUCCESS; + if (delta.is_null()) { + ret = OB_OBJ_TYPE_ERROR; + LOG_WARN("append NULL is illegal", K(ret), K(delta)); + } else if (OB_UNLIKELY(ObVarcharType != delta.get_type())) { + ret = OB_OBJ_TYPE_ERROR; + LOG_WARN("can only append varchar/varbinary type", K(ret), K(delta)); + } else { + const char *result_ptr = NULL; + int32_t result_len = 0; + + if (src.is_null()) { + // the result is set the delta when src is NULL + if (OB_UNLIKELY(ObVarcharType != target_type.get_type()) + && OB_UNLIKELY(ObTextTC != ob_obj_type_class(target_type.get_type()))) { + ret = OB_OBJ_TYPE_ERROR; + LOG_WARN("can only appended to string type", K(ret), K(target_type)); + } else { + result_ptr = delta.get_string_ptr(); + result_len = delta.get_string_len(); + } + } else if (OB_UNLIKELY(ObVarcharType != src.get_type()) + && OB_UNLIKELY(ObTextTC != ob_obj_type_class(src.get_type()))) { + ret = OB_OBJ_TYPE_ERROR; + LOG_WARN("can only appended to string type", K(ret), K(src)); + } else { + const int32_t len1 = src.get_string_len(); + const int32_t len2 = delta.get_string_len(); + const int32_t total_len = len1 + len2; + // the actual length will be checked in ObTableOperationTypeChecker + if (len1 <= 0) { + result_ptr = delta.get_string_ptr(); + result_len = delta.get_string_len(); + } else if (len2 <= 0) { + target = src; + result_len = -1; // no need to set the target + } else { + char *buf = NULL; + if (OB_ISNULL(buf = static_cast(row_allocator_.alloc(total_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_ERROR("alloc memory failed", K(ret), K(total_len)); + } else { + MEMCPY(buf, src.get_string_ptr(), len1); + MEMCPY(buf + len1, delta.get_string_ptr(), len2); + result_ptr = buf; + result_len = total_len; + } + } + } + + if (OB_SUCC(ret) && result_len >= 0) { + if (ob_is_text_tc(target_type.get_type())) { // LOB + target.set_lob_value(target_type.get_type(), result_ptr, result_len); + target.set_meta_type(target_type); + } else { // Varchar/Varbinary + target.set_varchar_value(result_ptr, result_len); + target.set_meta_type(target_type); + } + } + } + return ret; +} + +int ObTableApiUpdateRowIterator::int_add_int_with_check( + int64_t old_int, + int64_t delta_int, + common::ObObjType result_type, + common::ObObj &result) +{ + int ret = OB_SUCCESS; + int64_t result_int = old_int + delta_int; + if (sql::ObExprAdd::is_int_int_out_of_range(old_int, delta_int, result_int)) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("result of increment out of range", K(ret), + K(delta_int), K(old_int), K(result_int)); + } else if (result_int < INT_MIN_VAL[result_type] + || result_int > INT_MAX_VAL[result_type]) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("result of increment out of range", K(ret), + K(delta_int), K(old_int), K(result_int), + "min", INT_MIN_VAL[result_type], + "max", INT_MAX_VAL[result_type]); + } else { + result.set_type(result_type); + result.set_int_value(result_int); + } + return ret; +} + + +int ObTableApiUpdateRowIterator::uint_add_int_with_check( + uint64_t old_uint, + int64_t delta_int, + common::ObObjType result_type, + common::ObObj &result) +{ + int ret = OB_SUCCESS; + uint64_t result_uint = old_uint + delta_int; + if (sql::ObExprAdd::is_int_uint_out_of_range(delta_int, old_uint, result_uint)) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("result of increment out of range", K(ret), + K(delta_int), K(old_uint), K(result_uint)); + } else if (result_uint > UINT_MAX_VAL[result_type]) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("result of increment out of range", K(ret), + K(delta_int), K(old_uint), K(result_uint), + "max", UINT_MAX_VAL[result_type]); + } else { + result.set_type(result_type); + result.set_uint64_value(result_uint); + } + return ret; +} + + + +/** + * ------------------------------------------------------------------ObTableApiMultiUpdateRowIterator--------------------------------------------------------- + */ +ObTableApiMultiUpdateRowIterator::ObTableApiMultiUpdateRowIterator() + : batch_operation_(NULL), + batch_cnt_(0), + batch_idx_(0), + cur_update_idx_(-1), + is_iter_pause_(false) +{ +} + +ObTableApiMultiUpdateRowIterator::~ObTableApiMultiUpdateRowIterator() +{ +} + +void ObTableApiMultiUpdateRowIterator::reset() +{ + batch_operation_ = NULL; + batch_cnt_ = 0; + batch_idx_ = 0; + cur_update_idx_ = -1; + is_iter_pause_ = false; + ObTableApiUpdateRowIterator::reset(); +} + +int ObTableApiMultiUpdateRowIterator::open(const ObTableBatchOperation &batch_operation) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(cons_all_columns(batch_operation.at(0).entity()))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else if (OB_FAIL(cons_update_columns(false/*need_update_rowkey*/))) { + LOG_WARN("Fail to construct update columns, ", K(ret)); + } else if (OB_FAIL(fill_multi_get_param(*ctx_, batch_operation, scan_param_, table_param_))) { + LOG_WARN("Fail to fill get param, ", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(scan_param_, scan_iter_))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("fail to scan table", K(ret)); + } + } else { + batch_operation_ = &batch_operation; + batch_cnt_ = batch_operation.count(); + batch_idx_ = 0; + cur_update_idx_ = -1; + is_iter_pause_ = false; + } + return ret; +} + +int ObTableApiMultiUpdateRowIterator::get_next_row(ObNewRow *&row) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been inited, ", K(ret)); + } else if (OB_ISNULL(batch_operation_)) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been opened, ", K(ret)); + } else if (batch_idx_ >= batch_cnt_ || is_iter_pause_) { + ret = OB_ITER_END; + } else { + row_allocator_.reuse(); + if (0 == row_idx_ % 2) { + if (OB_FAIL(scan_iter_->get_next_row(row))) { + if (OB_ITER_END != ret) { + LOG_WARN("Fail to scan next row, ", K(ret), K_(row_idx)); + } else { + batch_idx_ = batch_cnt_; + } + old_row_ = NULL; + } else { + old_row_ = row; + } + } else { + ObRowkey old_row_key(old_row_->cells_, rowkey_column_cnt_); + row = NULL; + while(batch_idx_ < batch_cnt_) { + entity_ = &(batch_operation_->at(batch_idx_).entity()); + if (old_row_key.simple_equal((const_cast(entity_))->get_rowkey())) { + break; + } else { + batch_idx_++; + } + } + + if (batch_idx_ < batch_cnt_) { + if (OB_FAIL(cons_new_row(batch_operation_->at(batch_idx_), row))) { + LOG_WARN("Fail to construct new row, ", K(ret)); + } else { + cur_update_idx_ = batch_idx_; + batch_idx_++; + is_iter_pause_ = true; + } + } else { + ret = OB_ITER_END; + } + } + + if (OB_SUCC(ret)) { + row_idx_++; + } + } + return ret; +} + + +/** + * ------------------------------------------------------------------ObTableApiDeleteRowIterator--------------------------------------------------------- + */ +ObTableApiDeleteRowIterator::ObTableApiDeleteRowIterator() + : scan_param_(), + table_param_(stmt_allocator_), + scan_iter_(NULL) +{ +} + +ObTableApiDeleteRowIterator::~ObTableApiDeleteRowIterator() +{ + if (NULL != scan_iter_) { + if (NULL != part_service_) { + part_service_->revert_scan_iter(scan_iter_); + scan_iter_ = NULL; + } else { + LOG_ERROR("The part service is NULL, but the scan iter is NOT NULL!"); + } + } +} + +void ObTableApiDeleteRowIterator::reset() +{ + if (NULL != scan_iter_) { + if (NULL != part_service_) { + part_service_->revert_scan_iter(scan_iter_); + scan_iter_ = NULL; + } else { + LOG_ERROR("The part service is NULL, but the scan iter is NOT NULL!"); + } + } + scan_param_.destroy(); + table_param_.reset(); + + ObTableApiRowIterator::reset(); +} + +int ObTableApiDeleteRowIterator::open(const ObTableOperation &table_operation) +{ + int ret = OB_SUCCESS; + ObRowkey rowkey = const_cast(table_operation.entity()).get_rowkey(); + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(cons_all_columns(table_operation.entity()))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else if (OB_FAIL(fill_get_param(*ctx_, table_operation.type(), rowkey, + scan_param_, table_param_))) { + LOG_WARN("Fail to fill get param, ", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(scan_param_, scan_iter_))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("fail to scan table", K(ret)); + } + } else { + entity_ = &(table_operation.entity()); + } + return ret; +} + +int ObTableApiDeleteRowIterator::get_next_row(common::ObNewRow *&row) +{ + int ret = OB_SUCCESS; + if (NULL == scan_iter_) { + ret = OB_NOT_INIT; + LOG_WARN("The delete row iterator has not been inited, ", K(ret)); + } else { + if (OB_FAIL(scan_iter_->get_next_row(row))) { + if (OB_ITER_END != ret) { + LOG_WARN("Fail to get next delete row, ", K(ret)); + } + } + } + return ret; +} + +/** + * ------------------------------------------------------------------ObTableApiMultiDeleteRowIterator--------------------------------------------------------- + */ +ObTableApiMultiDeleteRowIterator::ObTableApiMultiDeleteRowIterator() + : batch_operation_(NULL), + batch_cnt_(0), + batch_idx_(0), + cur_delete_idx_(-1), + is_iter_pause_(false) +{ +} + +ObTableApiMultiDeleteRowIterator::~ObTableApiMultiDeleteRowIterator() +{ +} + +void ObTableApiMultiDeleteRowIterator::reset() +{ + batch_operation_ = NULL; + batch_cnt_ = 0; + batch_idx_ = 0; + cur_delete_idx_ = -1; + is_iter_pause_ = false; + ObTableApiDeleteRowIterator::reset(); +} + +int ObTableApiMultiDeleteRowIterator::open(const ObTableBatchOperation &batch_operation) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The multi delte row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(cons_all_columns(batch_operation.at(0).entity()))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else if (OB_FAIL(fill_multi_get_param(*ctx_, batch_operation, scan_param_, table_param_))) { + LOG_WARN("Fail to fill multi get param, ", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(scan_param_, scan_iter_))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("fail to scan table", K(ret)); + } + } else { + batch_operation_ = &batch_operation; + batch_cnt_ = batch_operation.count(); + batch_idx_ = 0; + cur_delete_idx_ = -1; + is_iter_pause_ = false; + } + return ret; +} + +int ObTableApiMultiDeleteRowIterator::get_next_row(ObNewRow *&row) +{ + int ret = OB_SUCCESS; + row = NULL; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api multi delete row iterator has not been inited, ", K(ret)); + } else if (OB_ISNULL(batch_operation_)) { + ret = OB_NOT_INIT; + LOG_WARN("The table api multi delete row iterator has not been opened, ", K(ret)); + } else if (batch_idx_ >= batch_cnt_ || is_iter_pause_) { + ret = OB_ITER_END; + } else { + if (OB_FAIL(scan_iter_->get_next_row(row))) { + if (OB_ITER_END != ret) { + LOG_WARN("Fail to get next delete row, ", K(ret)); + } else { + batch_idx_ = batch_cnt_; + } + } else { + ObRowkey row_key(row->cells_, rowkey_column_cnt_); + while(batch_idx_ < batch_cnt_) { + entity_ = &(batch_operation_->at(batch_idx_).entity()); + if (row_key.simple_equal((const_cast(entity_))->get_rowkey())) { + cur_delete_idx_ = batch_idx_; + break; + } else { + batch_idx_++; + } + } + batch_idx_++; + is_iter_pause_ = true; + } + } + + return ret; +} + + +/** + * ------------------------------------------------------------------ObTableApiGetRowIterator--------------------------------------------------------- + */ +ObTableApiGetRowIterator::ObTableApiGetRowIterator() + : scan_param_(), + table_param_(stmt_allocator_), + scan_iter_(NULL) +{ +} + +ObTableApiGetRowIterator::~ObTableApiGetRowIterator() +{ + if (NULL != scan_iter_) { + if (NULL != part_service_) { + part_service_->revert_scan_iter(scan_iter_); + scan_iter_ = NULL; + } else { + LOG_ERROR("The part service is NULL, but the scan iter is NOT NULL!"); + } + } +} + +void ObTableApiGetRowIterator::reset() +{ + scan_param_.destroy(); + table_param_.reset(); + if (NULL != scan_iter_) { + if (NULL != part_service_) { + part_service_->revert_scan_iter(scan_iter_); + scan_iter_ = NULL; + } else { + LOG_ERROR("The part service is NULL, but the scan iter is NOT NULL!"); + } + } + ObTableApiRowIterator::reset(); +} + +int ObTableApiGetRowIterator::open(const ObTableOperation &table_operation) +{ + int ret = OB_SUCCESS; + const bool ignore_missing_column = true; + ObRowkey rowkey = const_cast(table_operation.entity()).get_rowkey(); + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api update row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(cons_all_columns(table_operation.entity(), ignore_missing_column))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else if (OB_FAIL(fill_get_param(*ctx_, table_operation.type(), rowkey, + scan_param_, table_param_))) { + LOG_WARN("Fail to fill get param, ", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(scan_param_, scan_iter_))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("fail to scan table", K(ret)); + } + } else { + } + return ret; +} + +int ObTableApiGetRowIterator::get_next_row(common::ObNewRow *&row) +{ + int ret = OB_SUCCESS; + if (NULL == scan_iter_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api get row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(scan_iter_->get_next_row(row))) { + if (OB_ITER_END != ret) { + LOG_WARN("Fail to get next row, ", K(ret)); + } + } + return ret; +} + +/** + * ------------------------------------------------------------------ObTableApiMultiGetRowIterator--------------------------------------------------------- + */ +ObTableApiMultiGetRowIterator::ObTableApiMultiGetRowIterator() +{ +} + +ObTableApiMultiGetRowIterator::~ObTableApiMultiGetRowIterator() +{ +} + +int ObTableApiMultiGetRowIterator::open(const ObTableBatchOperation &batch_operation) +{ + int ret = OB_SUCCESS; + const bool ignore_missing_column = true; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("The table api multi get row iterator has not been inited, ", K(ret)); + } else if (OB_FAIL(cons_all_columns(batch_operation.at(0).entity(), ignore_missing_column))) { + LOG_WARN("Fail to construct all columns, ", K(ret)); + } else if (OB_FAIL(fill_multi_get_param(*ctx_, batch_operation, scan_param_, table_param_))) { + LOG_WARN("Fail to fill get param, ", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(scan_param_, scan_iter_))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("fail to scan table", K(ret)); + } + } else { + } + return ret; +} + +} +} diff --git a/src/observer/table/ob_table_api_row_iterator.h b/src/observer/table/ob_table_api_row_iterator.h new file mode 100644 index 0000000000..38702120d1 --- /dev/null +++ b/src/observer/table/ob_table_api_row_iterator.h @@ -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 &get_column_ids() { return column_ids_; } + OB_INLINE common::ObIArray &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 &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 &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 properties_; + common::ObSEArray column_ids_; + common::ObSEArray columns_type_; + common::ObSEArray column_descs_; + common::ObSEArray row_objs_; + common::ObSEArray missing_default_objs_; + common::ObSEArray generate_column_exprs_; + common::ObSEArray 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 &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 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 &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_ */ diff --git a/src/observer/table/ob_table_batch_execute_processor.cpp b/src/observer/table/ob_table_batch_execute_processor.cpp new file mode 100644 index 0000000000..7d6951293e --- /dev/null +++ b/src/observer/table/ob_table_batch_execute_processor.cpp @@ -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 &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(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 &part_ids) +{ + int ret = OB_SUCCESS; + uint64_t partition_id = arg_.partition_id_; + if (OB_INVALID_ID == partition_id) { + ObSEArray rowkeys_per_part; + ObSEArray 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 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 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 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 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 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 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 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; +} diff --git a/src/observer/table/ob_table_batch_execute_processor.h b/src/observer/table/ob_table_batch_execute_processor.h new file mode 100644 index 0000000000..71b216ad7d --- /dev/null +++ b/src/observer/table/ob_table_batch_execute_processor.h @@ -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 > +{ + typedef ObTableRpcProcessor > 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 &rowkeys); + int get_partition_ids(uint64_t table_id, common::ObIArray &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 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 */ diff --git a/src/observer/table/ob_table_end_trans_cb.cpp b/src/observer/table/ob_table_end_trans_cb.cpp new file mode 100644 index 0000000000..fce316f844 --- /dev/null +++ b/src/observer/table/ob_table_end_trans_cb.cpp @@ -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; +} diff --git a/src/observer/table/ob_table_end_trans_cb.h b/src/observer/table/ob_table_end_trans_cb.h new file mode 100644 index 0000000000..a95b63a980 --- /dev/null +++ b/src/observer/table/ob_table_end_trans_cb.h @@ -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 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 entity_factory_; + ObTableBatchOperationResult result_; + obrpc::ObRpcAsyncResponse response_sender_; + ObTableOperationType::Type table_operation_type_; +}; + +} // end namespace table +} // end namespace oceanbase + +#endif /* _OB_TABLE_END_TRANS_CB_H */ diff --git a/src/observer/table/ob_table_execute_processor.cpp b/src/observer/table/ob_table_execute_processor.cpp new file mode 100644 index 0000000000..102a0fcccf --- /dev/null +++ b/src/observer/table/ob_table_execute_processor.cpp @@ -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 rowkeys; + ObSEArray part_ids; + ObSEArray 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(arg_.table_operation_.entity()).get_rowkey(); + ObSEArray 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(arg_.table_operation_.entity()).get_rowkey(); + ObSEArray 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(arg_.table_operation_.entity()).get_rowkey(); + ObSEArray 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(arg_.table_operation_.entity()).get_rowkey(); + ObSEArray 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(arg_.table_operation_.entity()).get_rowkey(); + ObSEArray 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(arg_.table_operation_.entity()).get_rowkey(); + ObSEArray 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(arg_.table_operation_.entity()).get_rowkey(); + ObSEArray 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; +} diff --git a/src/observer/table/ob_table_execute_processor.h b/src/observer/table/ob_table_execute_processor.h new file mode 100644 index 0000000000..ccd87c01d8 --- /dev/null +++ b/src/observer/table/ob_table_execute_processor.h @@ -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 > +{ + typedef ObTableRpcProcessor > 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 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 */ diff --git a/src/observer/table/ob_table_query_processor.cpp b/src/observer/table/ob_table_query_processor.cpp new file mode 100644 index 0000000000..7d4b88555b --- /dev/null +++ b/src/observer/table/ob_table_query_processor.cpp @@ -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 &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 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; +} diff --git a/src/observer/table/ob_table_query_processor.h b/src/observer/table/ob_table_query_processor.h new file mode 100644 index 0000000000..bca6f21abc --- /dev/null +++ b/src/observer/table/ob_table_query_processor.h @@ -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 > +{ + typedef ObTableRpcProcessor > 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 &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 */ diff --git a/src/observer/table/ob_table_rpc_processor.cpp b/src/observer/table/ob_table_rpc_processor.cpp new file mode 100644 index 0000000000..c0fe24cfd2 --- /dev/null +++ b/src/observer/table/ob_table_rpc_processor.cpp @@ -0,0 +1,1014 @@ +/** + * 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_rpc_processor.h" +#include "observer/ob_service.h" +#include "storage/ob_partition_service.h" +#include "sql/ob_end_trans_callback.h" +#include "ob_table_end_trans_cb.h" +#include "sql/session/ob_sql_session_info.h" +#include "lib/stat/ob_diagnose_info.h" +#include "lib/stat/ob_session_stat.h" +#include "sql/ob_sql.h" +#include "share/table/ob_table_rpc_proxy.h" +#include "ob_table_rpc_processor_util.h" +#include "observer/mysql/ob_mysql_request_manager.h" +#include "share/ob_define.h" +#include "share/schema/ob_schema_mgr.h" + +using namespace oceanbase::observer; +using namespace oceanbase::common; +using namespace oceanbase::table; +using namespace oceanbase::share; +using namespace oceanbase::obrpc; + +int ObTableLoginP::process() +{ + int ret = OB_SUCCESS; + const ObTableLoginRequest &login = arg_; + if (1 != login.auth_method_ + || 1 != login.client_version_ + || 0 != login.reserved1_ + || 0 != login.reserved2_ + || 0 != login.reserved3_) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid login request", K(ret), K(login)); + } else if (1 != login.client_type_ && 2 != login.client_type_) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid client type", K(ret), K(login)); + } else if (login.tenant_name_.empty() + || login.user_name_.empty() + || login.database_name_.empty()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid tenant or user or database", K(ret), K(login)); + } else if (login.pass_scramble_.length() != 20) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid scramble", K(ret), K(login)); + } else if (0 > login.ttl_us_) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid ttl_us", K(ret), K(login)); + } else if (SS_STOPPING == GCTX.status_ + || SS_STOPPED == GCTX.status_) { + ret = OB_SERVER_IS_STOPPING; + LOG_WARN("server is stopping", K(GCTX.status_), K(ret), K(login)); + } else if (SS_SERVING != GCTX.status_) { + ret = OB_SERVER_IS_INIT; + LOG_WARN("server is not serving", K(GCTX.status_), K(ret), K(login)); + } else { + // @todo check client_capabilities and max_packet_size + uint64_t user_token = 0; + if (OB_FAIL(get_ids())) { + LOG_WARN("failed to get ids", K(ret), K(login)); + } else if (OB_FAIL(verify_password(login.tenant_name_, login.user_name_, + login.pass_secret_, login.pass_scramble_, + login.database_name_, user_token))) { + LOG_DEBUG("failed to verify password", K(ret), K(login)); + } else if (OB_FAIL(generate_credential(result_.tenant_id_, result_.user_id_, result_.database_id_, + login.ttl_us_, user_token, result_.credential_))) { + LOG_WARN("failed to generate credential", K(ret), K(login)); + } else { + result_.reserved1_ = 0; + result_.reserved2_ = 0; + result_.server_version_ = ObString::make_string(PACKAGE_STRING); + } + } + // whether the client should refresh location cache + if (OB_SUCCESS != ret && is_bad_routing_err(ret)) { + ObRpcProcessor::bad_routing_ = true; + LOG_WARN("[TABLE] login bad routing", K(ret), "bad_routing", ObRpcProcessor::bad_routing_); + } + ObTenantStatEstGuard stat_guard(result_.tenant_id_); +#ifndef NDEBUG + LOG_INFO("[TABLE] login", K(ret), K_(arg), K_(result), "timeout", rpc_pkt_->get_timeout()); +#else + // @todo LOG_DEBUG + LOG_INFO("[TABLE] login", K(ret), K_(arg), K_(result), "timeout", rpc_pkt_->get_timeout(), + "receive_ts", get_receive_timestamp()); +#endif + + if (common::OB_INVALID_ARGUMENT == ret) { + RPC_OBRPC_LOG(ERROR, "yyy retcode is 4002", "pkt", req_->get_packet()); + } + EVENT_INC(TABLEAPI_LOGIN_COUNT); + return ret; +} + +int ObTableLoginP::get_ids() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(gctx_.schema_service_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid schema service", K(ret), K(gctx_.schema_service_)); + } else { + schema::ObSchemaGetterGuard guard; + if (OB_FAIL(gctx_.schema_service_->get_schema_guard(guard))) { + LOG_WARN("get_schema_guard failed", K(ret)); + } else if (OB_FAIL(guard.get_tenant_id(arg_.tenant_name_, result_.tenant_id_))) { + LOG_WARN("get_tenant_id failed", K(ret), "tenant", arg_.tenant_name_); + } else if (OB_INVALID_ID == result_.tenant_id_) { + ret = OB_ERR_INVALID_TENANT_NAME; + LOG_WARN("get_tenant_id failed", K(ret), "tenant", arg_.tenant_name_); + } else if (OB_FAIL(guard.get_database_id(result_.tenant_id_, arg_.database_name_, + result_.database_id_))) { + LOG_WARN("failed to get database id", K(ret), "database", arg_.database_name_); + } else if (OB_INVALID_ID == result_.database_id_) { + ret = OB_WRONG_DB_NAME; + LOG_WARN("failed to get database id", K(ret), "database", arg_.database_name_); + } else if (OB_FAIL(guard.get_user_id(result_.tenant_id_, arg_.user_name_, + ObString::make_string("%")/*assume there is no specific host*/, + result_.user_id_))) { + LOG_WARN("failed to get user id", K(ret), "user", arg_.user_name_); + } else if (OB_INVALID_ID == result_.user_id_) { + ret = OB_ERR_USER_NOT_EXIST; + LOG_WARN("failed to get user id", K(ret), "user", arg_.user_name_); + } + } + return ret; +} + +int ObTableLoginP::verify_password(const ObString &tenant, const ObString &user, + const ObString &pass_secret, const ObString &pass_scramble, + const ObString &db, uint64_t &user_token) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(gctx_.schema_service_)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid schema service", K(ret), K(gctx_.schema_service_)); + } else { + schema::ObSchemaGetterGuard guard; + if (OB_FAIL(gctx_.schema_service_->get_schema_guard(guard))) { + LOG_WARN("get_schema_guard failed", K(ret)); + } + schema::ObUserLoginInfo login_info; + login_info.tenant_name_ = tenant; + login_info.user_name_ = user; + login_info.scramble_str_ = pass_scramble; + login_info.db_ = db; + login_info.passwd_ = pass_secret; + SSL *ssl_st = NULL;//TODO::@yanhua not support ssl now for table-api + schema::ObSessionPrivInfo session_priv; + const schema::ObUserInfo *user_info = nullptr; + if (OB_FAIL(guard.check_user_access(login_info, session_priv, ssl_st))) { + LOG_WARN("User access denied", K(login_info), K(ret)); + } else if (OB_FAIL(guard.get_user_info(session_priv.tenant_id_, session_priv.user_id_, user_info))) { + LOG_WARN("get user info failed", K(ret), K(session_priv)); + } else { + user_token = user_info->get_passwd_str().hash(); + } + } + return ret; +} + + +OB_SERIALIZE_MEMBER(ObTableApiCredential, + cluster_id_, + tenant_id_, + user_id_, + database_id_, + expire_ts_, + hash_val_); + +ObTableApiCredential::ObTableApiCredential() + :cluster_id_(0), + tenant_id_(0), + user_id_(0), + database_id_(0), + expire_ts_(0), + hash_val_(0) +{ + +} + +ObTableApiCredential::~ObTableApiCredential() +{ + +} + +uint64_t ObTableApiCredential::hash(uint64_t seed /*= 0*/) const +{ + uint64_t hash_val = murmurhash(&cluster_id_, sizeof(cluster_id_), seed); + hash_val = murmurhash(&tenant_id_, sizeof(tenant_id_), hash_val); + hash_val = murmurhash(&user_id_, sizeof(user_id_), hash_val); + hash_val = murmurhash(&database_id_, sizeof(database_id_), hash_val); + hash_val = murmurhash(&expire_ts_, sizeof(expire_ts_), hash_val); + return hash_val; +} + +int ObTableLoginP::generate_credential(uint64_t tenant_id, uint64_t user_id, uint64_t database, + int64_t ttl_us, uint64_t user_token, ObString &credential_str) +{ + int ret = OB_SUCCESS; + ObTableApiCredential credential; + credential.cluster_id_ = GCONF.cluster_id; + credential.tenant_id_ = tenant_id; + credential.user_id_ = user_id; + credential.database_id_ = database; + if (ttl_us > 0) { + credential.expire_ts_ = ObTimeUtility::current_time() + ttl_us; + } else { + credential.expire_ts_ = 0; + } + credential.hash_val_ = credential.hash(user_token); + int64_t pos = 0; + if (OB_FAIL(serialization::encode(credential_buf_, CREDENTIAL_BUF_SIZE, pos, credential))) { + LOG_WARN("failed to serialize credential", K(ret), K(pos)); + } else { + credential_str.assign_ptr(credential_buf_, static_cast(pos)); + } + return ret; +} + +//////////////////////////////////////////////////////////////// +ObTableApiProcessorBase::ObTableApiProcessorBase(const ObGlobalContext &gctx) + :gctx_(gctx), + part_service_(gctx_.par_ser_), + table_service_(gctx_.table_service_), + stat_event_type_(-1), + audit_row_count_(0), + need_audit_(false), + request_string_(NULL), + request_string_len_(0), + need_retry_in_queue_(false), + retry_count_(0), + did_async_end_trans_(false) +{ + need_audit_ = GCONF.enable_sql_audit; +} + +void ObTableApiProcessorBase::reset_ctx() +{ + participants_.reset(); + trans_state_.reset(); + trans_desc_.reset(); + part_epoch_list_.reset(); + did_async_end_trans_ = false; +} + +int ObTableApiProcessorBase::check_user_access(const ObString &credential_str) +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + share::schema::ObSchemaGetterGuard schema_guard; + const share::schema::ObUserInfo *user_info = NULL; + if (OB_ISNULL(gctx_.schema_service_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid schema service", K(ret)); + } else if (OB_FAIL(serialization::decode(credential_str.ptr(), credential_str.length(), pos, credential_))) { + LOG_WARN("failed to serialize credential", K(ret), K(pos)); + } else if (OB_FAIL(gctx_.schema_service_->get_schema_guard(schema_guard))) { + LOG_WARN("fail to get schema guard", K(ret)); + } else if (OB_FAIL(schema_guard.get_user_info(credential_.user_id_, user_info))) { + LOG_WARN("fail to get user info", K(ret), K(credential_)); + } else if (OB_ISNULL(user_info)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("user info is null", K(ret), K(credential_)); + } else { + const uint64_t user_token = user_info->get_passwd_str().hash(); + uint64_t hash_val = credential_.hash(user_token); + uint64_t my_cluster_id = GCONF.cluster_id; + if (hash_val != credential_.hash_val_) { + ret = OB_ERR_NO_PRIVILEGE; + LOG_WARN("invalid credential", K(ret), K_(credential), K(hash_val)); + } else if (my_cluster_id != credential_.cluster_id_) { + ret = OB_ERR_NO_PRIVILEGE; + LOG_WARN("invalid credential cluster id", K(ret), K_(credential), K(my_cluster_id)); + } else if (user_info->get_is_locked()) { // check whether user is locked. + ret = OB_ERR_USER_IS_LOCKED; + LOG_WARN("user is locked", K(ret), K(credential_)); + } else { + LOG_DEBUG("user can access", K(credential_)); + } + } + return ret; +} + +oceanbase::sql::ObSQLSessionInfo &session() +{ + static oceanbase::sql::ObSQLSessionInfo SESSION; + return SESSION; +} + +ObArenaAllocator &session_alloc() +{ + static ObArenaAllocator SESSION_ALLOC; + return SESSION_ALLOC; +} + +int ObTableApiProcessorBase::init_session() +{ + int ret = OB_SUCCESS; + static const uint32_t sess_version = 0; + static const uint32_t sess_id = 1; + static const uint64_t proxy_sess_id = 1; + if (OB_FAIL(session().test_init(sess_version, sess_id, proxy_sess_id, &session_alloc()))) { + LOG_WARN("init session failed", K(ret)); + } else if (OB_FAIL(session().load_default_sys_variable(false, true))) { + LOG_WARN("failed to load default sys var", K(ret)); + } + return ret; +} + +int ObTableApiProcessorBase::get_table_id( + const ObString &table_name, + const uint64_t arg_table_id, + uint64_t &real_table_id) const +{ + int ret = OB_SUCCESS; + real_table_id = arg_table_id; + if (OB_INVALID_ID == real_table_id || 0 == real_table_id) { + share::schema::ObSchemaGetterGuard schema_guard; + const bool is_index = false; + const int64_t tenant_id = credential_.tenant_id_; + const int64_t database_id = credential_.database_id_; + if (OB_FAIL(gctx_.schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { + LOG_WARN("failed to get schema guard", K(ret)); + } else if (OB_FAIL(schema_guard.get_table_id(tenant_id, database_id, table_name, + is_index, schema::ObSchemaGetterGuard::ALL_TYPES, real_table_id))) { + LOG_WARN("failed to get table id", K(ret), K(tenant_id), K(database_id), K(table_name)); + } else if (OB_INVALID_ID == real_table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("get invalid id", K(ret), K(tenant_id), K(database_id), K(table_name)); + } + } + return ret; +} + +// transaction control +int ObTableApiProcessorBase::get_partition_by_rowkey(uint64_t table_id, const ObIArray &rowkeys, + common::ObIArray &part_ids, + common::ObIArray &rowkeys_per_part) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard schema_guard; + sql::ObTableLocation location_calc; + const uint64_t tenant_id = extract_tenant_id(table_id); + if (OB_FAIL(gctx_.schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { + LOG_WARN("failed to get schema guard", K(ret)); + } else if (OB_FAIL(location_calc.calculate_partition_ids_by_rowkey( + session(), schema_guard, table_id, rowkeys, part_ids, rowkeys_per_part))) { + LOG_WARN("failed to calc partition id", K(ret)); + } else {} + return ret; +} + +int ObTableApiProcessorBase::get_participants_from_lc(uint64_t table_id, const common::ObIArray &part_ids, + ObPartitionLeaderArray &partition_leaders) +{ + int ret = OB_SUCCESS; + share::ObPartitionLocationCache *location_cache = gctx_.location_cache_; + ObAddr leader_addr; + const int64_t N = part_ids.count(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + ObPartitionKey part_key(table_id, part_ids.at(i), 0); + bool force_renew = false; + do { + if (OB_FAIL(location_cache->get_strong_leader(part_key, leader_addr, force_renew))) { + if (OB_LOCATION_LEADER_NOT_EXIST == ret && !force_renew) { + // retry one time + force_renew = true; + LOG_WARN("[yzfdebug] failed to get location and force renew", K(ret), K(part_key)); + } else { + force_renew = false; + LOG_WARN("failed to get location", K(ret), K(part_key)); + } + } else if (OB_FAIL(partition_leaders.push(part_key, leader_addr))) { + LOG_WARN("failed to add leader address", K(ret)); + } else { + LOG_DEBUG("get participants", K(part_key), K(leader_addr)); + } + } while (OB_LOCATION_LEADER_NOT_EXIST == ret && force_renew); + } // end for + return ret; +} + +int ObTableApiProcessorBase::get_participants_optimistic(uint64_t table_id, const common::ObIArray &part_ids, + common::ObPartitionLeaderArray &partition_leaders) +{ + int ret = OB_SUCCESS; + const int64_t N = part_ids.count(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + ObPartitionKey part_key(table_id, part_ids.at(i), 0); + if (OB_FAIL(partition_leaders.push(part_key, gctx_.self_addr_))) { + LOG_WARN("failed to add leader address", K(ret)); + } else { + LOG_DEBUG("get participants", K(part_key)); + } + } + return ret; +} + +int ObTableApiProcessorBase::get_participants(uint64_t table_id, const common::ObIArray &part_ids, + common::ObPartitionLeaderArray &partition_leaders) +{ + // for STRONG consistency, optimistic strategy is enough + // get_participants_from_lc() + return get_participants_optimistic(table_id, part_ids, partition_leaders); +} + +int ObTableApiProcessorBase::start_trans(bool is_readonly, const sql::stmt::StmtType stmt_type, + uint64_t table_id, const common::ObIArray &part_ids, int64_t timeout_ts) +{ + int ret = OB_SUCCESS; + NG_TRACE(T_start_trans_begin); + if (OB_FAIL(get_participants(table_id, part_ids, participants_))) { + LOG_WARN("failed to get participants", K(ret)); + } + const uint64_t tenant_id = credential_.tenant_id_; + const int64_t trans_timeout_ts = timeout_ts; + // 1. start transaction + if (OB_SUCC(ret)) { + transaction::ObStartTransParam start_trans_param; + int32_t access_mode = (is_readonly) ? transaction::ObTransAccessMode::READ_ONLY: transaction::ObTransAccessMode::READ_WRITE; + start_trans_param.set_access_mode(access_mode); + start_trans_param.set_type(transaction::ObTransType::TRANS_USER); + start_trans_param.set_isolation(transaction::ObTransIsolation::READ_COMMITED); + start_trans_param.set_autocommit(true); + // @todo ObTableConsistencyLevel::EVENTUAL + start_trans_param.set_consistency_type(transaction::ObTransConsistencyType::CURRENT_READ); + // By default only statement snapshot semantics, + // If need other semantics, please see ObTransConsistencyType and ObTransReadSnapshotType for reference + // ObSqlTransControl::decide_trans_read_interface_specs() decide the semantic of sql layer + start_trans_param.set_read_snapshot_type(transaction::ObTransReadSnapshotType::STATEMENT_SNAPSHOT); + start_trans_param.set_cluster_version(GET_MIN_CLUSTER_VERSION()); + + const uint32_t session_id = 1; // ignore + const uint64_t proxy_session_id = 1; // ignore + const uint64_t org_cluster_id = ObServerConfig::get_instance().cluster_id; + + if (true == trans_state_.is_start_trans_executed()) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("start_trans is executed", K(ret)); + } else { + if (OB_FAIL(part_service_->start_trans(tenant_id, + org_cluster_id, + start_trans_param, + trans_timeout_ts, + session_id, + proxy_session_id, trans_desc_))) { + LOG_WARN("fail start trans", K(ret), K(start_trans_param)); + } + trans_state_.set_start_trans_executed(OB_SUCC(ret)); + } + } + NG_TRACE(T_start_trans_end); + // 2. start stmt + if (OB_SUCC(ret)) { + transaction::ObStmtDesc &stmt_desc = trans_desc_.get_cur_stmt_desc(); + const bool is_sfu = false; + stmt_desc.phy_plan_type_ = sql::OB_PHY_PLAN_LOCAL; + stmt_desc.stmt_type_ = stmt_type; + stmt_desc.is_sfu_ = is_sfu; + stmt_desc.execution_id_ = 1; + // optimize out stmt_desc.set_sql_id_and_save_trace_id(""); + // stmt_desc.set_trans_app_trace_id_str(ObString::make_string("")); + stmt_desc.inner_sql_ = false; + stmt_desc.consistency_level_ = transaction::ObTransConsistencyLevel::STRONG; + stmt_desc.is_contain_inner_table_ = false; + const int64_t stmt_timeout_ts = trans_timeout_ts; + const bool is_retry_sql = false; + transaction::ObStmtParam stmt_param; + ObPartitionArray unreachable_partitions; + if (true == trans_state_.is_start_stmt_executed()) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("start_stmt is executed", K(ret)); + } else if (OB_FAIL(stmt_param.init(tenant_id, stmt_timeout_ts, is_retry_sql))) { + LOG_WARN("ObStmtParam init error", K(ret), K(tenant_id), K(is_retry_sql)); + } else if (OB_FAIL(part_service_->start_stmt(stmt_param, + trans_desc_, + participants_, unreachable_partitions))) { + LOG_WARN("failed to start stmt", K(ret), K(stmt_param)); + } + trans_state_.set_start_stmt_executed(OB_SUCC(ret)); + } + + // 3. start participant + NG_TRACE(T_start_part_begin); + if (OB_SUCC(ret)) { + if (true == trans_state_.is_start_participant_executed()) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("start_participant is executed", K(ret)); + } else if (OB_FAIL(part_service_->start_participant(trans_desc_, + participants_.get_partitions(), + part_epoch_list_))) { + LOG_WARN("fail start participants", K(ret)); + } + trans_state_.set_start_participant_executed(OB_SUCC(ret)); + } + NG_TRACE(T_start_part_end); + return ret; +} + +int ObTableApiProcessorBase::end_trans(bool is_rollback, rpc::ObRequest *req, int64_t timeout_ts, + bool use_sync /*=false*/) +{ + int ret = OB_SUCCESS; + NG_TRACE(T_end_part_begin); + int end_ret = OB_SUCCESS; + if (trans_state_.is_start_participant_executed() && trans_state_.is_start_participant_success()) { + if (OB_SUCCESS != (end_ret = part_service_->end_participant( + is_rollback, trans_desc_, participants_.get_partitions()))) { + ret = (OB_SUCCESS == ret) ? end_ret : ret; + LOG_WARN("fail to end participant", K(ret), K(end_ret), + K(is_rollback)); + } + trans_state_.clear_start_participant_executed(); + } + NG_TRACE(T_end_part_end); + if (trans_state_.is_start_stmt_executed() && trans_state_.is_start_stmt_success()) { + is_rollback = (is_rollback || OB_SUCCESS != ret); + bool is_incomplete = false; + ObPartitionArray discard_partitions; + if (OB_SUCCESS != (end_ret = part_service_->end_stmt( + is_rollback, is_incomplete, participants_.get_partitions(), + part_epoch_list_, discard_partitions, participants_, trans_desc_))) { + ret = (OB_SUCCESS == ret) ? end_ret : ret; + LOG_WARN("fail to end stmt", K(ret), K(end_ret), K(is_rollback)); + } + trans_state_.clear_start_stmt_executed(); + } + NG_TRACE(T_end_trans_begin); + if (trans_state_.is_start_trans_executed() && trans_state_.is_start_trans_success()) { + if (trans_desc_.is_readonly() || use_sync) { + ret = sync_end_trans(is_rollback, timeout_ts); + } else { + if (is_rollback) { + ret = sync_end_trans(true, timeout_ts); + } else { + ret = async_commit_trans(req, timeout_ts); + } + } + trans_state_.clear_start_trans_executed(); + } + trans_state_.reset(); + NG_TRACE(T_end_trans_end); + return ret; +} + +int ObTableApiProcessorBase::sync_end_trans(bool is_rollback, int64_t timeout_ts) +{ + int ret = OB_SUCCESS; + sql::ObEndTransSyncCallback callback; + if (OB_FAIL(callback.init(&trans_desc_, NULL))) { + LOG_WARN("fail init callback", K(ret)); + } else { + int wait_ret = OB_SUCCESS; + + callback.set_is_need_rollback(is_rollback); + callback.set_end_trans_type(sql::ObExclusiveEndTransCallback::END_TRANS_TYPE_IMPLICIT); + callback.handout(); + const int64_t stmt_timeout_ts = timeout_ts; + // whether end_trans is success or not, the callback MUST be invoked + if (OB_FAIL(part_service_->end_trans(is_rollback, trans_desc_, callback, stmt_timeout_ts))) { + LOG_WARN("fail end trans when session terminate", K(ret), K_(trans_desc), K(stmt_timeout_ts)); + } + // MUST wait here + if (OB_UNLIKELY(OB_SUCCESS != (wait_ret = callback.wait()))) { + LOG_WARN("sync end trans callback return an error!", K(ret), + K(wait_ret), K_(trans_desc), K(stmt_timeout_ts)); + } + ret = OB_SUCCESS != ret? ret : wait_ret; + bool has_called_txs_end_trans = false; + if (callback.is_txs_end_trans_called()) { // enter into the transaction layer + has_called_txs_end_trans = true; + } else { + //something wrong before enter into the transaction layer + has_called_txs_end_trans = false; + LOG_WARN("fail before trans service end trans, may disconnct", K(ret)); + if (OB_UNLIKELY(OB_SUCCESS == ret)) { + LOG_ERROR("callback before trans service end trans, but ret is OB_SUCCESS, it is BUG!!!", + K(callback.is_txs_end_trans_called())); + } + } + UNUSED(has_called_txs_end_trans); + ret = OB_SUCCESS != ret? ret : wait_ret; + } + return ret; +} + +int ObTableApiProcessorBase::async_commit_trans(rpc::ObRequest *req, int64_t timeout_ts) +{ + int ret = OB_SUCCESS; + const bool is_rollback = false; + ObTableAPITransCb* cb = new_callback(req); + if (NULL == cb) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc callback", K(ret)); + } else { + ObTableAPITransCb& callback = *cb; + callback.set_is_need_rollback(is_rollback); + callback.set_end_trans_type(sql::ObExclusiveEndTransCallback::END_TRANS_TYPE_IMPLICIT); + callback.handout(); + const int64_t stmt_timeout_ts = timeout_ts; + // whether end_trans is success or not, the callback MUST be invoked + if (OB_FAIL(part_service_->end_trans(is_rollback, trans_desc_, callback, stmt_timeout_ts))) { + LOG_WARN("fail end trans when session terminate", K(ret), K_(trans_desc), K(stmt_timeout_ts)); + } + // ignore the return code of end_trans + THIS_WORKER.disable_retry(); // can NOT retry after set end trans async to be true + did_async_end_trans_ = true; // don't send response in this worker thread + if (callback.is_txs_end_trans_called()) { // has entered into the transaction layer + } else { + // Something wrong before entered into transaction layer + // return error code to client + did_async_end_trans_ = false; + LOG_WARN("fail before trans service end trans, may disconnct", K(ret)); + if (OB_UNLIKELY(OB_SUCCESS == ret)) { + LOG_ERROR("callback before trans service end trans, but ret is OB_SUCCESS, it is BUG!!!", + K(callback.is_txs_end_trans_called())); + ret = OB_ERR_UNEXPECTED; + } + } + // @note the req_ may be freed, req_processor can not be read any more. + // The req_has_wokenup_ MUST set to be true, otherwise req_processor will invoke req_->set_process_start_end_diff, cause memory core + // @see ObReqProcessor::run() req_->set_process_start_end_diff(ObTimeUtility::current_time()); + this->set_req_has_wokenup(); + // @note after this code, the callback object can NOT be read any more!!! + callback.destroy_cb_if_no_ref(); + } + return ret; +} + +bool ObTableApiProcessorBase::need_audit() const +{ + return need_audit_; +} + +void ObTableApiProcessorBase::start_audit(const rpc::ObRequest *req) +{ + audit_record_.exec_record_.record_start(); + audit_record_.exec_timestamp_.before_process_ts_ = ObTimeUtility::current_time(); + if (OB_LIKELY(NULL != req)) { + audit_record_.user_client_addr_ = req->get_peer(); + MEMCPY(audit_record_.trace_id_, req->get_trace_id().get(), (int32_t)sizeof(audit_record_.trace_id_)); + audit_record_.exec_timestamp_.rpc_send_ts_ = req->get_send_timestamp(); + audit_record_.exec_timestamp_.receive_ts_ = req->get_receive_timestamp(); + audit_record_.exec_timestamp_.enter_queue_ts_ = req->get_enqueue_timestamp(); + save_request_string(); + generate_sql_id(); + } +} + +static int set_audit_name(const char *info_name, char *&audit_name, int64_t &audit_name_length, ObIAllocator &allocator) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(info_name)) { + audit_name = NULL; + audit_name_length = 0; + } else { + const int64_t name_length = strlen(info_name); + const int64_t buf_size = name_length + 1; + char *buf = reinterpret_cast(allocator.alloc(buf_size)); + if (OB_ISNULL(buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SERVER_LOG(WARN, "fail to alloc memory", K(ret), K(buf_size)); + } else { + strcpy(buf, info_name); + audit_name = buf; + audit_name_length = name_length; + } + } + return ret; +} + +void ObTableApiProcessorBase::end_audit() +{ + audit_record_.exec_record_.record_end(); + // credential info +// audit_record_.server_addr_; // not necessary, because gv_sql_audit_iterator use local addr automatically +// audit_record_.client_addr_; // not used for now + audit_record_.tenant_id_ = credential_.tenant_id_; + audit_record_.effective_tenant_id_ = credential_.tenant_id_; + audit_record_.user_id_ = credential_.user_id_; + audit_record_.db_id_ = credential_.database_id_; + + // update tenant_name, user_name, database_name + int ret = OB_SUCCESS; + share::schema::ObSchemaGetterGuard schema_guard; + if (OB_FAIL(gctx_.schema_service_->get_schema_guard(schema_guard))) { + SERVER_LOG(WARN, "fail to get schema guard", K(ret)); + } else { + { // set tenant name, ignore ret + const share::schema::ObSimpleTenantSchema *tenant_info = NULL; + if(OB_FAIL(schema_guard.get_tenant_info(credential_.tenant_id_, tenant_info))) { + SERVER_LOG(WARN, "fail to get tenant info", K(ret), K(credential_.tenant_id_)); + } else if (OB_ISNULL(tenant_info)) { + ret = OB_ERR_UNEXPECTED; + SERVER_LOG(WARN, "tenant info is null", K(ret)); + } else if (OB_FAIL(set_audit_name(tenant_info->get_tenant_name(), + audit_record_.tenant_name_, audit_record_.tenant_name_len_, audit_allocator_))){ + SERVER_LOG(WARN, "fail to set tenant name", K(ret), "tenant_name", tenant_info->get_tenant_name()); + } + } + + { // set user name, ignore ret + const share::schema::ObUserInfo *user_info = NULL; + if(OB_FAIL(schema_guard.get_user_info(credential_.user_id_, user_info))) { + SERVER_LOG(WARN, "fail to get user info", K(ret), K(credential_.user_id_)); + } else if (OB_ISNULL(user_info)) { + ret = OB_ERR_UNEXPECTED; + SERVER_LOG(WARN, "user info is null", K(ret)); + } else if (OB_FAIL(set_audit_name(user_info->get_user_name(), + audit_record_.user_name_, audit_record_.user_name_len_, audit_allocator_))) { + SERVER_LOG(WARN, "fail to set user name", K(ret), "user_name", user_info->get_user_name()); + } + } + + { // set database name, ignore ret + const share::schema::ObSimpleDatabaseSchema *database_info = NULL; + if(OB_FAIL(schema_guard.get_database_schema(credential_.database_id_, database_info))) { + SERVER_LOG(WARN, "fail to get database info", K(ret), K(credential_.database_id_)); + } else if (OB_ISNULL(database_info)) { + ret = OB_ERR_UNEXPECTED; + SERVER_LOG(WARN, "database info is null", K(ret)); + } else if (OB_FAIL(set_audit_name(database_info->get_database_name(), + audit_record_.db_name_, audit_record_.db_name_len_, audit_allocator_))) { + SERVER_LOG(WARN, "fail to set database name", K(ret), "database_name", database_info->get_database_name()); + } + } + } + + // append request string to query_sql + if (NULL != request_string_ && request_string_len_ > 0) { + static const char request_print_prefix[] = ", \nrequest: "; + const int64_t buf_size = audit_record_.sql_len_ + sizeof(request_print_prefix) + request_string_len_; + char *buf = reinterpret_cast(audit_allocator_.alloc(buf_size)); + if (NULL == buf) { + SERVER_LOG(WARN, "fail to alloc audit memory", K(buf_size), K(audit_record_.sql_), K(request_string_)); + } else { + memset(buf, 0, buf_size); + if (OB_NOT_NULL(audit_record_.sql_)) { + strcat(buf, audit_record_.sql_); + } + strcat(buf, request_print_prefix); + strcat(buf, request_string_); + audit_record_.sql_ = buf; + audit_record_.sql_len_ = buf_size; + } + } + + // request info + audit_record_.is_executor_rpc_ = false; // FIXME(wenqu): set false for print sql_id + audit_record_.is_hit_plan_cache_ = false; + audit_record_.is_inner_sql_ = false; + audit_record_.is_multi_stmt_ = false; + audit_record_.partition_cnt_ = 1; // always 1 for now; + audit_record_.plan_id_ = 0; + audit_record_.plan_type_ = OB_PHY_PLAN_UNINITIALIZED; + + // in-process info + audit_record_.execution_id_ = 0; // not used for table api + audit_record_.request_id_ = 0; // not used for table api + audit_record_.seq_ = 0; // not used + audit_record_.session_id_ = 0; // not used for table api + audit_record_.exec_timestamp_.exec_type_ = RpcProcessor; + + // use sched_info to record trace event, for debug + const char *arg_str = to_cstring(THE_TRACE); + if (OB_NOT_NULL(arg_str)) { + const int64_t buf_size = strlen(arg_str) + 1; + if (OB_FAIL(audit_record_.sched_info_.append(audit_allocator_, arg_str, buf_size))) { + LOG_WARN("fail to append sched info in table rpc", K(ret), K(buf_size)); + } + } + // update audit info and push + audit_record_.exec_timestamp_.net_t_ = audit_record_.exec_timestamp_.receive_ts_ - audit_record_.exec_timestamp_.rpc_send_ts_; + audit_record_.exec_timestamp_.net_wait_t_ = audit_record_.exec_timestamp_.enter_queue_ts_ - audit_record_.exec_timestamp_.receive_ts_; + audit_record_.update_stage_stat(); + + FETCH_ENTITY(TENANT_SPACE, credential_.tenant_id_) { + obmysql::ObMySQLRequestManager *req_manager = MTL_GET(obmysql::ObMySQLRequestManager*); + if (nullptr == req_manager) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get request manager for current tenant", K(ret)); + } else if (OB_FAIL(req_manager->record_request(audit_record_))) { + if (OB_SIZE_OVERFLOW == ret || OB_ALLOCATE_MEMORY_FAILED == ret) { + LOG_DEBUG("cannot allocate mem for record", K(ret)); + ret = OB_SUCCESS; + } else { + if (REACH_TIME_INTERVAL(100 * 1000)) { // in case logging is too frequent + LOG_WARN("failed to record request info in request manager", K(ret)); + } + } + } + } + +} + +int ObTableApiProcessorBase::process_with_retry(const ObString &credential, const int64_t timeout_ts) +{ + int ret = OB_SUCCESS; + audit_record_.exec_timestamp_.process_executor_ts_ = ObTimeUtility::current_time(); + ObWaitEventStat total_wait_desc; + if (OB_ISNULL(gctx_.ob_service_) || OB_ISNULL(gctx_.par_ser_)) { + ret = OB_INVALID_ARGUMENT; + LOG_ERROR("invalid argument", K(gctx_.ob_service_), K(ret)); + } else if (OB_FAIL(check_arg())) { + LOG_WARN("check arg failed", K(ret)); + } else if (OB_FAIL(check_user_access(credential))) { + LOG_WARN("check user access failed", K(ret)); + } else { + ObMaxWaitGuard max_wait_guard(&audit_record_.exec_record_.max_wait_event_); + ObTotalWaitGuard total_wait_guard(&total_wait_desc); + ObTenantStatEstGuard stat_guard(credential_.tenant_id_); + need_retry_in_queue_ = false; + bool did_local_retry = false; + do { + ret = try_process(); + did_local_retry = false; + // is_partition_change_error(ret) || is_master_changed_error(ret) retry in client + if ((OB_TRY_LOCK_ROW_CONFLICT == ret || OB_TRANSACTION_SET_VIOLATION == ret) + && retry_policy_.allow_retry()) { + int64_t now = ObTimeUtility::current_time(); + if (now > timeout_ts) { + LOG_WARN("process timeout", K(ret), K(now), K(timeout_ts)); + did_local_retry = false; + } else { + if (OB_TRY_LOCK_ROW_CONFLICT == ret) { + // throw to queue and retry + if (retry_policy_.allow_rpc_retry() && THIS_WORKER.set_retry_flag()) { + LOG_DEBUG("set retry flag and retry later when lock available"); + need_retry_in_queue_ = true; + } else { + // retry in current thread + did_local_retry = true; + } + } else if (OB_TRANSACTION_SET_VIOLATION == ret) { + EVENT_INC(TABLEAPI_TSC_VIOLATE_COUNT); + did_local_retry = true; + // @todo sleep for is_master_changed_error(ret) etc. ? + } + } + } + if (did_local_retry) { + if (retry_count_ < retry_policy_.max_local_retry_count_) { + ++retry_count_; + reset_ctx(); + } else { + did_local_retry = false; + } + } + } while (did_local_retry); + // record events + if (need_audit()) { + audit_on_finish(); + } + } + audit_record_.exec_record_.wait_time_end_ = total_wait_desc.time_waited_; + audit_record_.exec_record_.wait_count_end_ = total_wait_desc.total_waits_; + audit_record_.status_ = ret; + return ret; +} + +//////////////////////////////////////////////////////////////// +template class oceanbase::observer::ObTableRpcProcessor >; +template class oceanbase::observer::ObTableRpcProcessor >; +template class oceanbase::observer::ObTableRpcProcessor >; + +template +int ObTableRpcProcessor::deserialize() +{ + if (need_audit()) { + audit_record_.exec_timestamp_.run_ts_ = ObTimeUtility::current_time(); + } + return RpcProcessor::deserialize(); +} + +template +int ObTableRpcProcessor::before_process() +{ + if (need_audit()) { + start_audit(RpcProcessor::req_); + } + return RpcProcessor::before_process(); +} + +template +int ObTableRpcProcessor::process() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(process_with_retry(RpcProcessor::arg_.credential_, get_timeout_ts()))) { + if (OB_NOT_NULL(request_string_)) { // request_string_ has been generated if enable sql_audit + LOG_WARN("fail to process table_api request", K(ret), K(stat_event_type_), K(request_string_)); + } else if (did_async_end_trans()) { // req_ may be freed + LOG_WARN("fail to process table_api request", K(ret), K(stat_event_type_)); + } else { + LOG_WARN("fail to process table_api request", K(ret), K(stat_event_type_), "request", RpcProcessor::arg_); + } + // whether the client should refresh location cache + if (is_bad_routing_err(ret)) { + ObRpcProcessor::bad_routing_ = true; + LOG_WARN("table_api request bad routing", K(ret), "bad_routing", ObRpcProcessor::bad_routing_); + } + } + return ret; +} + +template +int ObTableRpcProcessor::before_response() +{ + if (need_audit()) { + const int64_t curr_time = ObTimeUtility::current_time(); + audit_record_.exec_timestamp_.executor_end_ts_ = curr_time; + // timestamp of start get plan, no need for table_api, set euqal to process_executor_ts_ + audit_record_.exec_timestamp_.single_process_ts_ = audit_record_.exec_timestamp_.process_executor_ts_; + const int64_t elapsed_us = curr_time - RpcProcessor::get_receive_timestamp(); + ObTableRpcProcessorUtil::record_stat(audit_record_, stat_event_type_, elapsed_us, audit_row_count_); + // todo: distinguish hbase rows and ob rows. + } + return RpcProcessor::before_response(); +} + +template +int ObTableRpcProcessor::response(const int retcode) +{ + int ret = OB_SUCCESS; + // if it is waiting for retry in queue, the response can NOT be sent. + if (!need_retry_in_queue_) { + ret = RpcProcessor::response(retcode); + } + return ret; +} + +template +int ObTableRpcProcessor::after_process() +{ + NG_TRACE(process_end); // print trace log if necessary + if (need_audit()) { + end_audit(); + } + return RpcProcessor::after_process(); +} + +template +void ObTableRpcProcessor::set_req_has_wokenup() +{ + RpcProcessor::req_has_wokenup_ = true; +} + +template +int64_t ObTableRpcProcessor::get_timeout_ts() const +{ + int64_t ts = 0; + if (NULL != RpcProcessor::rpc_pkt_) { + ts = RpcProcessor::get_receive_timestamp() + RpcProcessor::rpc_pkt_->get_timeout(); + } + return ts; +} + +template +void ObTableRpcProcessor::save_request_string() +{ + int ret = OB_SUCCESS; + const char *arg_str = to_cstring(RpcProcessor::arg_); + if (OB_NOT_NULL(arg_str)) { + const int64_t buf_size = strlen(arg_str) + 1; + char *buf = reinterpret_cast(audit_allocator_.alloc(buf_size)); + if (NULL == buf) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SERVER_LOG(WARN, "fail to alloc audit memory", K(ret), K(buf_size), K(arg_str)); + } else { + memset(buf, 0, buf_size); + strcpy(buf, arg_str); + request_string_ = buf; + request_string_len_ = buf_size; + } + } +} + +template +void ObTableRpcProcessor::generate_sql_id() +{ + uint64_t checksum = get_request_checksum(); + checksum = ob_crc64(checksum, &credential_.tenant_id_, sizeof(credential_.tenant_id_)); + checksum = ob_crc64(checksum, &credential_.user_id_, sizeof(credential_.user_id_)); + checksum = ob_crc64(checksum, &credential_.database_id_, sizeof(credential_.database_id_)); + snprintf(audit_record_.sql_id_, (int32_t)sizeof(audit_record_.sql_id_), + "TABLEAPI0x%04Xvv%016lX", RpcProcessor::PCODE, checksum); +} + +bool oceanbase::observer::is_bad_routing_err(const int err) +{ + // bad routing check : whether client should refresh location cache + // Now, following the same logic as in ../mysql/ob_query_retry_ctrl.cpp + return (is_master_changed_error(err) + || is_server_down_error(err) + || is_partition_change_error(err) + || is_server_status_error(err) + || is_unit_migrate(err) + || is_transaction_rpc_timeout_err(err) + || is_has_no_readable_replica_err(err) + || is_select_dup_follow_replic_err(err) + || is_trans_stmt_need_retry_error(err)); +} diff --git a/src/observer/table/ob_table_rpc_processor.h b/src/observer/table/ob_table_rpc_processor.h new file mode 100644 index 0000000000..eaa9c291ef --- /dev/null +++ b/src/observer/table/ob_table_rpc_processor.h @@ -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 > +{ + typedef obrpc::ObRpcProcessor > 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 &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 &rowkeys, + common::ObIArray &part_ids, + common::ObIArray &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 &part_ids, + common::ObPartitionLeaderArray &partition_leaders); + int get_participants_from_lc(uint64_t table_id, const common::ObIArray &part_ids, + common::ObPartitionLeaderArray &partition_leaders); + int get_participants_optimistic(uint64_t table_id, const common::ObIArray &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 ObTableRpcProcessor: public obrpc::ObRpcProcessor, public ObTableApiProcessorBase +{ + typedef obrpc::ObRpcProcessor 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 */ diff --git a/src/observer/table/ob_table_rpc_processor_util.h b/src/observer/table/ob_table_rpc_processor_util.h new file mode 100644 index 0000000000..685607b158 --- /dev/null +++ b/src/observer/table/ob_table_rpc_processor_util.h @@ -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(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 */ diff --git a/src/observer/table/ob_table_service.cpp b/src/observer/table/ob_table_service.cpp new file mode 100644 index 0000000000..c3bb450a4e --- /dev/null +++ b/src/observer/table/ob_table_service.cpp @@ -0,0 +1,2044 @@ +/** + * 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_service.h" +#include "ob_table_api_row_iterator.h" +#include "observer/ob_service.h" +#include "storage/ob_partition_service.h" +#include "ob_table_rpc_processor.h" +#include "sql/engine/expr/ob_expr_res_type.h" +#include "sql/resolver/expr/ob_raw_expr_util.h" +#include "lib/thread_local/ob_tsi_factory.h" +#include "sql/engine/expr/ob_expr_add.h" +using namespace oceanbase::observer; +using namespace oceanbase::common; +using namespace oceanbase::table; +using namespace oceanbase::share; +using namespace oceanbase::sql; + +OB_INLINE void replace_ret_code(int &ret) +{ + if (OB_ERR_PRIMARY_KEY_DUPLICATE == ret + || OB_BAD_NULL_ERROR == ret + || OB_OBJ_TYPE_ERROR == ret + || OB_ERR_COLLATION_MISMATCH == ret + || OB_ERR_DATA_TOO_LONG == ret + || OB_DATA_OUT_OF_RANGE == ret) { + ret = OB_SUCCESS; + } +} + +OB_INLINE int get_rowkey_column_ids( + const ObIArray &full_column_ids, + const int64_t rowkey_column_cnt, + ObIArray &rowkey_column_ids) +{ + int ret = OB_SUCCESS; + rowkey_column_ids.reset(); + if (OB_UNLIKELY(full_column_ids.count() <= 0 + || rowkey_column_cnt <= 0 + || rowkey_column_cnt > full_column_ids.count())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(full_column_ids), K(rowkey_column_cnt)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_column_cnt; ++i) { + if (OB_FAIL(rowkey_column_ids.push_back(full_column_ids.at(i)))) { + LOG_WARN("fail to push back current column id", K(ret), K(i), K(full_column_ids.at(i))); + } + } + } + return ret; +} + +int ObTableService::init(ObGlobalContext &gctx) +{ + int ret = OB_SUCCESS; + part_service_ = gctx.par_ser_; + schema_service_ = gctx.schema_service_; + return ret; +} + +int ObTableService::cons_rowkey_infos(const schema::ObTableSchema &table_schema, + ObIArray *column_ids, + ObIArray *columns_type) +{ + int ret = OB_SUCCESS; + const ObRowkeyInfo &rowkey_info = table_schema.get_rowkey_info(); + if (NULL != column_ids) { + if (OB_FAIL(rowkey_info.get_column_ids(*column_ids))) { + LOG_WARN("failed to get rowkey column ids", K(ret), K(rowkey_info)); + } + } + if (OB_SUCC(ret) && NULL != columns_type) { + const schema::ObColumnSchemaV2 *column_schema = NULL; + uint64_t column_id = OB_INVALID_ID; + ObExprResType column_type; + const int64_t N = rowkey_info.get_size(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(rowkey_info.get_column_id(i, column_id))) { + LOG_WARN("failed to get column id", K(ret), K(i)); + } else if (NULL == (column_schema = table_schema.get_column_schema(column_id))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rowkey column not exists", K(ret), K(column_id)); + } else if (OB_FAIL(cons_column_type(*column_schema, column_type))) { + LOG_WARN("failed to cons column type", K(ret)); + } else if (OB_FAIL(columns_type->push_back(column_type))) { + LOG_WARN("failed to push back", K(ret)); + } + } // end for + } + return ret; +} + +int ObTableService::cons_properties_infos(const schema::ObTableSchema &table_schema, + const ObIArray &properties, + ObIArray &column_ids, + ObIArray *columns_type) +{ + int ret = OB_SUCCESS; + const schema::ObColumnSchemaV2 *column_schema = NULL; + ObExprResType column_type; + const int64_t N = properties.count(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + const ObString &cname = properties.at(i); + if (NULL == (column_schema = table_schema.get_column_schema(cname))) { + ret = OB_ERR_COLUMN_NOT_FOUND; + LOG_WARN("column not exists", K(ret), K(cname)); + } else if (OB_FAIL(column_ids.push_back(column_schema->get_column_id()))) { + LOG_WARN("failed to add column id", K(ret)); + } else if (NULL != columns_type) { + if (OB_FAIL(cons_column_type(*column_schema, column_type))) { + LOG_WARN("failed to cons column type", K(ret)); + } else if (OB_FAIL(columns_type->push_back(column_type))) { + LOG_WARN("failed to push back", K(ret)); + } + } + } // end for + return ret; +} + +int ObTableService::cons_column_type(const schema::ObColumnSchemaV2 &column_schema, ObExprResType &column_type) +{ + int ret = OB_SUCCESS; + column_type.set_type(column_schema.get_data_type()); + column_type.set_result_flag(ObRawExprUtils::calc_column_result_flag(column_schema)); + if (ob_is_string_type(column_schema.get_data_type())) { + column_type.set_collation_type(column_schema.get_collation_type()); + column_type.set_collation_level(CS_LEVEL_IMPLICIT); + } else { + column_type.set_collation_type(CS_TYPE_BINARY); + column_type.set_collation_level(CS_LEVEL_NUMERIC); + } + const ObAccuracy &accuracy = column_schema.get_accuracy(); + column_type.set_accuracy(accuracy); + const bool is_zerofill = column_type.has_result_flag(ZEROFILL_FLAG); + if (is_zerofill) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("modifing column with ZEROFILL flag is not supported", K(ret), K(column_schema)); + } + return ret; +} + +// @pre column is not zerofill +int ObTableService::check_column_type(const ObExprResType &column_type, ObObj &obj) +{ + int ret = OB_SUCCESS; + const bool is_not_nullable = column_type.has_result_flag(NOT_NULL_FLAG); + const ObCollationType cs_type = column_type.get_collation_type(); + // 1. check nullable + if (is_not_nullable && obj.is_null()) { + ret = OB_BAD_NULL_ERROR; + } else if (obj.is_null()) { + // continue + } else if (column_type.get_type() != obj.get_type() + && !(ob_is_string_type(column_type.get_type()) && ob_is_string_type(obj.get_type()))) { + // 2. data type mismatch + ret = OB_OBJ_TYPE_ERROR; + LOG_WARN("object type mismatch with column type", K(ret), K(column_type), K(obj)); + } else { + // 3. check collation + if (!ob_is_string_type(obj.get_type())) { + // not string type, continue + } else { + if (cs_type == obj.get_collation_type()) { + // same collation type + } else if (cs_type == CS_TYPE_BINARY) { + // any collation type can be compatible with cs_type_binary + obj.set_collation_type(cs_type); + } else if (ObCharset::charset_type_by_coll(cs_type) == ObCharset::charset_type_by_coll(obj.get_collation_type())) { + // same charset, convert it + obj.set_collation_type(cs_type); + } else { + ret = OB_ERR_COLLATION_MISMATCH; + LOG_WARN("collation type mismatch with column", K(ret), K(column_type), K(obj)); + } + if (OB_SUCC(ret)) { + // convert obj type to the column type (char, varchar or text) + obj.set_type(column_type.get_type()); + } + } + // 4. check accuracy + if (OB_SUCC(ret)) { + if (OB_FAIL(ob_obj_accuracy_check_only(column_type.get_accuracy(), cs_type, obj))) { + LOG_WARN("accuracy check failed", K(ret), K(obj), K(column_type)); + } + } + } + return ret; +} + +int ObTableService::insert_or_update_can_use_put(uint64_t table_id, const ObITableEntity &entity, bool &use_put) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard schema_guard; + const schema::ObTableSchema *table_schema = NULL; + if (OB_FAIL(schema_service_->get_schema_guard(schema_guard))) { + LOG_WARN("failed to get schema guard", K(ret)); + } else if (OB_FAIL(schema_guard.get_table_schema(table_id, table_schema))) { + LOG_WARN("get table schema failed", K(table_id), K(ret)); + } else if (OB_ISNULL(table_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("NULL ptr", K(ret), K(table_schema)); + } else { + // 1. there is not any index + // 2. if there are some columns missing, the missing columns don't have a default value + use_put = (table_schema->get_index_tid_count() <= 0); + if (use_put) { + if (table_schema->get_column_count() - table_schema->get_rowkey_column_num() <= entity.get_properties_count()) { + // all columns are fine + } else { + // some columns are missing + schema::ObTableSchema::const_column_iterator iter = table_schema->column_begin(); + schema::ObTableSchema::const_column_iterator end = table_schema->column_end(); + ObObj obj; + ObExprResType column_type; + for (; OB_SUCC(ret) && iter != end; ++iter) { + const schema::ObColumnSchemaV2 *column = *iter; + if (OB_ISNULL(column)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid column schema", K(column)); + } else if (!column->is_rowkey_column()) { + const ObString &column_name = column->get_column_name_str(); + if (OB_FAIL(entity.get_property(column_name, obj))) { + ret = OB_SUCCESS; + // missing column + if (!column->get_cur_default_value().is_null()) { + use_put = false; + break; + } + } + } + } // end for + } + } + } + return ret; +} + +int ObTableService::execute_insert_or_update(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + const ObITableEntity &entity = table_operation.entity(); + bool can_use_put = true; + if (OB_FAIL(insert_or_update_can_use_put(ctx.param_.table_id_, entity, can_use_put))) { + LOG_WARN("failed to check", K(ret)); + } else if (can_use_put + && ctx.param_.binlog_row_image_type_ != ObBinlogRowImageType::FULL) { + ret = do_put(ctx, table_operation, result); + } else { + ret = do_insert_or_update(ctx, table_operation, result); + } + return ret; +} + +// If there are some columns missing in the property, can not fill default value. +// Because if row exists, it will do update, fill default value will cover the old value. +// If row does not exist, it will do insert, fill null value to missing columns is wrong. +// So, if and only if all columns are given or missing columns have not default value and there is not any index, we can use do_put +int ObTableService::do_put(ObTableServiceCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + const uint64_t table_id = ctx.param_.table_id_; + ObTableApiInsertRowIterator put_row_iter; + + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(put_row_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init put row iterator, ", K(ret), K(table_id)); + } else if (OB_FAIL(put_row_iter.open(table_operation))) { + LOG_WARN("Fail to open put row iterator, ", K(ret), K(table_id)); + } else { + int64_t affected_rows = 0; + // init dml param + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = false; // always false for put + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = put_row_iter.get_schema_version(); + + if (OB_FAIL(part_service_->put_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + put_row_iter.get_column_ids(), + &put_row_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to put row", K(ret), K(table_id)); + } + } else { + result.set_affected_rows(1); + } + } + result.set_errno(ret); + replace_ret_code(ret); + result.set_type(ObTableOperationType::INSERT_OR_UPDATE); + return ret; +} + +int ObTableService::do_insert_or_update(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + ObNewRowIterator *duplicate_row_iter = nullptr; + if (OB_FAIL(execute_insert(ctx, table_operation, result, duplicate_row_iter))) { + LOG_WARN("failed to execute_insert", K(ret)); + } else { + ret = result.get_errno(); + if (OB_ERR_PRIMARY_KEY_DUPLICATE == ret) { + ret = OB_SUCCESS; + ctx.reset_get_ctx(); + ObNewRow *duplicate_row = nullptr; + // update on duplicate key + if (OB_ISNULL(duplicate_row_iter)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("duplicate row iterator is null", K(ret)); + } else if (OB_FAIL(duplicate_row_iter->get_next_row(duplicate_row))) { + LOG_WARN("fail to get duplicate row", K(ret)); + } else if (OB_FAIL(execute_update(ctx, table_operation, duplicate_row, result))) { + LOG_WARN("failed to update", K(ret)); + } else { + // update succ + } + } else { + // insert succ + } + } + if (nullptr != duplicate_row_iter) { + ObPartitionKey pkey(ctx.param_.table_id_, ctx.param_.partition_id_, 0); + int tmp_ret = part_service_->revert_insert_iter(pkey, duplicate_row_iter); + if (OB_SUCCESS != tmp_ret) { + LOG_WARN("fail to revert duplicate_row_iter", K(tmp_ret), KP(duplicate_row_iter)); + } else { + duplicate_row_iter = nullptr; + } + } + result.set_errno(ret); + replace_ret_code(ret); + result.set_type(ObTableOperationType::INSERT_OR_UPDATE); + return ret; +} + +OB_INLINE int ObTableService::add_one_result(ObTableBatchOperationResult &result, + ObTableOperationType::Type op_type, + int32_t error_code, + int64_t affected_rows) +{ + int ret = OB_SUCCESS; + ObTableOperationResult one_result; + ObITableEntity *result_entity = result.get_entity_factory()->alloc(); + if (NULL == result_entity) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc memroy for result_entity", K(ret)); + } else { + one_result.set_entity(*result_entity); + one_result.set_type(op_type); + one_result.set_errno(error_code); + one_result.set_affected_rows(affected_rows); + if (OB_FAIL(result.push_back(one_result))) { + LOG_WARN("failed to push back", K(ret)); + } else { + LOG_DEBUG("[yzfdebug] add result", K(ret), K(one_result)); + } + } + return ret; +} + +int ObTableService::multi_insert_or_update(ObTableServiceGetCtx &ctx, + const ObTableBatchOperation &batch_operation, + ObTableBatchOperationResult &result) +{ + NG_TRACE(insertup_start_do); + int ret = OB_SUCCESS; + const ObTableOperation &one_op = batch_operation.at(0); + bool can_use_put = true; + if (OB_FAIL(insert_or_update_can_use_put(ctx.param_.table_id_, one_op.entity(), can_use_put))) { + LOG_WARN("failed to check", K(ret)); + } else if (can_use_put + && ctx.param_.binlog_row_image_type_ != ObBinlogRowImageType::FULL) { + ret = multi_put(ctx, batch_operation, result); + } else { + ret = do_multi_insert_or_update(ctx, batch_operation, result); + } + NG_TRACE(insertup_end); + return ret; +} + +int ObTableService::multi_put(ObTableServiceCtx &ctx, + const ObTableBatchOperation &batch_operation, + ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + ObTableApiMultiInsertRowIterator multi_put_iter; + const uint64_t table_id = ctx.param_.table_id_; + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(multi_put_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init multi put iterator, ", K(ret), K(table_id)); + } else if (OB_FAIL(multi_put_iter.open(batch_operation))) { + LOG_WARN("Fail to open multi put iterator, ", K(ret), K(table_id)); + } else { + int64_t affected_rows = 0; + // init dml param + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = false; // always false for put + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = multi_put_iter.get_schema_version(); + + const int64_t N = batch_operation.count(); + NG_TRACE_EXT(insertup_calc_new_row, OB_ID(input_count), N); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) { + multi_put_iter.continue_iter(); + if (OB_FAIL(part_service_->put_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + multi_put_iter.get_column_ids(), + &multi_put_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to put rows", K(ret), K(table_id)); + } else { + NG_TRACE(locked); + } + } else if (OB_FAIL(add_one_result(result, ObTableOperationType::INSERT_OR_UPDATE, OB_SUCCESS, affected_rows))) { + LOG_WARN("failed to add result", K(ret)); + } + } // end for + } + return ret; +} + +int ObTableService::do_multi_insert_or_update(ObTableServiceGetCtx &ctx, + const ObTableBatchOperation &batch_operation, + ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + const uint64_t table_id = ctx.param_.table_id_; + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else { + int64_t N = batch_operation.count(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + const ObTableOperation &table_operation = batch_operation.at(i); + ObTableOperationResult op_result; + ObITableEntity *result_entity = result.get_entity_factory()->alloc(); + if (NULL == result_entity) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc memroy for result_entity", K(ret)); + break; + } + op_result.set_entity(*result_entity); + if (OB_FAIL(do_insert_or_update(ctx, table_operation, op_result))) { + LOG_WARN("failed to do insert_or_update", K(ret)); + } else if (OB_FAIL(result.push_back(op_result))) { + LOG_WARN("failed to push back result", K(ret)); + } else { + ctx.reset_get_ctx(); + } + } // end for + } + return ret; +} + +//////////////////////////////////////////////////////////////// + +int ObTableService::fill_scan_param(ObTableServiceCtx &ctx, + const ObIArray &output_column_ids, + int64_t schema_version, + storage::ObTableScanParam &scan_param) +{ + int ret = OB_SUCCESS; + const uint64_t table_id = ctx.param_.table_id_; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + scan_param.timeout_ = ctx.param_.timeout_ts_; + ObQueryFlag query_flag(ObQueryFlag::KeepOrder, // scan_order KeepOrder! + false, // daily_merge + false, // optimize + false, // whole_macro_scan + false, // full_row + false, // index_back + false, // query_stat + ObQueryFlag::MysqlMode, // sql_mode + true // read_latest + ); + scan_param.scan_flag_.flag_ = query_flag.flag_; + scan_param.reserved_cell_count_ = output_column_ids.count() + 10; + scan_param.for_update_ = false; + scan_param.column_ids_.reset(); + scan_param.pkey_ = part_key; + scan_param.schema_version_ = schema_version; + if (OB_FAIL(scan_param.column_ids_.assign(output_column_ids))) { + LOG_WARN("fail to assign column id", K(ret)); + } else { + SQL_ENG_LOG(DEBUG, "set scan param", K(output_column_ids)); + scan_param.expr_ctx_.calc_buf_ = NULL; + scan_param.expr_ctx_.my_session_ = NULL; + scan_param.expr_ctx_.phy_plan_ctx_ = NULL; + scan_param.limit_param_.limit_ = -1; + scan_param.limit_param_.offset_ = 0; + scan_param.trans_desc_ = &ctx.param_.processor_->get_trans_desc(); + scan_param.index_id_ = table_id; + scan_param.sql_mode_ = SMO_DEFAULT; + scan_param.allocator_->set_tenant_id(scan_param.pkey_.get_tenant_id()); + } + return ret; +} + +int ObTableService::fill_get_result( + ObTableServiceCtx &ctx, + const ObIArray &properties, + ObTableApiRowIterator *scan_result, + ObTableOperationResult &operation_result) +{ + int ret = OB_SUCCESS; + ObNewRow *row = NULL; + table::ObITableEntity *entity = NULL; + if (OB_FAIL(operation_result.get_entity(entity))) { + LOG_WARN("failed to get entity", K(ret)); + } else if (OB_FAIL(scan_result->get_next_row(row))) { + if (OB_ITER_END != ret) { + LOG_WARN("failed to get next row", K(ret)); + } + } else { + const int64_t rowkey_column_cnt = scan_result->get_rowkey_column_cnt(); + const int64_t N = row->get_count(); + ObObj cell_clone; + for (int64_t i = rowkey_column_cnt; OB_SUCCESS == ret && i < N; ++i) + { + const ObString &name = properties.at(i - rowkey_column_cnt); + const ObObj &cell = row->get_cell(i); + if (OB_FAIL(ob_write_obj(*ctx.param_.allocator_, cell, cell_clone))) { + LOG_WARN("failed to copy obj", K(ret)); + } else if (OB_FAIL(entity->set_property(name, cell_clone))) { + LOG_WARN("failed to set property", K(ret), K(name), K(cell)); + } else { + LOG_DEBUG("yzfdebug get cell", K(name), K(i), K(cell), "row", (*row)); + } + } // end for + } + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + } + return ret; +} + +ObTableServiceGetCtx::ObTableServiceGetCtx(common::ObArenaAllocator &alloc) + :table_param_on_stack_(alloc), + table_param_(&table_param_on_stack_), + scan_result_(NULL) +{ + table_param_ = &table_param_on_stack_; + scan_param_.table_param_ = table_param_; +} + +int ObTableService::execute_get(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + NG_TRACE(S_table_scan_begin); + ObTableApiGetRowIterator get_row_iter; + ctx.scan_result_ = NULL; + if (OB_FAIL(get_row_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init get row iterator, ", K(ret)); + } else if (OB_FAIL(get_row_iter.open(table_operation))) { + LOG_WARN("Fail to open get row iterator, ", K(ret)); + } else if (OB_FAIL(fill_get_result(ctx, get_row_iter.get_properties(), &get_row_iter, result))) { + LOG_WARN("failed to send result"); + } else { + LOG_DEBUG("[yzfdebug] execute_get", "prop", get_row_iter.get_properties(), "table_param", *(ctx.table_param_), "scan_param", ctx.scan_param_, "trans_desc", ctx.param_.processor_->get_trans_desc()); + } + + result.set_type(ObTableOperationType::GET); + result.set_errno(ret); + replace_ret_code(ret); + NG_TRACE(S_table_scan_end); + return ret; +} + +int ObTableService::fill_multi_get_result( + ObTableServiceGetCtx &ctx, + const ObTableBatchOperation &batch_operation, + ObTableApiRowIterator *scan_result, + ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + const int64_t rowkey_size = batch_operation.at(0).entity().get_rowkey_size(); + ObNewRow *row = NULL; + const int64_t N = batch_operation.count(); + bool did_get_next_row = true; + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + // left join + const ObTableEntity &entity = static_cast(batch_operation.at(i).entity()); + ObRowkey expected_key = const_cast(entity).get_rowkey(); + ObTableOperationResult op_result; + ObITableEntity *result_entity = result.get_entity_factory()->alloc(); + if (NULL == result_entity) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc memroy for result_entity", K(ret)); + break; + } + op_result.set_entity(*result_entity); + op_result.set_type(ObTableOperationType::GET); + if (did_get_next_row) { + if (OB_FAIL(scan_result->get_next_row(row))) { + if (OB_ITER_END == ret) { + // push empty entity + ret = OB_SUCCESS; + op_result.set_errno(OB_SUCCESS); + if (OB_FAIL(result.push_back(op_result))) { + LOG_WARN("failed to push back result", K(ret), K(i)); + } + continue; + } else { + LOG_WARN("failed to get next row", K(ret)); + } + } else { + LOG_DEBUG("[yzfdebug] multi get row", K(ret), K(i), K(*row)); + } + } + if (OB_SUCC(ret)) { + ObRowkey the_key(row->cells_, rowkey_size); + ObObj cell_clone; + if (expected_key.simple_equal(the_key)) { + const int64_t N = row->get_count(); + for (int64_t i = rowkey_size; OB_SUCCESS == ret && i < N; ++i) + { + const ObString &name = scan_result->get_properties().at(i-rowkey_size); + ObObj &cell = row->get_cell(i); + if (OB_FAIL(ob_write_obj(*ctx.param_.allocator_, cell, cell_clone))) { + LOG_WARN("failed to copy obj", K(ret)); + } else if (OB_FAIL(result_entity->set_property(name, cell_clone))) { + LOG_WARN("failed to set property", K(ret)); + } else { + LOG_DEBUG("get cell", K(name), K(i), K(cell)); + } + } // end for + did_get_next_row = true; + } else { + did_get_next_row = false; + LOG_DEBUG("[yzfdebug] the row not exist", K(ret), K(expected_key)); + } + op_result.set_errno(OB_SUCCESS); + } + if (OB_SUCC(ret)) { + if (OB_FAIL(result.push_back(op_result))) { + LOG_WARN("failed to push back result", K(ret), K(i)); + } + } + } // end for + return ret; +} + +int ObTableService::multi_get(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + ObTableApiMultiGetRowIterator multi_get_iter; + if (OB_FAIL(multi_get_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init multi get iter, ", K(ret)); + } else if (OB_FAIL(multi_get_iter.open(batch_operation))) { + LOG_WARN("Fail to open multi get iter, ", K(ret)); + } else if (OB_FAIL(fill_multi_get_result(ctx, batch_operation, &multi_get_iter, result))) { + LOG_WARN("failed to send result"); + } + return ret; +} + +// for update, if schema is index type, all columns in index schema +// should be added, including index defined columns, rowkey columns +// in data schema, and storing columns. +// for delete, the storing columns might be useless, but still added. +// +int ObTableService::add_index_columns_if_missing(schema::ObSchemaGetterGuard &schema_guard, + uint64_t data_table_id, + const schema::ObTableSchema *index_schema, + ObIArray &column_ids, + ObIArray *columns_type) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_INVALID_ID; + const schema::ObColumnSchemaV2 *column_schema = NULL; + ObExprResType column_type; + schema::ObTableSchema::const_column_iterator b = index_schema->column_begin(); + schema::ObTableSchema::const_column_iterator e = index_schema->column_end(); + for (; OB_SUCC(ret) && b != e; ++b) { // for all columns of the index, include the storing columns + if (NULL == (*b)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to to get column schema", K(*b)); + } else { + if ((*b)->is_shadow_column()) { + continue; + } + column_id = (*b)->get_column_id(); + bool found = false; + const int64_t N = column_ids.count(); + for (int64_t i = 0; !found && i < N; ++i) + { + if (column_id == column_ids.at(i)) { + found = true; + } + } // end for + if (!found) { + if (OB_FAIL(schema_guard.get_column_schema(data_table_id, column_id, column_schema))) { + LOG_WARN("get column schema failed", K(data_table_id), K(column_id)); + } else if (NULL == column_schema) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to get column schema", K(data_table_id), K(column_id)); + } else { + if (OB_FAIL(column_ids.push_back(column_schema->get_column_id()))) { + LOG_WARN("failed to add column id", K(ret)); + } else if (NULL != columns_type) { + if (OB_FAIL(cons_column_type(*column_schema, column_type))) { + LOG_WARN("failed to cons column type", K(ret)); + } else if (OB_FAIL(columns_type->push_back(column_type))) { + LOG_WARN("failed to push back", K(ret)); + } else { + LOG_DEBUG("[yzfdebug] add missing index column for update/delete", + K(column_id), K(column_type)); + } + } + } + } // end if !found + } + } // end for + return ret; +} + +int ObTableService::delete_can_use_put(uint64_t table_id, bool &use_put) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard schema_guard; + const schema::ObTableSchema *table_schema = NULL; + if (OB_FAIL(schema_service_->get_schema_guard(schema_guard))) { + LOG_WARN("failed to get schema guard", K(ret)); + } else if (OB_FAIL(schema_guard.get_table_schema(table_id, table_schema))) { + LOG_WARN("get table schema failed", K(table_id), K(ret)); + } else if (OB_ISNULL(table_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("NULL ptr", K(ret), K(table_schema)); + } else { + // 1. there is not any index + use_put = (table_schema->get_index_tid_count() <= 0); + } + return ret; +} +//////////////////////////////////////////////////////////////// +int ObTableService::execute_delete(ObTableServiceGetCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + int64_t affected_rows = 0; + ObTableApiDeleteRowIterator delete_row_iter; + const uint64_t table_id = ctx.param_.table_id_; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(delete_row_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init delete row iterator, ", K(ret)); + } else if (OB_FAIL(delete_row_iter.open(table_operation))) { + LOG_WARN("Fail to open delete row iterator, ", K(ret)); + } else { + // init dml param + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = delete_row_iter.get_schema_version(); + if (OB_FAIL(part_service_->delete_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + delete_row_iter.get_delete_column_ids(), + &delete_row_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to delete", K(ret), K(table_id)); + } + } else { + LOG_DEBUG("[yzfdebug] delete rows", K(ret), K(affected_rows)); + } + } + + + if (ctx.param_.returning_affected_rows_) { + result.set_affected_rows(affected_rows); + } else { + result.set_affected_rows(-1); // always return -1 + } + result.set_errno(ret); + replace_ret_code(ret); + result.set_type(ObTableOperationType::DEL); + return ret; +} + +int ObTableService::cons_all_index_properties(schema::ObSchemaGetterGuard &schema_guard, + const schema::ObTableSchema &table_schema, + ObIArray &column_ids, + ObIArray *columns_type) +{ + int ret = OB_SUCCESS; + if (table_schema.get_index_tid_count() <= 0) { + // no index, do nothing + } else { + const schema::ObTableSchema *index_schema = NULL; + const uint64_t table_id = table_schema.get_table_id(); + uint64_t index_tids[OB_MAX_INDEX_PER_TABLE + 1]; + int64_t index_count = ARRAYSIZEOF(index_tids); + // get all the indexes + if (OB_FAIL(schema_guard.get_can_write_index_array(table_id, index_tids, index_count))) { + LOG_WARN("fail to get index", K(ret), K(table_id)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < index_count; ++i) { // for each writable index + const uint64_t index_id = index_tids[i]; + // get index schema + if (OB_FAIL(schema_guard.get_table_schema(index_id, index_schema))) { + LOG_WARN("get index schema failed", K(index_id)); + } else { + if (OB_FAIL(add_index_columns_if_missing(schema_guard, table_id, index_schema, + column_ids, columns_type))) { + LOG_WARN("failed to add missing index column", K(ret)); + } + } + } // end for + } + } + return ret; +} + +int ObTableService::multi_delete(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + int64_t affected_rows = 0; + ObTableApiMultiDeleteRowIterator delete_row_iter; + const uint64_t table_id = ctx.param_.table_id_; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(delete_row_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init delete row iterator, ", K(ret)); + } else if (OB_FAIL(delete_row_iter.open(batch_operation))) { + LOG_WARN("Fail to open delete row iterator, ", K(ret)); + } else { + // init dml param + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = delete_row_iter.get_schema_version(); + + int64_t result_idx = 0; + while(OB_SUCC(ret) && !delete_row_iter.has_finished()) { + affected_rows = 0; + delete_row_iter.continue_iter(); + // get one row + if (OB_FAIL(part_service_->delete_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + delete_row_iter.get_delete_column_ids(), + &delete_row_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to delete", K(ret), K(table_id)); + } + } else { + // add result for not exist rows + const int64_t cur_delete_idx = delete_row_iter.get_cur_delete_idx(); + for (; OB_SUCC(ret) && result_idx < cur_delete_idx; ++result_idx) { + if (OB_FAIL(add_one_result(result, ObTableOperationType::DEL, OB_SUCCESS, 0))) { + LOG_WARN("failed to add result", K(ret), K(result_idx)); + } else { + LOG_DEBUG("Success to add delete result, ", K(result_idx), K(cur_delete_idx), K(part_key)); + } + } // end for + // add result for this row + if (OB_SUCC(ret) && result_idx <= cur_delete_idx) { + if (OB_FAIL(add_one_result( + result, + ObTableOperationType::DEL, + OB_SUCCESS, + ctx.param_.returning_affected_rows_ ? affected_rows : -1))) { + LOG_WARN("failed to add result", K(ret)); + } else { + result_idx++; + LOG_DEBUG("Success to add delete result, ", K(result_idx), K(cur_delete_idx), K(affected_rows), K(part_key)); + } + } + } + } //end while + + if (OB_SUCC(ret)) { + const int64_t N = batch_operation.count(); + for (; OB_SUCC(ret) && result_idx < N; result_idx++) { + if (OB_FAIL(add_one_result(result, ObTableOperationType::DEL, OB_SUCCESS, 0))) { + LOG_WARN("failed to add result", K(ret), K(result_idx)); + } + } + } + } + return ret; +} + +//////////////////////////////////////////////////////////////// +int ObTableService::execute_insert( + ObTableServiceCtx &ctx, + const ObTableOperation &table_operation, + ObTableOperationResult &result, + ObNewRowIterator *&duplicate_row_iter) +{ + int ret = OB_SUCCESS; + duplicate_row_iter = NULL; + ObTableApiInsertRowIterator insert_iter; + ObNewRow *row = NULL; + const uint64_t table_id = ctx.param_.table_id_; + + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(insert_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init insert row iterator, ", K(ret)); + } else if (OB_FAIL(insert_iter.open(table_operation))) { + LOG_WARN("Fail to open insert row iterator, ", K(ret)); + } else if (OB_FAIL(insert_iter.get_next_row(row))) { + LOG_WARN("Fail to get next row, ", K(ret)); + } else { + int64_t affected_rows = 0; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + // init dml param + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = insert_iter.get_schema_version(); + + if (ObTableOperationType::INSERT_OR_UPDATE == table_operation.type()) { + // use insert row with duplicate rows + ObSEArray rowkey_column_ids; + ObIArray &full_column_ids = insert_iter.get_column_ids(); + const int64_t rowkey_column_cnt = table_operation.entity().get_rowkey_size(); + if (OB_FAIL(get_rowkey_column_ids(full_column_ids, rowkey_column_cnt, rowkey_column_ids))) { + LOG_WARN("failed to fill rowkey column ids, ", K(ret)); + } else if (OB_FAIL(part_service_->insert_row( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + full_column_ids, + rowkey_column_ids, + *row, + storage::ObInsertFlag::INSERT_RETURN_ONE_DUP, + affected_rows, + duplicate_row_iter))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to insert", K(ret), K(table_id)); + } + } + } else { + // directly insert + ObSingleRowIteratorWrapper single_row_iter(row); + if (OB_FAIL(part_service_->insert_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + insert_iter.get_column_ids(), + &single_row_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to insert", K(ret), K(table_id)); + } + } + } + + if (OB_SUCC(ret)) { + result.set_affected_rows(1); + // for execute_increment() + if (ObTableOperationType::INSERT != table_operation.type() /*increment/append*/ + && ctx.param_.returning_affected_entity_) { + // need to return the new values to the client + ObITableEntity *new_entity = NULL; + if (OB_FAIL(result.get_entity(new_entity)) || OB_ISNULL(new_entity)) { + LOG_WARN("failed to get entity", K(ret), K(new_entity)); + } else if (OB_FAIL(fill_new_entity( + ctx.param_.returning_rowkey_, + *row, + table_operation.entity().get_rowkey_size(), + insert_iter.get_properties(), + *ctx.param_.allocator_, + new_entity))) { + LOG_WARN("failed to return new entity", K(ret)); + } else { + LOG_DEBUG("[yzfdebug] increment return new entity", K(*new_entity)); + } + } + } + } + result.set_errno(ret); + replace_ret_code(ret); + result.set_type(table_operation.type()); + return ret; +} + +int ObTableService::multi_insert(ObTableServiceCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + ObTableApiMultiInsertRowIterator insert_iter; + const uint64_t table_id = ctx.param_.table_id_; + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(insert_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init multi insert iterator, ", K(ret)); + } else if (OB_FAIL(insert_iter.open(batch_operation))) { + LOG_WARN("Fail to open multi insert iterator, ", K(ret)); + } else { + int64_t affected_rows = 0; + // init dml param + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = insert_iter.get_schema_version(); + + const int64_t N = batch_operation.count(); + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { + insert_iter.continue_iter(); + if (OB_FAIL(part_service_->insert_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + insert_iter.get_column_ids(), + &insert_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to insert", K(ret), K(table_id)); + } + } + if (OB_SUCCESS == ret || OB_ERR_PRIMARY_KEY_DUPLICATE == ret) { + // ignore OB_ERR_PRIMARY_KEY_DUPLICATE + if (OB_FAIL(add_one_result(result, ObTableOperationType::INSERT, ret, affected_rows))) { + LOG_WARN("failed to add result", K(ret)); + } + } + } // end for + } + return ret; +} + +//////////////////////////////////////////////////////////////// +int ObTableService::do_replace( + ObTableServiceCtx &ctx, + ObPartitionKey &part_key, + storage::ObDMLBaseParam &dml_param, + ObIArray &column_ids, + ObIArray &rowkey_column_ids, + ObNewRow &row, + int64_t &affected_rows) +{ + int ret = OB_SUCCESS; + ObSingleRowIteratorWrapper single_row_iter(&row); + ObNewRowIterator *duplicated_rows = NULL; + int64_t del_rows = 0; + + // 1. try to insert + if (OB_FAIL(part_service_->insert_row( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + column_ids, + rowkey_column_ids, + row, + storage::ObInsertFlag::INSERT_RETURN_ALL_DUP, + affected_rows, + duplicated_rows))) { + if (OB_ERR_PRIMARY_KEY_DUPLICATE == ret) { + // 2. delete the row + if (OB_FAIL(do_replace_delete( + ctx, + part_key, + dml_param, + column_ids, + duplicated_rows, + del_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("Fail to delete replace duplicate row, ", K(ret), K(part_key)); + } + } else { + // 3. insert the row again + single_row_iter.reset(); + if (OB_FAIL(part_service_->insert_rows(ctx.param_.processor_->get_trans_desc(), dml_param, part_key, + column_ids, &single_row_iter, affected_rows))) { + if (OB_ERR_PRIMARY_KEY_DUPLICATE == ret) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("we have already delete the rowkey and lock it, maybe have unique index", K(ret), K(row)); + } else if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to insert", K(ret), K(part_key), K(row)); + } + } else { + // replace succ. + affected_rows += del_rows; + } + } + } else if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to insert", K(ret), K(part_key)); + } + } else { + // insert succ + } + + if (NULL != duplicated_rows) { + (void) part_service_->revert_insert_iter(part_key, duplicated_rows); + duplicated_rows = NULL; + } + return ret; +} + +int ObTableService::do_replace_delete( + ObTableServiceCtx &ctx, + common::ObPartitionKey &part_key, + storage::ObDMLBaseParam &dml_param, + common::ObIArray &column_ids, + common::ObNewRowIterator *duplicated_rows, + int64_t &affected_rows) +{ + int ret = OB_SUCCESS; + storage::ObTableScanParam scan_param; + ObNewRow *dup_row = NULL; + common::ObNewRowIterator *scan_iter = NULL; + + if (NULL == duplicated_rows) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Invalid duplicate row iterator, ", K(ret)); + } + + while (OB_SUCC(ret) && OB_SUCC(duplicated_rows->get_next_row(dup_row))) { + ObRowkey tmp_key(dup_row->cells_, dup_row->count_); + ObRowkey key; + if (OB_FAIL(tmp_key.deep_copy(key, *ctx.param_.allocator_))) { + LOG_WARN("fail to deep copy rowkey", K(ret)); + } else { + common::ObNewRange range; + if (OB_FAIL(range.build_range(part_key.table_id_, key))) { + LOG_WARN("fail to build key range", K(ret), K(part_key.table_id_), K(key)); + } else if (OB_FAIL(scan_param.key_ranges_.push_back(range))) { + LOG_WARN("fail to push back key range", K(ret), K(range)); + } + } + } + if (ret != OB_ITER_END) { + LOG_WARN("get next row not return ob_iter_end", K(ret)); + if (OB_SUCC(ret)) { + ret = OB_ERR_UNEXPECTED; + } + } else if (OB_FAIL(fill_query_scan_param( + ctx, + column_ids, + dml_param.schema_version_, + ObQueryFlag::ScanOrder::Forward, + part_key.table_id_, + -1, //limit + 0, //offset + scan_param))) { + LOG_WARN("fail to fill query scan param, ", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(scan_param, scan_iter))) { + LOG_WARN("fail to table scan, ", K(ret)); + } else if (OB_FAIL(part_service_->delete_rows(ctx.param_.processor_->get_trans_desc(), dml_param, part_key, + column_ids, scan_iter, affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to delete", K(ret), K(part_key)); + } + } else { + //delete success + } + + if (NULL != scan_iter) { + (void) part_service_->revert_scan_iter(scan_iter); + scan_iter = NULL; + } + return ret; +} + +int ObTableService::execute_replace(ObTableServiceCtx &ctx, const ObTableOperation &table_operation, ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + ObTableApiInsertRowIterator replace_iter; + ObNewRow *row = NULL; + const uint64_t table_id = ctx.param_.table_id_; + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(replace_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init replace iter, ", K(ret), K(table_id)); + } else if (OB_FAIL(replace_iter.open(table_operation))) { + LOG_WARN("Fail to open replace iter, ", K(ret), K(table_id)); + } else { + int64_t affected_rows = 0; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + // init dml param + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = replace_iter.get_schema_version(); + + ObIArray &full_column_ids = replace_iter.get_column_ids(); + const int64_t rowkey_column_cnt = table_operation.entity().get_rowkey_size(); + ObSEArray rowkey_column_ids; + if (OB_FAIL(get_rowkey_column_ids(full_column_ids, rowkey_column_cnt, rowkey_column_ids))) { + LOG_WARN("failed to fill rowkey column ids, ", K(ret)); + } else if (OB_FAIL(replace_iter.get_next_row(row))) { + LOG_WARN("Fail to get replace row, ", K(ret), K(table_id)); + } else if (OB_FAIL(do_replace(ctx, part_key, dml_param, + full_column_ids, rowkey_column_ids, *row, affected_rows))) { + LOG_WARN("failed to do replace", K(ret)); + } else { + result.set_affected_rows(affected_rows); + } + } + result.set_errno(ret); + replace_ret_code(ret); + result.set_type(ObTableOperationType::REPLACE); + return ret; +} + +int ObTableService::multi_replace(ObTableServiceCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + const uint64_t table_id = ctx.param_.table_id_; + ObTableApiMultiInsertRowIterator replace_iter; + ObNewRow *row = NULL; + + if (OB_INVALID_ID == table_id) { + ret = OB_ERR_UNKNOWN_TABLE; + LOG_WARN("table id is invalid", K(ret), K(table_id)); + } else if (OB_FAIL(replace_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init insert iter, ", K(ret), K(table_id)); + } else if (OB_FAIL(replace_iter.open(batch_operation))) { + LOG_WARN("Fail to open batch operation, ", K(ret), K(table_id)); + } else { + // init dml param + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = replace_iter.get_schema_version(); + + // fill column ids + ObIArray &column_ids = replace_iter.get_column_ids(); + ObSEArray rowkey_column_ids; + const ObTableOperation &one_operation = batch_operation.at(0); + const ObITableEntity &one_entity = one_operation.entity(); + + for (int64_t i = 0; OB_SUCC(ret) && i < one_entity.get_rowkey_size(); ++i) { + if (OB_FAIL(rowkey_column_ids.push_back(column_ids.at(i)))) { + LOG_WARN("failed to fill rowkey column ids, ", K(ret)); + } + } + + if (OB_SUCC(ret)) { + int64_t affected_rows = 0; + const int64_t N = batch_operation.count(); + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { + replace_iter.continue_iter(); + if (OB_FAIL(replace_iter.get_next_row(row))) { + LOG_WARN("Fail to get next replace row, ", K(ret), K(i)); + } else if (OB_FAIL(do_replace(ctx, part_key, dml_param, column_ids, rowkey_column_ids, *row, affected_rows))) { + LOG_WARN("failed to do replace", K(ret)); + } else if (OB_FAIL(add_one_result(result, ObTableOperationType::REPLACE, OB_SUCCESS, affected_rows))) { + LOG_WARN("failed to add result", K(ret)); + } + } // end for + } + } + return ret; +} +//////////////////////////////////////////////////////////////// +// update & multi_update +//////////////////////////////////////////////////////////////// + +int ObTableService::execute_update(ObTableServiceGetCtx &ctx, + const ObTableOperation &table_operation, ObNewRow *target_row, ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + int64_t affected_rows = 0; + const uint64_t table_id = ctx.param_.table_id_; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + ObTableApiUpdateRowIterator update_iter; + ObRowkey rowkey; + if (NULL == target_row) { + rowkey = const_cast(table_operation.entity()).get_rowkey(); + } else { + const int64_t rowkey_cnt = table_operation.entity().get_rowkey_size(); + if (OB_UNLIKELY(rowkey_cnt > target_row->count_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(rowkey_cnt), K(target_row)); + } else { + rowkey.assign(target_row->cells_, rowkey_cnt); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(update_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init update row iterator, ", K(ret)); + } else if (OB_FAIL(update_iter.open(table_operation, rowkey, + NULL != target_row/*need_update_rowkey*/))) { + LOG_WARN("Fail to open update row iterator, ", K(ret)); + } else { + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = update_iter.get_schema_version(); + + if (OB_FAIL(part_service_->update_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + update_iter.get_column_ids(), + update_iter.get_update_column_ids(), + &update_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to update_rows", K(ret), K(part_key)); + } + } else { + result.set_affected_rows(affected_rows); + } + } + + result.set_errno(ret); + replace_ret_code(ret); + result.set_type(ObTableOperationType::UPDATE); + return ret; +} + + +int ObTableService::fill_new_entity( + bool returning_rowkey, + const common::ObNewRow &row, + const int64_t primary_key_size, + const common::ObIArray &properties, + common::ObIAllocator &alloc, + ObITableEntity *new_entity) +{ + int ret = OB_SUCCESS; + new_entity->reset(); + ObObj cell_clone; + ObString name_clone; + if (primary_key_size > row.count_ || OB_ISNULL(new_entity)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected rowkey size", K(ret), K(primary_key_size), KP(new_entity)); + } else if (returning_rowkey) { + for (int64_t i = 0; i < primary_key_size && OB_SUCCESS == ret; ++i) + { + if (OB_FAIL(ob_write_obj(alloc, row.cells_[i], cell_clone))) { + LOG_WARN("failed to copy obj", K(ret)); + } else if (OB_FAIL(new_entity->add_rowkey_value(cell_clone))) { + LOG_WARN("failed to add rowkey value", K(ret), K(cell_clone)); + } + } + } + if (OB_SUCC(ret)) { + const int64_t N = primary_key_size + properties.count(); + for (int64_t i = primary_key_size, j = 0; OB_SUCCESS == ret && i < N; ++i, ++j) + { + // deep copy property + const ObString &name = properties.at(j); + const ObObj &cell = row.cells_[i]; + if (OB_FAIL(ob_write_string(alloc, name, name_clone))) { + LOG_WARN("failed to copy string", K(ret), K(name)); + } else if (OB_FAIL(ob_write_obj(alloc, cell, cell_clone))) { + LOG_WARN("failed to copy obj", K(ret)); + } else if (OB_FAIL(new_entity->set_property(name_clone, cell_clone))) { + LOG_WARN("failed to set property", K(ret)); + } else { + LOG_DEBUG("[yzfdebug] affected new cell", K(name), K(i), K(cell_clone)); + } + } // end for + } + return ret; +} + +int ObTableService::execute_increment_by_update( + ObTableServiceGetCtx &ctx, + const ObTableOperation &table_operation, + ObTableOperationResult &result) +{ + int ret = OB_SUCCESS; + int64_t affected_rows = 0; + ObTableApiUpdateRowIterator increment_row_iter; + const uint64_t table_id = ctx.param_.table_id_; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + ObRowkey rowkey = const_cast(table_operation.entity()).get_rowkey(); + if (OB_FAIL(increment_row_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init increment row iter, ", K(ret)); + } else if (OB_FAIL(increment_row_iter.open(table_operation, rowkey))) { + LOG_WARN("Fail to open increment row iter, ", K(ret)); + } else { + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = increment_row_iter.get_schema_version(); + + if (OB_FAIL(part_service_->update_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + increment_row_iter.get_column_ids(), + increment_row_iter.get_update_column_ids(), + &increment_row_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to update_rows", K(ret), K(part_key)); + } + } else { + if (affected_rows > 0) { + affected_rows = 1; + if (ctx.param_.returning_affected_entity_) { + // need to return the new values to the client + ObITableEntity *new_entity = NULL; + ObNewRow *new_row = NULL; + if (NULL == (new_row = increment_row_iter.get_cur_new_row())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("Unexpected error, the new row is null, ", K(ret), K(part_key)); + } else if (OB_FAIL(result.get_entity(new_entity)) || OB_ISNULL(new_entity)) { + LOG_WARN("failed to get entity", K(ret), K(new_entity)); + } else if (OB_FAIL(fill_new_entity( + ctx.param_.returning_rowkey_, + *new_row, + table_operation.entity().get_rowkey_size(), + increment_row_iter.get_properties(), + *ctx.param_.allocator_, + new_entity))) { + LOG_WARN("failed to return new entity", K(ret)); + } else { + LOG_DEBUG("[yzfdebug] increment return new entity", K(*new_entity)); + } + } + } + + if (NULL == (increment_row_iter.get_cur_new_row())) { + ret = OB_EMPTY_RESULT; + } + } + } + + if (OB_EMPTY_RESULT != ret) { + result.set_affected_rows(affected_rows); + result.set_errno(ret); + replace_ret_code(ret); + result.set_type(table_operation.type()); + } + return ret; +} + +int ObTableService::execute_increment(ObTableServiceGetCtx &ctx, + const ObTableOperation &table_operation, + ObTableOperationResult &result) +{ + int ret = execute_increment_by_update(ctx, table_operation, result); + if (OB_EMPTY_RESULT == ret) { + // the row not exist, insert it + ctx.reset_get_ctx(); + ObNewRowIterator *duplicate_row_iter = nullptr; + ret = execute_insert(ctx, table_operation, result, duplicate_row_iter); + if (OB_SUCC(ret) && OB_ERR_PRIMARY_KEY_DUPLICATE == result.get_errno()) { + // concurrent inserted by another thread, try again in process() + ret = OB_TRY_LOCK_ROW_CONFLICT; + LOG_WARN("insert failed again after update failed", K(ret), K(result)); + } + } + + if (OB_FAIL(ret)) { + LOG_WARN("Fail to execute increment, ", K(ret)); + } else { + LOG_DEBUG("Success to execute increment, ", K(result)); + } + return ret; +} + +//////////////////////////////////////////////////////////////// +// multi update +int ObTableService::multi_update(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + ObTableApiMultiUpdateRowIterator update_row_iter; + const uint64_t table_id = ctx.param_.table_id_; + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + int64_t affected_rows = 0; + int64_t result_idx = 0; + + if (OB_FAIL(update_row_iter.init(*part_service_, *schema_service_, ctx))) { + LOG_WARN("Fail to init multi update row iterator, ", K(ret)); + } else if (OB_FAIL(update_row_iter.open(batch_operation))) { + LOG_WARN("Fail to open multi update row iterator, ", K(ret)); + } else { + storage::ObDMLBaseParam dml_param; + dml_param.timeout_ = ctx.param_.timeout_ts_; + dml_param.is_total_quantity_log_ = (ObBinlogRowImageType::FULL == ctx.param_.binlog_row_image_type_); + dml_param.tz_info_ = NULL; + dml_param.sql_mode_ = SMO_DEFAULT; + dml_param.schema_version_ = update_row_iter.get_schema_version(); + + while(OB_SUCC(ret) && !update_row_iter.has_finished()) { + affected_rows = 0; + update_row_iter.continue_iter(); + if (OB_FAIL(part_service_->update_rows( + ctx.param_.processor_->get_trans_desc(), + dml_param, + part_key, + update_row_iter.get_column_ids(), + update_row_iter.get_update_column_ids(), + &update_row_iter, + affected_rows))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("failed to update_rows", K(ret), K(part_key)); + } + } else { + // add result for not exist rows + const int64_t cur_update_idx = update_row_iter.get_cur_update_idx(); + for (; OB_SUCC(ret) && result_idx < cur_update_idx; ++result_idx) { + if (OB_FAIL(add_one_result(result, ObTableOperationType::UPDATE, OB_SUCCESS, 0))) { + LOG_WARN("failed to add result", K(ret), K(result_idx)); + } else { + LOG_DEBUG("Success to add result, ", K(result_idx), K(cur_update_idx), K(part_key)); + } + } // end for + // add result for this row + if (OB_SUCC(ret) && result_idx <= cur_update_idx) { + if (OB_FAIL(add_one_result(result, ObTableOperationType::UPDATE, OB_SUCCESS, affected_rows))) { + LOG_WARN("failed to add result", K(ret)); + } else { + result_idx++; + LOG_DEBUG("Success to add result, ", K(result_idx), K(cur_update_idx), K(part_key)); + } + } + } + } // end for + + if (OB_SUCC(ret)) { + const int64_t N = batch_operation.count(); + for (; OB_SUCC(ret) && result_idx < N; result_idx++) { + if (OB_FAIL(add_one_result(result, ObTableOperationType::UPDATE, OB_SUCCESS, 0))) { + LOG_WARN("failed to add result", K(ret), K(result_idx)); + } + } + } + } + return ret; +} + +int ObTableService::batch_execute(ObTableServiceGetCtx &ctx, const ObTableBatchOperation &batch_operation, ObTableBatchOperationResult &result) +{ + int ret = OB_SUCCESS; + int64_t N = batch_operation.count(); + ObNewRowIterator *duplicate_row_iter = nullptr; + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + const ObTableOperation &table_operation = batch_operation.at(i); + ObTableOperationResult op_result; + ObITableEntity *result_entity = result.get_entity_factory()->alloc(); + if (NULL == result_entity) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc memroy for result_entity", K(ret)); + break; + } + op_result.set_entity(*result_entity); + switch(table_operation.type()) { + case ObTableOperationType::GET: + ret = execute_get(ctx, table_operation, op_result); + break; + case ObTableOperationType::INSERT: + ret = execute_insert(ctx, table_operation, op_result, duplicate_row_iter); + break; + case ObTableOperationType::DEL: + ret = execute_delete(ctx, table_operation, op_result); + break; + case ObTableOperationType::UPDATE: + ret = execute_update(ctx, table_operation, nullptr, op_result); + break; + case ObTableOperationType::INSERT_OR_UPDATE: + ret = execute_insert_or_update(ctx, table_operation, op_result); + break; + case ObTableOperationType::REPLACE: + ret = execute_replace(ctx, table_operation, op_result); + break; + case ObTableOperationType::APPEND: + case ObTableOperationType::INCREMENT: + ret = execute_increment(ctx, table_operation, op_result); + // no need to deep copy cells here which have already been deep copied in execute_increment() + break; + default: + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("unexpected operation type", "type", batch_operation.at(0).type()); + break; + } + if (OB_SUCC(ret)) { + if (OB_FAIL(result.push_back(op_result))) { + LOG_WARN("failed to push back result", K(ret)); + } else { + ctx.reset_get_ctx(); + } + } else { + LOG_WARN("failed to execute, ", K(ret), K(table_operation.type())); + } + } // end for + return ret; +} +//////////////////////////////////////////////////////////////// +// execute query +//////////////////////////////////////////////////////////////// +int ObTableService::cons_index_key_type(schema::ObSchemaGetterGuard &schema_guard, + const share::schema::ObTableSchema *index_schema, + uint64_t data_table_id, + common::ObIArray &columns_type) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_INVALID_ID; + const schema::ObColumnSchemaV2 *column_schema = NULL; + ObExprResType column_type; + const ObIndexInfo &index_key_info = index_schema->get_index_info(); + const int64_t N = index_key_info.get_size(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(index_key_info.get_column_id(i, column_id))) { + LOG_WARN("failed to get index column", K(ret), K(i)); + } else if (OB_FAIL(schema_guard.get_column_schema(data_table_id, column_id, column_schema))) { + LOG_WARN("get column schema failed", K(data_table_id), K(column_id)); + } else if (NULL == column_schema) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to get column schema", K(data_table_id), K(column_id)); + } else if (OB_FAIL(cons_column_type(*column_schema, column_type))) { + LOG_WARN("failed to cons column type", K(ret)); + } else if (OB_FAIL(columns_type.push_back(column_type))) { + LOG_WARN("failed to push back", K(ret)); + } else { + LOG_DEBUG("[yzfdebug] add index column for scan", + K(data_table_id), K(column_id), K(column_type)); + } + } // end for + return ret; +} + +int ObTableService::get_index_id_by_name(schema::ObSchemaGetterGuard &schema_guard, uint64_t base_table_id, + const ObString &index_name, uint64_t &index_id, + ObIArray &columns_type, + const share::schema::ObTableSchema *&index_schema) +{ + int ret = OB_SUCCESS; + uint64_t tids[OB_MAX_INDEX_PER_TABLE]; + int64_t table_index_count = OB_MAX_INDEX_PER_TABLE; + index_schema = nullptr; + if (index_name.empty() + || 0 == index_name.case_compare(ObStmtHint::PRIMARY_KEY)) { + index_id = base_table_id; + } else if (OB_FAIL(schema_guard.get_can_read_index_array(base_table_id, tids, table_index_count, false))) { + LOG_WARN("failed to get can read index", K(ret)); + } else if (table_index_count > OB_MAX_INDEX_PER_TABLE) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("Table index count is bigger than OB_MAX_INDEX_PER_TABLE", K(ret), K(table_index_count)); + } else { + LOG_DEBUG("get readable index", K(table_index_count)); + bool found = false; + for (int64_t i = 0; OB_SUCC(ret) && !found && i < table_index_count; ++i) { + uint64_t this_index_id = tids[i]; + ObString this_index; + if (OB_FAIL(schema_guard.get_table_schema(this_index_id, index_schema)) + || OB_ISNULL(index_schema)) { + ret = OB_SCHEMA_ERROR; + LOG_WARN("fail to get table schema", K(this_index_id), K(ret)); + } else if (OB_FAIL(index_schema->get_index_name(this_index))) { + LOG_WARN("fail to get index name", K(this_index), K(ret), K(this_index_id)); + } else if (0 != index_name.case_compare(this_index)) { + //do nothing, just continue + continue; + } else { + found = true; + index_id = this_index_id; + //record the column type in index to check the object type in scan range + ret = cons_index_key_type(schema_guard, index_schema, base_table_id, columns_type); + } + } // end for + if (!found) { + ret = OB_ERR_INDEX_UNKNOWN; + } + } + return ret; +} + +int ObTableService::fill_query_table_param(uint64_t table_id, + const ObIArray &properties, + const ObString &index_name, + share::schema::ObTableParam &table_param, + ObIArray &output_column_ids, + common::ObIArray &rowkey_columns_type, + int64_t &schema_version, + uint64_t &index_id, + int64_t &padding_num) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard schema_guard; + const schema::ObTableSchema *table_schema = NULL; + const schema::ObTableSchema *index_schema = NULL; + if (OB_FAIL(schema_service_->get_schema_guard(schema_guard))) { + LOG_WARN("failed to get schema guard", K(ret)); + } else if (OB_FAIL(schema_guard.get_table_schema(table_id, table_schema))) { + LOG_WARN("get table schema failed", K(table_id), K(ret)); + } else if (OB_ISNULL(table_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("NULL ptr", K(ret), K(table_schema)); + } else if (OB_FAIL(get_index_id_by_name(schema_guard, table_id, index_name, index_id, + rowkey_columns_type, index_schema))) { + LOG_WARN("failed to get index id", K(ret), K(index_name), K(table_id)); + } else { + NG_TRACE_EXT(idx, OB_ID(idx), index_id, OB_ID(tag1), index_name); + schema_version = table_schema->get_schema_version(); + const int64_t key_column_cnt = rowkey_columns_type.count(); + padding_num = (NULL == index_schema) ? -1 : (index_schema->get_rowkey_column_num() - key_column_cnt); + LOG_DEBUG("[xilin debug]padding", K(padding_num), K(key_column_cnt), K(index_name)); + + const bool index_back = (index_id != table_id); + if (OB_FAIL(cons_rowkey_infos(*table_schema, NULL, index_back ? NULL : &rowkey_columns_type))) { + } else if (OB_FAIL(cons_properties_infos(*table_schema, properties, output_column_ids, NULL))) { + } else if (OB_FAIL(table_param.convert(*table_schema, ((NULL == index_schema) ? *table_schema: *index_schema), + output_column_ids, index_back))) { + LOG_WARN("failed to convert table param", K(ret)); + } else { + //do nothing + } + } + return ret; +} + +int ObTableService::fill_query_scan_ranges(ObTableServiceCtx &ctx, + const ObTableQuery &query, + int64_t padding_num, + storage::ObTableScanParam &scan_param) +{ + int ret = OB_SUCCESS; + scan_param.key_ranges_.reset(); + const ObIArray &scan_ranges = query.get_scan_ranges(); + int64_t N = scan_ranges.count(); + // check obj type in ranges + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) // foreach range + { + const ObNewRange &range = scan_ranges.at(i); + // check column type + for (int64_t j = 0; OB_SUCCESS == ret && j < 2; ++j) + { + const ObRowkey *p_key = nullptr; + if (0 == j) { + p_key = &range.get_start_key(); + } else { + p_key = &range.get_end_key(); + } + if (p_key->is_min_row() || p_key->is_max_row()) { + continue; + } else { + if (p_key->get_obj_cnt() != ctx.columns_type_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("wrong rowkey size", K(ret), K(i), K(j), K(*p_key), K_(ctx.columns_type)); + } else { + const int64_t M = p_key->get_obj_cnt(); + for (int64_t k = 0; OB_SUCCESS == ret && k < M; ++k) + { + ObObj &obj = const_cast(p_key->get_obj_ptr()[k]); + if (obj.is_min_value() || obj.is_max_value()) { + continue; + } else if (OB_FAIL(ObTableService::check_column_type(ctx.columns_type_.at(k), obj))) { + } + } + } + } // end else + } // end for + if (OB_UNLIKELY(padding_num > 0)) { + // index scan need fill primary key object + ObNewRange index_key_range = range; + for (int64_t j = 0; OB_SUCCESS == ret && j < 2; ++j) + { + const ObRowkey *p_key = nullptr; + if (0 == j) { + p_key = &range.get_start_key(); + } else { + p_key = &range.get_end_key(); + } + if (p_key->is_min_row() || p_key->is_max_row()) { + continue; + } else { + const int64_t old_objs_num = p_key->get_obj_cnt(); + const int64_t new_objs_num = old_objs_num + padding_num; + ObObj *new_objs = static_cast(ctx.param_.allocator_->alloc(sizeof(ObObj)*new_objs_num)); + if (nullptr == new_objs) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("no memory", K(ret)); + } else { + const ObObj *old_objs = p_key->get_obj_ptr(); + for (int64_t k = 0; k < old_objs_num; ++k) + { + new_objs[k] = old_objs[k]; // shallow copy + } // end for + if (0 == j) { // padding for startkey + for (int64_t k = 0; k < padding_num; ++k) { + new_objs[k+old_objs_num] = ObObj::make_min_obj(); + } + index_key_range.start_key_.assign(new_objs, new_objs_num); + } else { // padding for endkey + for (int64_t k = 0; k < padding_num; ++k) { + new_objs[k+old_objs_num] = ObObj::make_max_obj(); + } + index_key_range.end_key_.assign(new_objs, new_objs_num); + } + } + } + } // end for + if (OB_SUCC(ret)) { + if (OB_FAIL(scan_param.key_ranges_.push_back(index_key_range))) { + LOG_WARN("fail to push back key range", K(ret), K(index_key_range)); + } else { + LOG_DEBUG("[yzfdebug] add key range for index scan", K(ret), K(index_key_range)); + } + } + } else { + if (OB_SUCC(ret)) { + if (OB_FAIL(scan_param.key_ranges_.push_back(range))) { + LOG_WARN("fail to push back key range", K(ret), K(range)); + } + } + } + } // end for + return ret; +} + +int ObTableService::fill_query_scan_param(ObTableServiceCtx &ctx, + const ObIArray &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) +{ + int ret = OB_SUCCESS; + const uint64_t table_id = ctx.param_.table_id_; + const bool index_back = (index_id != table_id); + ObPartitionKey part_key(table_id, ctx.param_.partition_id_, 0); + scan_param.timeout_ = ctx.param_.timeout_ts_; + ObQueryFlag query_flag(scan_order, + false, // daily_merge + false, // optimize + false, // whole_macro_scan + false, // full_row + index_back, // index_back + false, // query_stat + ObQueryFlag::MysqlMode, // sql_mode + true // read_latest + ); + scan_param.scan_flag_.flag_ = query_flag.flag_; + scan_param.reserved_cell_count_ = output_column_ids.count() + 10; + scan_param.for_update_ = false; + scan_param.column_ids_.reset(); + scan_param.pkey_ = part_key; + scan_param.schema_version_ = schema_version; + if (OB_FAIL(scan_param.column_ids_.assign(output_column_ids))) { + LOG_WARN("fail to assign column id", K(ret)); + } else { + SQL_ENG_LOG(DEBUG, "set scan param", K(output_column_ids)); + scan_param.expr_ctx_.calc_buf_ = NULL; + scan_param.expr_ctx_.my_session_ = NULL; + scan_param.expr_ctx_.phy_plan_ctx_ = NULL; + scan_param.limit_param_.limit_ = limit; + scan_param.limit_param_.offset_ = offset; + scan_param.trans_desc_ = &ctx.param_.processor_->get_trans_desc(); + scan_param.index_id_ = index_id; + scan_param.sql_mode_ = SMO_DEFAULT; + scan_param.allocator_->set_tenant_id(scan_param.pkey_.get_tenant_id()); + LOG_DEBUG("[yzfdebug] scan param", K(scan_param)); + } + return ret; +} + +int ObNormalTableQueryResultIterator::get_next_result(table::ObTableQueryResult *&next_result) +{ + int ret = OB_SUCCESS; + if (is_first_result_) { + is_first_result_ = false; + if (0 != one_result_.get_property_count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("property should be empty", K(ret)); + } + const ObIArray &select_columns = query_->get_select_columns(); + const int64_t N = select_columns.count(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(one_result_.add_property_name(select_columns.at(i)))) { + LOG_WARN("failed to copy name", K(ret)); + } + } // end for + last_row_ = NULL; + } else { + one_result_.reset_except_property(); + } + + if (OB_SUCC(ret)) { + if (NULL != last_row_) { + if (OB_FAIL(one_result_.add_row(*last_row_))) { + LOG_WARN("failed to add row, ", K(ret)); + } + last_row_ = NULL; + } + } + + if (OB_SUCC(ret)) { + next_result = &one_result_; + ObNewRow *row = nullptr; + while (OB_SUCC(ret) && OB_SUCC(scan_result_->get_next_row(row))) { + LOG_DEBUG("[yzfdebug] scan result", "row", *row); + if (OB_FAIL(one_result_.add_row(*row))) { + if (OB_SIZE_OVERFLOW == ret) { + ret = OB_SUCCESS; + last_row_ = row; + break; + } else { + LOG_WARN("failed to add row", K(ret)); + } + } else if (one_result_.reach_batch_size_or_result_size(batch_size_, max_result_size_)) { + NG_TRACE(tag9); + break; + } else { + LOG_DEBUG("[yzfdebug] scan return one row", "row", *row); + } + } // end while + if (OB_ITER_END == ret) { + has_more_rows_ = false; + if (one_result_.get_row_count() > 0) { + ret = OB_SUCCESS; + } + } + } + return ret; +} + +bool ObNormalTableQueryResultIterator::has_more_result() const +{ + return has_more_rows_; +} + +ObNormalTableQueryResultIterator *ObTableServiceQueryCtx::get_normal_result_iterator( + const ObTableQuery &query, table::ObTableQueryResult &one_result) +{ + if (NULL == normal_result_iterator_) { + normal_result_iterator_ = OB_NEWx(ObNormalTableQueryResultIterator, param_.allocator_, query, one_result); + if (NULL == normal_result_iterator_) { + LOG_WARN("failed to allocate result iterator"); + } + } + return normal_result_iterator_; +} + +void ObTableServiceQueryCtx::destroy_result_iterator(storage::ObPartitionService *part_service) +{ + if (NULL != normal_result_iterator_) { + normal_result_iterator_->~ObNormalTableQueryResultIterator(); + normal_result_iterator_ = NULL; + } + if (NULL != scan_result_) { + if (NULL == part_service) { + LOG_ERROR("part_service is NULL, memory leak"); + } else { + part_service->revert_scan_iter(scan_result_); + scan_result_ = NULL; + } + } +} + +int ObTableService::execute_query(ObTableServiceQueryCtx &ctx, const ObTableQuery &query, + table::ObTableQueryResult &one_result, + table::ObTableQueryResultIterator *&query_result) +{ + int ret = OB_SUCCESS; + ObSEArray output_column_ids; + int64_t schema_version = 0; + ctx.scan_result_ = NULL; + const uint64_t table_id = ctx.param_.table_id_; + uint64_t index_id = OB_INVALID_ID; + int64_t padding_num = 0; + + if (NULL == (query_result = ctx.get_normal_result_iterator(query, one_result))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate result iterator", K(ret)); + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(fill_query_table_param(table_id, query.get_select_columns(), + query.get_index_name(), + *(ctx.table_param_), output_column_ids, + ctx.columns_type_, schema_version, + index_id, padding_num))) { // @todo optimize, table_param_ can be cached + LOG_WARN("failed to fill param", K(ret)); + } else if (OB_FAIL(fill_query_scan_ranges(ctx, query, + (table_id != index_id) ? padding_num : -1, + ctx.scan_param_))) { + LOG_WARN("failed to fill range", K(ret)); + } else if (OB_FAIL(fill_query_scan_param(ctx, output_column_ids, schema_version, + query.get_scan_order(), index_id, query.get_limit(), + query.get_offset(), ctx.scan_param_))) { + LOG_WARN("failed to fill param", K(ret)); + } else if (OB_FAIL(part_service_->table_scan(ctx.scan_param_, ctx.scan_result_))) { + if (OB_TRY_LOCK_ROW_CONFLICT != ret) { + LOG_WARN("fail to scan table", K(ret)); + } + } else { + ctx.normal_result_iterator_->set_scan_result(ctx.scan_result_); + } + return ret; +} diff --git a/src/observer/table/ob_table_service.h b/src/observer/table/ob_table_service.h new file mode 100644 index 0000000000..f79fa035d2 --- /dev/null +++ b/src/observer/table/ob_table_service.h @@ -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 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(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 *column_ids, + common::ObIArray *columns_type); + static int cons_properties_infos(const share::schema::ObTableSchema &table_schema, + const common::ObIArray &properties, + common::ObIArray &column_ids, + common::ObIArray *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 &column_ids, + common::ObIArray *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 &output_column_ids, + int64_t schema_version, + storage::ObTableScanParam &scan_param); + int fill_get_result( + ObTableServiceCtx &ctx, + const ObIArray &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 &column_ids, + common::ObIArray *columns_type); + + // for replace + int do_replace( + ObTableServiceCtx &ctx, + common::ObPartitionKey &part_key, + storage::ObDMLBaseParam &dml_param, + common::ObIArray &column_ids, + common::ObIArray &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 &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 &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 &columns_type, + const share::schema::ObTableSchema *&index_schema); + int fill_query_table_param(uint64_t table_id, + const common::ObIArray &properties, + const ObString &index_name, + share::schema::ObTableParam &table_param, + common::ObIArray &output_column_ids, + common::ObIArray &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 &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 &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 */ diff --git a/src/share/CMakeLists.txt b/src/share/CMakeLists.txt index 4e8b183a03..e465639997 100644 --- a/src/share/CMakeLists.txt +++ b/src/share/CMakeLists.txt @@ -34,6 +34,11 @@ ob_set_subtarget(ob_share config 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") ob_set_subtarget(ob_share inner_table inner_table/ob_inner_table_schema_misc.ipp diff --git a/src/share/table/ob_table.cpp b/src/share/table/ob_table.cpp new file mode 100644 index 0000000000..82f834dfe4 --- /dev/null +++ b/src/share/table/ob_table.cpp @@ -0,0 +1,1381 @@ +/** + * 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.h" +#include "share/ob_errno.h" +#include "lib/utility/ob_unify_serialize.h" +#include "common/row/ob_row.h" +#include "rpc/obrpc/ob_rpc_packet.h" + +using namespace oceanbase::table; +using namespace oceanbase::common; + +OB_DEF_SERIALIZE(ObITableEntity) +{ + int ret = OB_SUCCESS; + if (OB_SUCC(ret)) { + const int64_t rowkey_size = get_rowkey_size(); + OB_UNIS_ENCODE(rowkey_size); + ObObj obj; + for (int64_t i = 0; i < rowkey_size && OB_SUCCESS == ret; ++i) { + if (OB_FAIL(this->get_rowkey_value(i, obj))) { + LOG_WARN("failed to get value", K(ret), K(i)); + } + OB_UNIS_ENCODE(obj); + } + } + if (OB_SUCC(ret)) { + ObSEArray, 8> properties; + if (OB_FAIL(this->get_properties(properties))) { // @todo optimize, use iterator + LOG_WARN("failed to get properties", K(ret)); + } else { + const int64_t properties_count = properties.count(); + OB_UNIS_ENCODE(properties_count); + for (int64_t i = 0; i < properties_count && OB_SUCCESS == ret; ++i) { + const std::pair &kv_pair = properties.at(i); + OB_UNIS_ENCODE(kv_pair.first); + OB_UNIS_ENCODE(kv_pair.second); + } + } + } + return ret; +} + +OB_DEF_DESERIALIZE(ObITableEntity) +{ + int ret = OB_SUCCESS; + reset(); + ObString key; + ObObj value; + if (NULL == alloc_) { + // shallow copy + if (OB_SUCC(ret)) { + int64_t rowkey_size = -1; + OB_UNIS_DECODE(rowkey_size); + for (int64_t i = 0; OB_SUCCESS == ret && i < rowkey_size; ++i) { + OB_UNIS_DECODE(value); + if (OB_SUCC(ret)) { + if (OB_FAIL(this->add_rowkey_value(value))) { + LOG_WARN("failed to add rowkey value", K(ret), K(value)); + } + } + } + } + if (OB_SUCC(ret)) { + int64_t properties_count = -1; + OB_UNIS_DECODE(properties_count); + for (int64_t i = 0; i < properties_count && OB_SUCCESS == ret; ++i) { + OB_UNIS_DECODE(key); + OB_UNIS_DECODE(value); + if (OB_SUCC(ret)) { + if (OB_FAIL(this->set_property(key, value))) { + LOG_WARN("failed to set property", K(ret), K(key), K(value)); + } + } + } + } + } else { + // deep copy + ObObj value_clone; + if (OB_SUCC(ret)) { + int64_t rowkey_size = -1; + OB_UNIS_DECODE(rowkey_size); + for (int64_t i = 0; OB_SUCCESS == ret && i < rowkey_size; ++i) { + OB_UNIS_DECODE(value); + if (OB_SUCC(ret)) { + if (OB_FAIL(ob_write_obj(*alloc_, value, value_clone))) { + LOG_WARN("failed to copy value", K(ret)); + } else if (OB_FAIL(this->add_rowkey_value(value_clone))) { + LOG_WARN("failed to add rowkey value", K(ret), K(value_clone)); + } + } + } + } + if (OB_SUCC(ret)) { + ObString key_clone; + int64_t properties_count = -1; + OB_UNIS_DECODE(properties_count); + for (int64_t i = 0; i < properties_count && OB_SUCCESS == ret; ++i) { + OB_UNIS_DECODE(key); + OB_UNIS_DECODE(value); + if (OB_SUCC(ret)) { + if (OB_FAIL(ob_write_string(*alloc_, key, key_clone))) { + LOG_WARN("failed to clone string", K(ret)); + } else if (OB_FAIL(ob_write_obj(*alloc_, value, value_clone))) { + LOG_WARN("failed to copy value", K(ret)); + } else if (OB_FAIL(this->set_property(key_clone, value_clone))) { + LOG_WARN("failed to set property", K(ret), K(key), K(value_clone)); + } + } + } + } + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObITableEntity) +{ + int64_t len = 0; + int ret = OB_SUCCESS; + ObString key; + ObObj value; + const int64_t rowkey_size = get_rowkey_size(); + OB_UNIS_ADD_LEN(rowkey_size); + for (int64_t i = 0; i < rowkey_size && OB_SUCCESS == ret; ++i) { + if (OB_FAIL(this->get_rowkey_value(i, value))) { + LOG_WARN("failed to get value", K(ret), K(i)); + } + OB_UNIS_ADD_LEN(value); + } + if (OB_SUCC(ret)) { + ObSEArray, 8> properties; + if (OB_FAIL(this->get_properties(properties))) { // @todo optimize, use iterator + LOG_WARN("failed to get properties", K(ret)); + } else { + const int64_t properties_count = properties.count(); + OB_UNIS_ADD_LEN(properties_count); + for (int64_t i = 0; i < properties_count && OB_SUCCESS == ret; ++i) { + const std::pair &kv_pair = properties.at(i); + OB_UNIS_ADD_LEN(kv_pair.first); + OB_UNIS_ADD_LEN(kv_pair.second); + } + } + } + return len; +} + +int ObITableEntity::deep_copy(common::ObIAllocator &allocator, const ObITableEntity &other) +{ + int ret = OB_SUCCESS; + reset(); + if (OB_FAIL(deep_copy_rowkey(allocator, other))) { + } else if (OB_FAIL(deep_copy_properties(allocator, other))) { + } + return ret; +} + +int ObITableEntity::deep_copy_rowkey(common::ObIAllocator &allocator, const ObITableEntity &other) +{ + int ret = OB_SUCCESS; + ObObj value; + ObObj cell_clone; + const int64_t rowkey_size = other.get_rowkey_size(); + for (int64_t i = 0; OB_SUCCESS == ret && i < rowkey_size; ++i) + { + if (OB_FAIL(other.get_rowkey_value(i, value))) { + LOG_WARN("failed to get rowkey value", K(ret), K(i), K(value)); + } else if (OB_FAIL(ob_write_obj(allocator, value, cell_clone))) { + LOG_WARN("failed to copy cell", K(ret)); + } else if (OB_FAIL(this->add_rowkey_value(cell_clone))) { + LOG_WARN("failed to add rowkey value", K(ret), K(value)); + } + } // end for + return ret; +} + +int ObITableEntity::deep_copy_properties(common::ObIAllocator &allocator, const ObITableEntity &other) +{ + int ret = OB_SUCCESS; + ObObj value; + ObObj cell_clone; + ObString name_clone; + ObSEArray, 8> properties; + if (OB_FAIL(other.get_properties(properties))) { // @todo optimize, use iterator + LOG_WARN("failed to get properties", K(ret)); + } else { + const int64_t properties_count = properties.count(); + for (int64_t i = 0; i < properties_count && OB_SUCCESS == ret; ++i) { + const std::pair &kv_pair = properties.at(i); + if (OB_FAIL(ob_write_string(allocator, kv_pair.first, name_clone))) { + LOG_WARN("failed to clone string", K(ret)); + } else if (OB_FAIL(ob_write_obj(allocator, kv_pair.second, cell_clone))) { + LOG_WARN("failed to copy cell", K(ret)); + } else if (OB_FAIL(this->set_property(name_clone, cell_clone))) { + LOG_WARN("failed to set property", K(ret)); + } + } // end for + } + return ret; +} + +int ObITableEntity::add_retrieve_property(const ObString &prop_name) +{ + ObObj null_obj; + return set_property(prop_name, null_obj); +} + +//////////////////////////////////////////////////////////////// +ObTableEntity::ObTableEntity() +{} + +ObTableEntity::~ObTableEntity() +{} + +inline int ObTableEntity::try_init() +{ + int ret = OB_SUCCESS; + static const int64_t PROPERTY_MAP_BUCKET_SIZE = 107; + if (!properties_.created()) { + if (OB_FAIL(properties_.create(PROPERTY_MAP_BUCKET_SIZE, ObModIds::OB_HASH_BUCKET_SQL_COLUMN_MAP, + ObModIds::OB_HASH_NODE_SQL_COLUMN_MAP))) { + LOG_WARN("failed to init properties", K(ret)); + } + } + return ret; +} + +int ObTableEntity::set_rowkey_value(int64_t idx, const ObObj &value) +{ + int ret = OB_SUCCESS; + if (idx < 0) { + ret = OB_INDEX_OUT_OF_RANGE; + } else if (idx < rowkey_.count()) { + rowkey_.at(idx) = value; + } else { + int64_t N = rowkey_.count(); + ObObj null_obj; + for (int64_t i = N; OB_SUCC(ret) && i < idx; ++i) { + if (OB_FAIL(rowkey_.push_back(null_obj))) { + LOG_WARN("failed to pad null obj", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(rowkey_.push_back(value))) { + LOG_WARN("failed to add value obj", K(ret), K(value)); + } + } + } + return ret; +} + +int ObTableEntity::add_rowkey_value(const ObObj &value) +{ + return rowkey_.push_back(value); +} + +int ObTableEntity::get_rowkey_value(int64_t idx, ObObj &value) const +{ + return rowkey_.at(idx, value); +} + +int ObTableEntity::set_rowkey(const ObRowkey &rowkey) +{ + int ret = OB_SUCCESS; + rowkey_.reset(); + const int64_t N = rowkey.get_obj_cnt(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(rowkey_.push_back(rowkey.ptr()[i]))) { + LOG_WARN("failed to push back rowkey", K(ret), K(i)); + } + } // end for + return ret; +} + +int ObTableEntity::set_rowkey(const ObITableEntity &other) +{ + int ret = OB_SUCCESS; + const ObTableEntity *other_entity = dynamic_cast(&other); + if (NULL == other_entity) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid type of other entity"); + } else { + rowkey_.reset(); + int64_t N = other_entity->rowkey_.count(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(rowkey_.push_back(other_entity->rowkey_.at(i)))) { + LOG_WARN("failed to push back rowkey"); + } + } // end for + } + return ret; +} + +int64_t ObTableEntity::hash_rowkey() const +{ + int64_t hash_value = 0; + const int64_t N = rowkey_.count(); + for (int64_t i = 0; i < N; ++i) + { + hash_value = rowkey_.at(i).hash(hash_value); + } // end for + return hash_value; +} + +int ObTableEntity::get_property(const ObString &prop_name, ObObj &prop_value) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(const_cast(this)->try_init())) { + LOG_WARN("failed to init hash map", K(ret)); + } else if (OB_FAIL(properties_.get_refactored(prop_name, prop_value))) { + LOG_DEBUG("failed to get property", K(ret), K(prop_name)); + } + return ret; +} + +int ObTableEntity::set_property(const ObString &prop_name, const ObObj &prop_value) +{ + int ret = OB_SUCCESS; + if (prop_name.empty()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("property name should not be empty string", K(ret), K(prop_name)); + } else if (OB_FAIL(try_init())) { + LOG_WARN("failed to init hash map", K(ret)); + } else if (OB_FAIL(properties_.set_refactored(prop_name, prop_value, 1))) { + LOG_WARN("failed to set property", K(ret), K(prop_name)); + } + return ret; +} + +class ObTableEntity::GetPropertyFn +{ +public: + GetPropertyFn(ObIArray > &properties) + :properties_(properties) + {} + bool operator()(const hash::HashMapPair &kv) + { + int ret = OB_SUCCESS; + if (OB_FAIL(properties_.push_back(std::make_pair(kv.first, kv.second)))) { + LOG_WARN("failed to push back", K(ret)); + } + return OB_SUCCESS == ret; + } +private: + ObIArray > &properties_; +}; + +int ObTableEntity::get_properties(ObIArray > &properties) const +{ + int ret = OB_SUCCESS; + GetPropertyFn func(properties); + if (OB_FAIL(const_cast(this)->try_init())) { + LOG_WARN("failed to init hash map", K(ret)); + } else if (OB_FAIL(const_cast(properties_).foreach_refactored(func))) { + LOG_WARN("for each properties fail", K(ret)); + } + return ret; +} + +class ObTableEntity::GetPropertyNameFn +{ +public: + GetPropertyNameFn(ObIArray &names) + :names_(names) + {} + bool operator()(const hash::HashMapPair &kv) + { + int ret = OB_SUCCESS; + if (OB_FAIL(names_.push_back(kv.first))) { + LOG_WARN("failed to push back", K(ret)); + } + return OB_SUCCESS == ret; + } +private: + ObIArray &names_; +}; + +int ObTableEntity::get_properties_names(ObIArray &properties_names) const +{ + int ret = OB_SUCCESS; + GetPropertyNameFn func(properties_names); + if (OB_FAIL(const_cast(this)->try_init())) { + LOG_WARN("failed to init hash map", K(ret)); + } else if (OB_FAIL(const_cast(properties_).foreach_refactored(func))) { + LOG_WARN("for each properties fail", K(ret)); + } + return ret; +} + +class ObTableEntity::GetPropertyValueFn +{ +public: + GetPropertyValueFn(ObIArray &values) + :values_(values) + {} + bool operator()(const hash::HashMapPair &kv) + { + int ret = OB_SUCCESS; + if (OB_FAIL(values_.push_back(kv.second))) { + LOG_WARN("failed to push back", K(ret)); + } + return OB_SUCCESS == ret; + } +private: + ObIArray &values_; +}; + +int ObTableEntity::get_properties_values(ObIArray &properties_values) const +{ + int ret = OB_SUCCESS; + GetPropertyValueFn func(properties_values); + if (OB_FAIL(const_cast(this)->try_init())) { + LOG_WARN("failed to init hash map", K(ret)); + } else if (OB_FAIL(const_cast(properties_).foreach_refactored(func))) { + LOG_WARN("for each properties fail", K(ret)); + } + return ret; +} + +int64_t ObTableEntity::get_properties_count() const +{ + return properties_.size(); +} + +ObRowkey ObTableEntity::get_rowkey() +{ + ObRowkey rowkey; + int64_t obj_cnt = rowkey_.count(); + if (obj_cnt > 0) { + rowkey.assign(&rowkey_.at(0), obj_cnt); + } + return rowkey; +} + +DEF_TO_STRING(ObTableEntity) +{ + int64_t pos = 0; + J_OBJ_START(); + + J_NAME("rowkey"); + J_COLON(); + BUF_PRINTO(rowkey_); + + ObSEArray, 8> properties; + int ret = OB_SUCCESS; + if (OB_FAIL(this->get_properties(properties))) { + LOG_WARN("failed to get properties", K(ret)); + } else { + J_COMMA(); + J_NAME("properties"); + J_COLON(); + J_OBJ_START(); + int64_t N = properties.count(); + for (int64_t i = 0; OB_SUCCESS == ret && i < N; ++i) + { + const std::pair &prop = properties.at(i); + if (0 != i) { + J_COMMA(); + } + BUF_PRINTF("%.*s", prop.first.length(), prop.first.ptr()); + J_COLON(); + BUF_PRINTO(prop.second); + } + J_OBJ_END(); + } + J_OBJ_END(); + return pos; +} + +//////////////////////////////////////////////////////////////// +ObTableOperation ObTableOperation::insert(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::INSERT; + op.entity_ = &entity; + return op; +} + +ObTableOperation ObTableOperation::del(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::DEL; + op.entity_ = &entity; + return op; +} + +ObTableOperation ObTableOperation::update(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::UPDATE; + op.entity_ = &entity; + return op; +} + +ObTableOperation ObTableOperation::insert_or_update(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::INSERT_OR_UPDATE; + op.entity_ = &entity; + return op; +} + +ObTableOperation ObTableOperation::replace(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::REPLACE; + op.entity_ = &entity; + return op; +} + +ObTableOperation ObTableOperation::retrieve(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::GET; + op.entity_ = &entity; + return op; +} + +ObTableOperation ObTableOperation::increment(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::INCREMENT; + op.entity_ = &entity; + return op; +} + +ObTableOperation ObTableOperation::append(const ObITableEntity &entity) +{ + ObTableOperation op; + op.operation_type_ = ObTableOperationType::APPEND; + op.entity_ = &entity; + return op; +} + +int ObTableOperation::get_entity(ObITableEntity *&entity) +{ + int ret = OB_SUCCESS; + if (NULL == entity_) { + ret = OB_ERR_NULL_VALUE; + } else { + entity = const_cast(this->entity_); + } + return ret; +} + +uint64_t ObTableOperation::get_checksum() +{ + uint64_t checksum = 0; + ObITableEntity *entity = NULL; + if (OB_SUCCESS != get_entity(entity) || OB_ISNULL(entity)) { + // ignore + } else { + const int64_t rowkey_size = entity->get_rowkey_size(); + const int64_t property_count = entity->get_properties_count(); + checksum = ob_crc64(checksum, &rowkey_size, sizeof(rowkey_size)); + checksum = ob_crc64(checksum, &property_count, sizeof(property_count)); + } + checksum = ob_crc64(checksum, &operation_type_, sizeof(operation_type_)); + return checksum; +} + +OB_SERIALIZE_MEMBER(ObTableOperation, operation_type_, const_cast(*entity_)); +//////////////////////////////////////////////////////////////// +ObTableRequestOptions::ObTableRequestOptions() + :consistency_level_(ObTableConsistencyLevel::STRONG), + server_timeout_us_(10*1000*1000), + max_execution_time_us_(10*1000*1000), + retry_policy_(NULL), + returning_affected_rows_(false), + returning_rowkey_(false), + returning_affected_entity_(false), + binlog_row_image_type_(ObBinlogRowImageType::FULL) +{} + +//////////////////////////////////////////////////////////////// +int ObTableBatchOperation::add(const ObTableOperation &table_operation) +{ + int ret = table_operations_.push_back(table_operation); + if (OB_SUCC(ret)) { + if (is_readonly_ && ObTableOperationType::GET != table_operation.type()) { + is_readonly_ = false; + } + if (is_same_type_) { + const int64_t N = table_operations_.count(); + if (N >= 2 && table_operations_.at(N-1).type() != table_operations_.at(N-2).type()) { + is_same_type_ = false; + } + } + if (is_same_properties_names_) { + const int64_t num = table_operations_.count(); + if (num >= 2) { + const ObTableOperation &prev = table_operations_.at(num-2); + const ObTableOperation &curr = table_operations_.at(num-1); + ObSEArray prev_columns; + ObSEArray curr_columns; + if (OB_FAIL(prev.entity().get_properties_names(prev_columns))) { + LOG_WARN("failed to get retrieve columns", K(ret)); + } else if (OB_FAIL(curr.entity().get_properties_names(curr_columns))) { + LOG_WARN("failed to get retrieve columns", K(ret)); + } else if (prev_columns.count() != curr_columns.count()) { + is_same_properties_names_ = false; + } else if (prev_columns.count() == 1) { + is_same_properties_names_ = (prev_columns.at(0) == curr_columns.at(0)); + } else { + int64_t N = prev_columns.count(); + ObObj value; + for (int64_t i = 0; OB_SUCCESS == ret && i < N && is_same_properties_names_; ++i) + { + const ObString &name = prev_columns.at(i); + if (OB_FAIL(curr.entity().get_property(name, value))) { + if (OB_HASH_NOT_EXIST == ret) { + is_same_properties_names_ = false; + } + } + } // end for + } + } + } + } + return ret; +} + +int ObTableBatchOperation::insert(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::insert(entity); + return add(op); +} + +int ObTableBatchOperation::del(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::del(entity); + return add(op); +} + +int ObTableBatchOperation::update(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::update(entity); + return add(op); +} + +int ObTableBatchOperation::insert_or_update(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::insert_or_update(entity); + return add(op); +} + +int ObTableBatchOperation::replace(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::replace(entity); + return add(op); +} + +int ObTableBatchOperation::retrieve(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::retrieve(entity); + return add(op); +} + +int ObTableBatchOperation::increment(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::increment(entity); + return add(op); +} + +int ObTableBatchOperation::append(const ObITableEntity &entity) +{ + ObTableOperation op = ObTableOperation::append(entity); + return add(op); +} + +uint64_t ObTableBatchOperation::get_checksum() +{ + uint64_t checksum = 0; + const int64_t op_count = table_operations_.count(); + if (op_count > 0) { + if (is_same_type()) { + const uint64_t first_op_checksum = table_operations_.at(0).get_checksum(); + checksum = ob_crc64(checksum, &first_op_checksum, sizeof(first_op_checksum)); + } else { + for (int64_t i = 0; i < op_count; ++i) { + const uint64_t cur_op_checksum = table_operations_.at(i).get_checksum(); + checksum = ob_crc64(checksum, &cur_op_checksum, sizeof(cur_op_checksum)); + } + } + } + checksum = ob_crc64(checksum, &is_readonly_, sizeof(is_readonly_)); + checksum = ob_crc64(checksum, &is_same_type_, sizeof(is_same_type_)); + checksum = ob_crc64(checksum, &is_same_properties_names_, sizeof(is_same_properties_names_)); + + return checksum; +} + +void ObTableBatchOperation::reset() +{ + table_operations_.reset(); + is_readonly_ = true; + is_same_type_ = true; + is_same_properties_names_ = true; +} +/* +OB_SERIALIZE_MEMBER(ObTableBatchOperation, + table_operations_, + is_readonly_, + is_same_type_, + is_same_properties_names_); +*/ +OB_UNIS_DEF_SERIALIZE(ObTableBatchOperation, + table_operations_, + is_readonly_, + is_same_type_, + is_same_properties_names_); + +OB_UNIS_DEF_SERIALIZE_SIZE(ObTableBatchOperation, + table_operations_, + is_readonly_, + is_same_type_, + is_same_properties_names_); + +OB_DEF_DESERIALIZE(ObTableBatchOperation,) +{ + int ret = OB_SUCCESS; + UNF_UNUSED_DES; + reset(); + int64_t batch_size = 0; + OB_UNIS_DECODE(batch_size); + ObITableEntity *entity = NULL; + ObTableOperation table_operation; + for (int64_t i = 0; OB_SUCCESS == ret && i < batch_size; ++i) + { + if (NULL == (entity = entity_factory_->alloc())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + } else { + table_operation.set_entity(*entity); + OB_UNIS_DECODE(table_operation); + if (OB_SUCC(ret)) { + if (OB_FAIL(table_operations_.push_back(table_operation))) { + LOG_WARN("failed to push back", K(ret)); + } + } + } + } // end for + LST_DO_CODE(OB_UNIS_DECODE, + is_readonly_, + is_same_type_, + is_same_properties_names_); + return ret; +} + +//////////////////////////////////////////////////////////////// +OB_SERIALIZE_MEMBER(ObTableResult, errno_, sqlstate_, msg_); + +int ObTableResult::assign(const ObTableResult &other) +{ + errno_ = other.errno_; + strncpy(sqlstate_, other.sqlstate_, sizeof(sqlstate_)); + strncpy(msg_, other.msg_, sizeof(msg_)); + return OB_SUCCESS; +} + +//////////////////////////////////////////////////////////////// +ObTableOperationResult::ObTableOperationResult() + :operation_type_(ObTableOperationType::GET), + entity_(NULL), + affected_rows_(0) +{} + +int ObTableOperationResult::get_entity(const ObITableEntity *&entity) const +{ + int ret = OB_SUCCESS; + if (NULL == entity_) { + ret = OB_ERR_NULL_VALUE; + } else { + entity = entity_; + } + return ret; +} + +int ObTableOperationResult::get_entity(ObITableEntity *&entity) +{ + int ret = OB_SUCCESS; + if (NULL == entity_) { + ret = OB_ERR_NULL_VALUE; + } else { + entity = entity_; + } + return ret; +} + +DEF_TO_STRING(ObTableOperationResult) +{ + int64_t pos = 0; + J_OBJ_START(); + J_KV(K_(errno), + K_(operation_type), + K_(affected_rows)); + J_COMMA(); + if (NULL == entity_) { + J_NAME("entity"); + J_COLON(); + J_NULL(); + } else { + J_KV("entity", *entity_); + } + J_OBJ_END(); + return pos; +} + +OB_SERIALIZE_MEMBER((ObTableOperationResult, ObTableResult), + operation_type_, *entity_, affected_rows_); + +int ObTableOperationResult::deep_copy(common::ObIAllocator &allocator, + ObITableEntityFactory &entity_factory, const ObTableOperationResult &other) +{ + int ret = OB_SUCCESS; + const ObITableEntity *src_entity = NULL; + ObITableEntity *dest_entity = NULL; + if (OB_FAIL(other.get_entity(src_entity))) { + LOG_WARN("failed to get entity", K(ret)); + } else if (NULL == (dest_entity = entity_factory.alloc())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("no memory", K(ret)); + } else if (OB_FAIL(dest_entity->deep_copy(allocator, *src_entity))) { + LOG_WARN("failed to copy entity", K(ret)); + } else if (OB_FAIL(ObTableResult::assign(other))) { + LOG_WARN("failed to copy result", K(ret)); + } else { + operation_type_ = other.operation_type_; + entity_ = dest_entity; + affected_rows_ = other.affected_rows_; + } + return ret; +} + +//////////////////////////////////////////////////////////////// +OB_DEF_DESERIALIZE(ObTableBatchOperationResult,) +{ + int ret = OB_SUCCESS; + UNF_UNUSED_DES; + int64_t batch_size = 0; + OB_UNIS_DECODE(batch_size); + ObITableEntity *entity = NULL; + ObTableOperationResult table_operation_result; + reset(); + if (NULL == alloc_) { + // shallow copy properties + for (int64_t i = 0; OB_SUCCESS == ret && i < batch_size; ++i) + { + if (NULL == (entity = entity_factory_->alloc())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + } else { + table_operation_result.set_entity(*entity); + if (OB_FAIL(serialization::decode(buf, data_len, pos, table_operation_result))) { + LOG_WARN("fail to decode array item", K(ret), K(i), K(batch_size), K(data_len), + K(pos), K(table_operation_result)); + } else if (OB_FAIL(push_back(table_operation_result))) { + LOG_WARN("fail to add item to array", K(ret), K(i), K(batch_size)); + } + } + } // end for + } else { + // deep copy properties + for (int64_t i = 0; OB_SUCCESS == ret && i < batch_size; ++i) + { + if (NULL == (entity = entity_factory_->alloc())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + } else { + entity->set_allocator(alloc_); + table_operation_result.set_entity(*entity); + if (OB_FAIL(serialization::decode(buf, data_len, pos, table_operation_result))) { + LOG_WARN("fail to decode array item", K(ret), K(i), K(batch_size), K(data_len), + K(pos), K(table_operation_result)); + } else if (OB_FAIL(push_back(table_operation_result))) { + LOG_WARN("fail to add item to array", K(ret), K(i), K(batch_size)); + } + } + } // end for + } + return ret; +} + +OB_UNIS_DEF_SERIALIZE((ObTableBatchOperationResult, ObTableBatchOperationResult::BaseType), ); +OB_UNIS_DEF_SERIALIZE_SIZE((ObTableBatchOperationResult, ObTableBatchOperationResult::BaseType), ); +//////////////////////////////////////////////////////////////// +void ObTableQuery::reset() +{ + deserialize_allocator_ = NULL; + key_ranges_.reset(); + select_columns_.reset(); + filter_string_.reset(); + limit_ = -1; // no limit + offset_ = 0; + scan_order_ = ObQueryFlag::Forward; + index_name_.reset(); + batch_size_ = -1; + max_result_size_ = -1; +} + +bool ObTableQuery::is_valid() const +{ + return (limit_ == -1 || limit_ > 0) + && (offset_ >= 0) + && key_ranges_.count() > 0 + && select_columns_.count() > 0; +} + +int ObTableQuery::add_scan_range(common::ObNewRange &scan_range) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(key_ranges_.push_back(scan_range))) { + LOG_WARN("failed to add rowkey range", K(ret), K(scan_range)); + } + return ret; +} + +int ObTableQuery::set_scan_order(common::ObQueryFlag::ScanOrder scan_order) +{ + int ret = OB_SUCCESS; + if (scan_order != ObQueryFlag::Forward + && scan_order != ObQueryFlag::Reverse) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid scan order", K(ret), K(scan_order)); + } else { + scan_order_ = scan_order; + } + return ret; +} + +int ObTableQuery::add_select_column(const ObString &column) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(select_columns_.push_back(column))) { + LOG_WARN("failed to add select column", K(ret), K(column)); + } + return ret; +} + +int ObTableQuery::set_scan_index(const ObString &index_name) +{ + index_name_ = index_name; + return OB_SUCCESS; +} + +int ObTableQuery::set_filter(const ObString &filter) +{ + LOG_WARN("general filter not supported", K(filter)); + return OB_NOT_SUPPORTED; +} + +int ObTableQuery::set_limit(int32_t limit) +{ + int ret = OB_SUCCESS; + if (limit < -1 || 0 == limit) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("limit cannot be negative or zero", K(ret), K(limit)); + } else { + limit_ = limit; + } + return ret; +} + +int ObTableQuery::set_offset(int32_t offset) +{ + int ret = OB_SUCCESS; + if (offset < 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("offset cannot be negative", K(ret), K(offset)); + } else { + offset_ = offset; + } + return ret; +} + +int ObTableQuery::set_batch(int32_t batch_size) +{ + int ret = OB_SUCCESS; + if (batch_size == 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("batch_size cannot be zero", K(ret), K(batch_size)); + } else { + batch_size_ = batch_size; + } + return ret; +} + +int ObTableQuery::set_max_result_size(int64_t max_result_size) +{ + int ret = OB_SUCCESS; + if (max_result_size == 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("max_result_size cannot be zero", K(ret), K(max_result_size)); + } else { + max_result_size_ = max_result_size; + } + return ret; +} + +uint64_t ObTableQuery::get_checksum() const +{ + uint64_t checksum = 0; + const int64_t range_count = get_range_count(); + checksum = ob_crc64(checksum, &range_count, sizeof(range_count)); + for (int64_t i = 0; i < select_columns_.count(); ++i) { + const ObString &cur_column = select_columns_.at(i); + checksum = ob_crc64(checksum, cur_column.ptr(), cur_column.length()); + } + checksum = ob_crc64(checksum, filter_string_.ptr(), filter_string_.length()); + checksum = ob_crc64(checksum, &limit_, sizeof(limit_)); + checksum = ob_crc64(checksum, &offset_, sizeof(offset_)); + checksum = ob_crc64(checksum, &scan_order_, sizeof(scan_order_)); + checksum = ob_crc64(checksum, index_name_.ptr(), index_name_.length()); + checksum = ob_crc64(checksum, &batch_size_, sizeof(batch_size_)); + checksum = ob_crc64(checksum, &max_result_size_, sizeof(max_result_size_)); + return checksum; +} + +OB_UNIS_DEF_SERIALIZE(ObTableQuery, + key_ranges_, + select_columns_, + filter_string_, + limit_, + offset_, + scan_order_, + index_name_, + batch_size_, + max_result_size_); + +OB_UNIS_DEF_SERIALIZE_SIZE(ObTableQuery, + key_ranges_, + select_columns_, + filter_string_, + limit_, + offset_, + scan_order_, + index_name_, + batch_size_, + max_result_size_); + +OB_DEF_DESERIALIZE(ObTableQuery,) +{ + int ret = OB_SUCCESS; + UNF_UNUSED_DES; + if (OB_SUCC(ret)) { + int64_t count = 0; + key_ranges_.reset(); + if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &count))) { + LOG_WARN("fail to decode key ranges count", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < count; i ++) { + if (OB_ISNULL(deserialize_allocator_)) { + ret = OB_NOT_INIT; + LOG_WARN("deserialize allocator is NULL", K(ret)); + } else { + ObObj array[OB_MAX_ROWKEY_COLUMN_NUMBER * 2]; + ObNewRange copy_range; + ObNewRange key_range; + copy_range.start_key_.assign(array, OB_MAX_ROWKEY_COLUMN_NUMBER); + copy_range.end_key_.assign(array + OB_MAX_ROWKEY_COLUMN_NUMBER, OB_MAX_ROWKEY_COLUMN_NUMBER); + if (OB_FAIL(copy_range.deserialize(buf, data_len, pos))) { + LOG_WARN("fail to deserialize range", K(ret)); + } else if (OB_FAIL(common::deep_copy_range(*deserialize_allocator_, copy_range, key_range))) { + LOG_WARN("fail to deep copy range", K(ret)); + } else if (OB_FAIL(key_ranges_.push_back(key_range))) { + LOG_WARN("fail to add key range to array", K(ret)); + } + } + } + } + if (OB_SUCC(ret)) { + LST_DO_CODE(OB_UNIS_DECODE, + select_columns_, + filter_string_, + limit_, + offset_, + scan_order_, + index_name_, + batch_size_, + max_result_size_ + ); + } + return ret; +} + +//////////////////////////////////////////////////////////////// +ObTableEntityIterator::~ObTableEntityIterator() +{} + +//////////////////////////////////////////////////////////////// +ObTableQueryResult::ObTableQueryResult() + :row_count_(0), + allocator_(ObModIds::TABLE_PROC), + fixed_result_size_(0), + curr_idx_(0) +{ +} + +void ObTableQueryResult::reset_except_property() +{ + row_count_ = 0; + buf_.reset(); + allocator_.reset(); + fixed_result_size_ = 0; + curr_idx_ = 0; + curr_entity_.reset(); +} + +void ObTableQueryResult::reset() +{ + properties_names_.reset(); + reset_except_property(); +} + +void ObTableQueryResult::rewind() +{ + curr_idx_ = 0; + buf_.get_position() = 0; +} + +int ObTableQueryResult::get_next_entity(const ObITableEntity *&entity) +{ + int ret = OB_SUCCESS; + if (0 >= properties_names_.count()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid properties_names", K(ret)); + } else if (curr_idx_ >= row_count_) { + ret = OB_ITER_END; + } else { + curr_entity_.reset(); + ObObj value; + const int64_t N = properties_names_.count(); + for (int i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(value.deserialize(buf_.get_data(), buf_.get_capacity(), buf_.get_position()))) { + LOG_WARN("failed to deserialize obj", K(ret), K_(buf)); + } else if (OB_FAIL(curr_entity_.set_property(properties_names_.at(i), value))) { + LOG_WARN("failed to set entity property", K(ret), K(i), K(value)); + } + } // end for + if (OB_SUCC(ret)) { + entity = &curr_entity_; + ++curr_idx_; + } + } + return ret; +} + +int ObTableQueryResult::get_first_row(common::ObNewRow &row) const +{ + int ret = OB_SUCCESS; + if (row.is_invalid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid row object", K(ret)); + } else if (0 >= row_count_) { + ret = OB_ITER_END; + } else { + const char *databuf = buf_.get_data(); + const int64_t datalen = buf_.get_position(); + int64_t pos = 0; + const int64_t N = row.count_; + for (int i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(row.cells_[i].deserialize(databuf, datalen, pos))) { + LOG_WARN("failed to deserialize obj", K(ret), K(datalen), K(pos)); + } + } // end for + } + return ret; +} + +int ObTableQueryResult::add_property_name(const ObString &name) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(properties_names_.push_back(name))) { + LOG_WARN("failed to add name", K(ret), K(name)); + } + return ret; +} + +int ObTableQueryResult::alloc_buf_if_need(const int64_t need_size) +{ + int ret = OB_SUCCESS; + if (need_size <= 0 || need_size > MAX_BUF_BLOCK_SIZE) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(need_size), LITERAL_K(MAX_BUF_BLOCK_SIZE)); + } else if (NULL == buf_.get_data()) { // first alloc + int64_t actual_size = 0; + if (need_size <= DEFAULT_BUF_BLOCK_SIZE) { + actual_size = DEFAULT_BUF_BLOCK_SIZE; + } else { + actual_size = need_size; + } + char *tmp_buf = static_cast(allocator_.alloc(actual_size)); + if (NULL == tmp_buf) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("no memory", K(ret), K(actual_size)); + } else { + buf_.set_data(tmp_buf, actual_size); + } + } else if (buf_.get_remain() < need_size) { + if (need_size + buf_.get_position() > MAX_BUF_BLOCK_SIZE) { // check max buf size when expand buf + ret = OB_BUF_NOT_ENOUGH; + LOG_WARN("will exceed max buf need_size", K(ret), K(need_size), K(buf_.get_position()), LITERAL_K(MAX_BUF_BLOCK_SIZE)); + } else { + int64_t actual_size = MAX(need_size + buf_.get_position(), 2 * buf_.get_capacity()); + actual_size = MIN(actual_size, MAX_BUF_BLOCK_SIZE); + char *tmp_buf = static_cast(allocator_.alloc(actual_size)); + if (NULL == tmp_buf) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("no memory", K(ret), K(actual_size)); + } else { + const int64_t old_buf_size = buf_.get_position(); + MEMCPY(tmp_buf, buf_.get_data(), old_buf_size); + buf_.set_data(tmp_buf, actual_size); + buf_.get_position() = old_buf_size; + } + } + + } + return ret; +} + +int ObTableQueryResult::add_row(const ObNewRow &row) +{ + int ret = OB_SUCCESS; + ret = alloc_buf_if_need(row.get_serialize_size()); + const int64_t N = row.get_count(); + if (OB_SUCC(ret)) { + if (0 != properties_names_.count() + && N != properties_names_.count()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("cell count not match with property count", K(ret), K(N), + "properties_count", properties_names_.count()); + } + } + for (int i = 0; OB_SUCCESS == ret && i < N; ++i) + { + if (OB_FAIL(row.get_cell(i).serialize(buf_.get_data(), buf_.get_capacity(), buf_.get_position()))) { + LOG_WARN("failed to serialize obj", K(ret), K_(buf)); + } + } // end for + if (OB_SUCC(ret)) { + ++row_count_; + } + return ret; +} + +int ObTableQueryResult::add_all_property(const ObTableQueryResult &other) +{ + int ret = OB_SUCCESS; + if (0 != properties_names_.count()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid properties which has been initialized", K(ret)); + } else if (OB_FAIL(append(properties_names_, other.properties_names_))) { + LOG_WARN("failed to append property", K(ret)); + } + return ret; +} + +int ObTableQueryResult::add_all_row(const ObTableQueryResult &other) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(alloc_buf_if_need(other.buf_.get_position()))) { + } else if (buf_.get_remain() < other.buf_.get_position()) { + ret = OB_BUF_NOT_ENOUGH; + } else { + MEMCPY(buf_.get_cur_pos(), other.buf_.get_data(), other.buf_.get_position()); + buf_.get_position() += other.buf_.get_position(); + row_count_ += other.row_count_; + } + return ret; +} + +int64_t ObTableQueryResult::get_result_size() +{ + if (0 >= fixed_result_size_) { + fixed_result_size_ = properties_names_.get_serialize_size(); + fixed_result_size_ += obrpc::ObRpcPacketHeader::HEADER_SIZE + + 8/*appr. row_count*/ + + 8/*appr. buf_position*/; + } + return fixed_result_size_ + buf_.get_position(); +} + +bool ObTableQueryResult::reach_batch_size_or_result_size(const int32_t batch_count, + const int64_t max_result_size) +{ + bool reach_size = false; + if (batch_count > 0 && this->get_row_count() >= batch_count) { + LOG_DEBUG("[yzfdebug] reach batch limit", K(batch_count)); + reach_size = true; + } else if (max_result_size > 0 && this->get_result_size() >= max_result_size) { + LOG_DEBUG("[yzfdebug] reach size limit", K(max_result_size)); + reach_size = true; + } + return reach_size; +} + +OB_DEF_SERIALIZE(ObTableQueryResult) +{ + int ret = OB_SUCCESS; + LST_DO_CODE(OB_UNIS_ENCODE, + properties_names_, + row_count_, + buf_.get_position()); + if (OB_SUCC(ret)) { + if (buf_len - pos < buf_.get_position()) { + LOG_WARN("failed to serialize ObTableQueryResult", K(ret), K(buf_len), K(pos), "datalen", buf_.get_position()); + } else { + MEMCPY(buf+pos, buf_.get_data(), buf_.get_position()); + pos += buf_.get_position(); + } + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObTableQueryResult) +{ + int64_t len = 0; + LST_DO_CODE(OB_UNIS_ADD_LEN, + properties_names_, + row_count_, + buf_.get_position()); + len += buf_.get_position(); + return len; +} + +OB_DEF_DESERIALIZE(ObTableQueryResult) +{ + int ret = OB_SUCCESS; + allocator_.reset(); // deep copy all + properties_names_.reset(); + curr_idx_ = 0; + int64_t databuff_len = 0; + int64_t properties_count = 0; + OB_UNIS_DECODE(properties_count); + if (OB_SUCC(ret)) { + ObString property_name; + ObString name_clone; + for (int64_t i = 0; OB_SUCCESS == ret && i < properties_count; ++i) + { + OB_UNIS_DECODE(property_name); + if (OB_SUCC(ret)) { + if (OB_FAIL(ob_write_string(allocator_, property_name, name_clone))) { + LOG_WARN("failed to deep copy string", K(ret), K(property_name)); + } else if (OB_FAIL(properties_names_.push_back(name_clone))) { + LOG_WARN("failed to push back", K(ret)); + } + } + } // end for + } + LST_DO_CODE(OB_UNIS_DECODE, + row_count_, + databuff_len); + char *buff1 = NULL; + if (OB_FAIL(ret)) { + } else if (databuff_len > data_len - pos) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("invalid data", K(ret), K(databuff_len), K(pos), K(data_len)); + } else if (databuff_len == 0) { + buf_.set_data(NULL, 0); + } else if (NULL == (buff1 = static_cast(allocator_.alloc(databuff_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("no memory", K(ret), K(databuff_len)); + } else { + // deep copy + MEMCPY(buff1, buf+pos, databuff_len); + buf_.set_data(buff1, databuff_len); + pos += databuff_len; + } + return ret; +} diff --git a/src/share/table/ob_table.h b/src/share/table/ob_table.h new file mode 100644 index 0000000000..a98a144a7b --- /dev/null +++ b/src/share/table/ob_table.h @@ -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 +{ + 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 > &properties) const = 0; // @todo property iterator + virtual int get_properties_names(ObIArray &properties) const = 0; + virtual int get_properties_values(ObIArray &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 > &properties) const override; + virtual int get_properties_names(ObIArray &properties_names) const override; + virtual int get_properties_values(ObIArray &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 PropertiesMap; +private: + ObSEArray rowkey_; + PropertiesMap properties_; +}; + +enum class ObTableEntityType +{ + ET_DYNAMIC = 0, + ET_KV = 1 +}; + +// @note not thread-safe +template +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 used_list_; + common::ObDList free_list_; +}; + +template +ObTableEntityFactory::~ObTableEntityFactory() +{ + free_all(); +} + +template +ObITableEntity *ObTableEntityFactory::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 +void ObTableEntityFactory::free(ObITableEntity *entity) +{ + if (NULL != entity) { + entity->reset(); + entity->set_allocator(NULL); + used_list_.remove(entity); + free_list_.add_last(entity); + } +} + +template +void ObTableEntityFactory::free_and_reuse() +{ + while (!used_list_.is_empty()) { + this->free(used_list_.get_first()); + } +} + +template +void ObTableEntityFactory::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 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 ObITableBatchOperationResult; +class ObTableBatchOperationResult: public common::ObSEArrayImpl +{ + 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 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 &get_select_columns() const { return select_columns_; } + const ObIArray &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 key_ranges_; + ObSEArray 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 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 */ diff --git a/src/share/table/ob_table_rpc_proxy.h b/src/share/table/ob_table_rpc_proxy.h new file mode 100644 index 0000000000..3d67f8ef05 --- /dev/null +++ b/src/share/table/ob_table_rpc_proxy.h @@ -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 */ diff --git a/src/share/table/ob_table_rpc_struct.cpp b/src/share/table/ob_table_rpc_struct.cpp new file mode 100644 index 0000000000..e9b2530e78 --- /dev/null +++ b/src/share/table/ob_table_rpc_struct.cpp @@ -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_ + ); diff --git a/src/share/table/ob_table_rpc_struct.h b/src/share/table/ob_table_rpc_struct.h new file mode 100644 index 0000000000..2257002d48 --- /dev/null +++ b/src/share/table/ob_table_rpc_struct.h @@ -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 */ diff --git a/unittest/observer/CMakeLists.txt b/unittest/observer/CMakeLists.txt index 1572534617..c4c20d6de7 100644 --- a/unittest/observer/CMakeLists.txt +++ b/unittest/observer/CMakeLists.txt @@ -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_token_calcer omt/test_token_calcer.cpp) ob_unittest(test_information_schema) +ob_unittest(test_tableapi tableapi/test_tableapi.cpp) diff --git a/unittest/observer/tableapi/test_tableapi.cpp b/unittest/observer/tableapi/test_tableapi.cpp new file mode 100644 index 0000000000..c41cbe5f57 --- /dev/null +++ b/unittest/observer/tableapi/test_tableapi.cpp @@ -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 +#include +#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(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 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 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 &select_columns = query2.get_select_columns(); + const ObIArray &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(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 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(); +}