diff --git a/deps/oblib/src/lib/ob_define.h b/deps/oblib/src/lib/ob_define.h index 95eb8b8b1..24ad68f4b 100644 --- a/deps/oblib/src/lib/ob_define.h +++ b/deps/oblib/src/lib/ob_define.h @@ -826,6 +826,7 @@ const uint64_t OB_SYS_UNIT_ID = 1; const uint64_t OB_INIT_SERVER_ID = 1; const uint64_t OB_INIT_DDL_TASK_ID = 1; const uint64_t OB_SYS_UNIT_GROUP_ID = 1; +const uint64_t OB_INIT_REWRITE_RULE_VERSION = 1; const uint64_t OB_USER_UNIT_CONFIG_ID = 1000; const uint64_t OB_USER_RESOURCE_POOL_ID = 1000; const uint64_t OB_USER_UNIT_ID = 1000; diff --git a/deps/oblib/src/lib/oblog/ob_log_module.h b/deps/oblib/src/lib/oblog/ob_log_module.h index b67f1aa1c..01153bedd 100644 --- a/deps/oblib/src/lib/oblog/ob_log_module.h +++ b/deps/oblib/src/lib/oblog/ob_log_module.h @@ -180,6 +180,7 @@ DEFINE_LOG_SUB_MOD(CG) // code_generator DEFINE_LOG_SUB_MOD(MONITOR) // monitor DEFINE_LOG_SUB_MOD(DTL) // data transfer layer DEFINE_LOG_SUB_MOD(DAS) // data access service +DEFINE_LOG_SUB_MOD(QRR) // query rewrite rule LOG_MOD_END(SQL) // observer submodules @@ -740,6 +741,10 @@ LOG_MOD_END(PL) info_string, ##args) #define _SQL_DAS_LOG(level, _fmt_, args...) _OB_SUB_MOD_LOG(SQL, DAS, level, \ _fmt_, ##args) +#define SQL_QRR_LOG(level, info_string, args...) OB_SUB_MOD_LOG(SQL, QRR, level, \ + info_string, ##args) +#define _SQL_QRR_LOG(level, _fmt_, args...) _OB_SUB_MOD_LOG(SQL, QRR, level, \ + _fmt_, ##args) #define DETECT_LOG_LOG(level, info_string, args...) OB_SUB_MOD_LOG(DETECT, LOG,level, \ info_string, ##args) diff --git a/deps/oblib/src/lib/stat/ob_latch_define.h b/deps/oblib/src/lib/stat/ob_latch_define.h index 4eba924a3..abe878323 100644 --- a/deps/oblib/src/lib/stat/ob_latch_define.h +++ b/deps/oblib/src/lib/stat/ob_latch_define.h @@ -299,6 +299,7 @@ LATCH_DEF(DEADLOCK_DETECT_LOCK, 288, "deadlock detect lock", LATCH_FIFO, 2000, 0 LATCH_DEF(FREEZE_THREAD_POOL_LOCK, 289, "freeze thread pool lock", LATCH_FIFO, 2000, 0, FREEZE_THREAD_POOL_WAIT, "freeze thread pool lock") LATCH_DEF(DAS_ASYNC_RPC_LOCK, 290, "das wait remote response lock", LATCH_FIFO, 2000, 0, DAS_ASYNC_RPC_LOCK_WAIT, "das wait remote response lock") LATCH_DEF(CLOG_CKPT_RWLOCK, 291, "clog checkpoint rwlock", LATCH_READ_PREFER, 2000, 0, CLOG_CKPT_RWLOCK_WAIT, "clog checkpoint rwlock") +LATCH_DEF(REWRITE_RULE_ITEM_LOCK, 292, "rewrite rule item lock", LATCH_FIFO, 2000, 0, REWRITE_RULE_ITEM_LOCK_WAIT, "rewrite rule item lock") // LATCH_DEF(TENANT_MGR_TENANT_BUCKET_LOCK, 290, "tenant mgr tenant bucket lock", LATCH_READ_PREFER, INT64_MAX, 0, TENANT_MGR_TENANT_BUCKET_LOCK_WAIT, "tenant mgr tenant bucket lock") LATCH_DEF(SRS_LOCK, 292, "srs lock", LATCH_READ_PREFER, 2000, 0, SRS_LOCK_WAIT, "srs lock") diff --git a/deps/oblib/src/lib/wait_event/ob_wait_event.h b/deps/oblib/src/lib/wait_event/ob_wait_event.h index 37476e8ca..614193532 100644 --- a/deps/oblib/src/lib/wait_event/ob_wait_event.h +++ b/deps/oblib/src/lib/wait_event/ob_wait_event.h @@ -55,6 +55,7 @@ WAIT_EVENT_DEF(ELECTION_LOCK_WAIT, 15008, "spinlock: election lock wait", "addre WAIT_EVENT_DEF(TRANS_CTX_LOCK_WAIT, 15009, "latch: trans ctx lock wait", "address", "number", "tries", CONCURRENCY, "latch: trans ctx lock wait", true) WAIT_EVENT_DEF(PARTITION_LOG_LOCK_WAIT, 15010, "latch: partition log lock wait", "address", "number", "tries", CONCURRENCY, "latch: partition log lock wait", true) WAIT_EVENT_DEF(PCV_SET_LOCK_WAIT, 15011, "latch: plan cache pcv set lock wait", "address", "number", "tries", CONCURRENCY, "latch: plan cache pcv set lock wait", true) +WAIT_EVENT_DEF(REWRITE_RULE_ITEM_LOCK_WAIT, 15093, "latch: rewrite rule item lock wait", "address", "number", "tries", CONCURRENCY, "latch: rewrite rule item lock wait", true) WAIT_EVENT_DEF(CLOG_HISTORY_REPORTER_LOCK_WAIT, 15012, "latch: clog history reporter lock wait", "address", "number", "tries", CONCURRENCY, "latch: clog history reporter lock wait", true) WAIT_EVENT_DEF(CLOG_EXTERNAL_EXEC_LOCK_WAIT, 15013, "latch: clog external executor lock wait", "address", "number", "tries", CONCURRENCY, "latch: clog external executor lock wait", true) diff --git a/src/observer/mysql/ob_mysql_request_manager.cpp b/src/observer/mysql/ob_mysql_request_manager.cpp index 2fe23a792..86a0d382c 100644 --- a/src/observer/mysql/ob_mysql_request_manager.cpp +++ b/src/observer/mysql/ob_mysql_request_manager.cpp @@ -138,7 +138,8 @@ int ObMySQLRequestManager::record_request(const ObAuditRecordData &audit_record, + audit_record.tenant_name_len_ + audit_record.user_name_len_ + audit_record.db_name_len_ - + audit_record.params_value_len_; + + audit_record.params_value_len_ + + audit_record.rule_name_len_; if (NULL == (buf = (char*)alloc(total_size))) { ret = OB_ALLOCATE_MEMORY_FAILED; if (REACH_TIME_INTERVAL(100 * 1000)) { @@ -161,6 +162,12 @@ int ObMySQLRequestManager::record_request(const ObAuditRecordData &audit_record, record->data_.params_value_ = buf + pos; pos += audit_record.params_value_len_; } + //deep copy rule name + if ((audit_record.rule_name_len_ > 0) && (NULL != audit_record.rule_name_)) { + MEMCPY(buf + pos, audit_record.rule_name_, audit_record.rule_name_len_); + record->data_.rule_name_ = buf + pos; + pos += audit_record.rule_name_len_; + } //deep copy tenant_name if ((audit_record.tenant_name_len_ > 0) && (NULL != audit_record.tenant_name_)) { int64_t tenant_len = min(audit_record.tenant_name_len_, OB_MAX_TENANT_NAME_LENGTH); diff --git a/src/observer/mysql/obmp_query.cpp b/src/observer/mysql/obmp_query.cpp index 14a4e5f46..837df4e59 100644 --- a/src/observer/mysql/obmp_query.cpp +++ b/src/observer/mysql/obmp_query.cpp @@ -918,6 +918,8 @@ OB_INLINE int ObMPQuery::do_process(ObSQLSessionInfo &session, audit_record.table_scan_ = plan->contain_table_scan(); audit_record.plan_id_ = plan->get_plan_id(); audit_record.plan_hash_ = plan->get_plan_hash_value(); + audit_record.rule_name_ = const_cast(plan->get_rule_name().ptr()); + audit_record.rule_name_len_ = plan->get_rule_name().length(); } audit_record.affected_rows_ = result.get_affected_rows(); audit_record.return_rows_ = result.get_return_rows(); diff --git a/src/observer/mysql/obmp_stmt_execute.cpp b/src/observer/mysql/obmp_stmt_execute.cpp index d7246df23..c33e3821a 100644 --- a/src/observer/mysql/obmp_stmt_execute.cpp +++ b/src/observer/mysql/obmp_stmt_execute.cpp @@ -964,6 +964,7 @@ int ObMPStmtExecute::execute_response(ObSQLSessionInfo &session, { int ret = OB_SUCCESS; ObPsStmtId inner_stmt_id = OB_INVALID_ID; + ObIAllocator &alloc = CURRENT_CONTEXT->get_arena_allocator(); if (OB_ISNULL(session.get_ps_cache())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ps : ps cache is null.", K(ret), K(stmt_id_)); @@ -1237,6 +1238,8 @@ int ObMPStmtExecute::do_process(ObSQLSessionInfo &session, audit_record.table_scan_ = plan->contain_table_scan(); audit_record.plan_id_ = plan->get_plan_id(); audit_record.plan_hash_ = plan->get_plan_hash_value(); + audit_record.rule_name_ = const_cast(plan->get_rule_name().ptr()); + audit_record.rule_name_len_ = plan->get_rule_name().length(); } audit_record.affected_rows_ = result.get_affected_rows(); audit_record.return_rows_ = result.get_return_rows(); diff --git a/src/observer/ob_rpc_processor_simple.cpp b/src/observer/ob_rpc_processor_simple.cpp index 75e23dfb2..68f70b137 100644 --- a/src/observer/ob_rpc_processor_simple.cpp +++ b/src/observer/ob_rpc_processor_simple.cpp @@ -55,6 +55,7 @@ #include "storage/slog_ckpt/ob_tenant_checkpoint_slog_handler.h" #include "observer/ob_req_time_service.h" #include "observer/ob_server_event_history_table_operator.h" +#include "sql/udr/ob_udr_mgr.h" namespace oceanbase { @@ -2136,5 +2137,29 @@ int ObEstimateTabletBlockCountP::process() return ret; } +int ObSyncRewriteRulesP::process() +{ + int ret = OB_SUCCESS; + uint64_t tenant_id = arg_.tenant_id_; + MAKE_TENANT_SWITCH_SCOPE_GUARD(guard); + sql::ObUDRMgr *rule_mgr = nullptr; + + if (tenant_id != MTL_ID()) { + ret = guard.switch_to(tenant_id); + } + if (OB_SUCC(ret)) { + LOG_INFO("start do sync rewrite rules from inner table", K(arg_)); + + rule_mgr = MTL(sql::ObUDRMgr*); + if (OB_ISNULL(rule_mgr)) { + ret = OB_ERR_UNEXPECTED; + COMMON_LOG(ERROR, "mtl ObUDRMgr should not be null", K(ret)); + } else if (OB_FAIL(rule_mgr->sync_rule_from_inner_table())) { + LOG_WARN("failed to sync rewrite rules from inner table", K(ret)); + } + } + return ret; +} + } // end of namespace observer } // end of namespace oceanbase diff --git a/src/observer/ob_rpc_processor_simple.h b/src/observer/ob_rpc_processor_simple.h index 03745f6fe..75759c91c 100644 --- a/src/observer/ob_rpc_processor_simple.h +++ b/src/observer/ob_rpc_processor_simple.h @@ -214,6 +214,7 @@ OB_DEFINE_PROCESSOR_S(Srv, OB_GET_LS_ACCESS_MODE, ObRpcGetLSAccessModeP); OB_DEFINE_PROCESSOR_S(Srv, OB_CHANGE_LS_ACCESS_MODE, ObRpcChangeLSAccessModeP); OB_DEFINE_PROCESSOR_S(Srv, OB_ESTIMATE_TABLET_BLOCK_COUNT, ObEstimateTabletBlockCountP); OB_DEFINE_PROCESSOR_S(Srv, OB_DDL_CHECK_TABLET_MERGE_STATUS, ObRpcDDLCheckTabletMergeStatusP); +OB_DEFINE_PROCESSOR_S(Srv, OB_SYNC_REWRITE_RULES, ObSyncRewriteRulesP); } // end of namespace observer } // end of namespace oceanbase diff --git a/src/observer/ob_srv_xlator_partition.cpp b/src/observer/ob_srv_xlator_partition.cpp index 97020b300..092fc1e1a 100644 --- a/src/observer/ob_srv_xlator_partition.cpp +++ b/src/observer/ob_srv_xlator_partition.cpp @@ -231,4 +231,5 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator *xlator) { //sql optimizer estimate tablet block count RPC_PROCESSOR(ObEstimateTabletBlockCountP, gctx_); + RPC_PROCESSOR(ObSyncRewriteRulesP, gctx_); } diff --git a/src/observer/ob_srv_xlator_rootserver.cpp b/src/observer/ob_srv_xlator_rootserver.cpp index 1797898f9..ae08d581b 100644 --- a/src/observer/ob_srv_xlator_rootserver.cpp +++ b/src/observer/ob_srv_xlator_rootserver.cpp @@ -253,5 +253,6 @@ void oceanbase::observer::init_srv_xlator_for_rootserver(ObSrvRpcXlator *xlator) RPC_PROCESSOR(rootserver::ObRpcDropDirectoryP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObRpcDisasterRecoveryTaskReplyP, *gctx_.root_service_); + RPC_PROCESSOR(rootserver::ObRpcAdminSyncRewriteRulesP, *gctx_.root_service_); } diff --git a/src/observer/omt/ob_multi_tenant.cpp b/src/observer/omt/ob_multi_tenant.cpp index bf7382ab1..40aee4e54 100644 --- a/src/observer/omt/ob_multi_tenant.cpp +++ b/src/observer/omt/ob_multi_tenant.cpp @@ -84,6 +84,7 @@ #include "logservice/leader_coordinator/ob_leader_coordinator.h" #include "storage/lob/ob_lob_manager.h" #include "share/deadlock/ob_deadlock_detector_mgr.h" +#include "sql/udr/ob_udr_mgr.h" #include "storage/blocksstable/ob_shared_macro_block_manager.h" #include "storage/tx_storage/ob_tablet_gc_service.h" #include "share/ob_occam_time_guard.h" @@ -316,6 +317,7 @@ int ObMultiTenant::init(ObAddr myaddr, MTL_BIND2(mtl_new_default, ObDASIDService::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObAccessService::mtl_init, nullptr, mtl_stop_default, nullptr, mtl_destroy_default); MTL_BIND2(mtl_new_default, ObCheckPointService::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default); + MTL_BIND2(mtl_new_default, ObUDRMgr::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default); MTL_BIND(ObPxPools::mtl_init, ObPxPools::mtl_destroy); MTL_BIND(ObTenantDfc::mtl_init, ObTenantDfc::mtl_destroy); diff --git a/src/observer/virtual_table/ob_gv_sql.cpp b/src/observer/virtual_table/ob_gv_sql.cpp index e324a4e92..eb6c9950e 100644 --- a/src/observer/virtual_table/ob_gv_sql.cpp +++ b/src/observer/virtual_table/ob_gv_sql.cpp @@ -853,6 +853,21 @@ int ObGVSql::fill_cells(const ObPlanCacheObject *cache_obj, const ObPlanCache &p cells[i].set_int(cache_obj->get_obj_status()); break; } + case share::ALL_VIRTUAL_PLAN_STAT_CDE::RULE_NAME: { + ObString rule_name; + if (!cache_obj->is_sql_crsr() || plan->get_rule_name()) { + cells[i].set_null(); + } else if (OB_FAIL(ob_write_string(*allocator_, + plan->get_rule_name(), + rule_name))) { + SERVER_LOG(ERROR, "copy rule_name failed", K(ret)); + } else { + cells[i].set_varchar(rule_name); + cells[i].set_collation_type(ObCharset::get_default_collation( + ObCharset::get_default_charset())); + } + break; + } default: { ret = OB_ERR_UNEXPECTED; SERVER_LOG(WARN, diff --git a/src/observer/virtual_table/ob_gv_sql_audit.cpp b/src/observer/virtual_table/ob_gv_sql_audit.cpp index ec95c4184..e82be0bfc 100644 --- a/src/observer/virtual_table/ob_gv_sql_audit.cpp +++ b/src/observer/virtual_table/ob_gv_sql_audit.cpp @@ -958,6 +958,15 @@ int ObGvSqlAudit::fill_cells(obmysql::ObMySQLRequestRecord &record) cells[cell_idx].set_collation_type(ObCharset::get_default_collation( ObCharset::get_default_charset())); } break; + case RULE_NAME: { + if ((record.data_.rule_name_len_ > 0) && (NULL != record.data_.rule_name_)) { + cells[cell_idx].set_varchar(record.data_.rule_name_, record.data_.rule_name_len_); + } else { + cells[cell_idx].set_varchar(""); + } + cells[cell_idx].set_collation_type(ObCharset::get_default_collation( + ObCharset::get_default_charset())); + } break; default: { ret = OB_ERR_UNEXPECTED; SERVER_LOG(WARN, "invalid column id", K(ret), K(cell_idx), K(col_id)); diff --git a/src/observer/virtual_table/ob_gv_sql_audit.h b/src/observer/virtual_table/ob_gv_sql_audit.h index 5c0ace39e..f0b0cb8b8 100644 --- a/src/observer/virtual_table/ob_gv_sql_audit.h +++ b/src/observer/virtual_table/ob_gv_sql_audit.h @@ -158,7 +158,8 @@ private: PLAN_HASH, USER_GROUP, LOCK_FOR_READ_TIME, - PARAMS_VALUE + PARAMS_VALUE, + RULE_NAME }; const static int64_t PRI_KEY_IP_IDX = 0; diff --git a/src/pl/ob_pl_interface_pragma.h b/src/pl/ob_pl_interface_pragma.h index dbcbb2047..0f06d9341 100644 --- a/src/pl/ob_pl_interface_pragma.h +++ b/src/pl/ob_pl_interface_pragma.h @@ -20,6 +20,8 @@ #include "pl/sys_package/ob_dbms_session.h" #include "pl/sys_package/ob_dbms_monitor.h" #include "pl/sys_package/ob_dbms_sql.h" +#include "pl/sys_package/ob_dbms_user_define_rule.h" + #ifdef INTERFACE_DEF INTERFACE_DEF(INTERFACE_START, "TEST", (void*)(ObPLInterfaceImpl::call)) diff --git a/src/pl/ob_pl_package_manager.cpp b/src/pl/ob_pl_package_manager.cpp index 64b5b35a9..c57ad5550 100644 --- a/src/pl/ob_pl_package_manager.cpp +++ b/src/pl/ob_pl_package_manager.cpp @@ -213,6 +213,7 @@ static ObSysPackageFile mysql_sys_package_file_table[] = { {"dbms_application", "dbms_application_mysql.sql", "dbms_application_body_mysql.sql"}, {"dbms_session", "dbms_session_mysql.sql", "dbms_session_body_mysql.sql"}, {"dbms_monitor", "dbms_monitor_mysql.sql", "dbms_monitor_body_mysql.sql"}, + {"dbms_udr", "dbms_udr_mysql.sql", "dbms_udr_body_mysql.sql"} }; int ObPLPackageManager::load_sys_package(ObMySQLProxy &sql_proxy, ObString &package_name, ObCompatibilityMode compa_mode) diff --git a/src/pl/sys_package/ob_dbms_user_define_rule.h b/src/pl/sys_package/ob_dbms_user_define_rule.h new file mode 100644 index 000000000..a02afe67a --- /dev/null +++ b/src/pl/sys_package/ob_dbms_user_define_rule.h @@ -0,0 +1,84 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + +#ifndef OCEANBASE_SRC_PL_SYS_PACKAGE_DBMS_QUERY_REWRITE_H_ +#define OCEANBASE_SRC_PL_SYS_PACKAGE_DBMS_QUERY_REWRITE_H_ +#include "sql/engine/ob_exec_context.h" +#include "share/object/ob_obj_cast.h" +#include "sql/udr/ob_udr_struct.h" + +namespace oceanbase +{ +namespace pl +{ + +#define DEF_UDR_PROCESSOR(name) \ + class name##Processor : public ObUDRProcessor \ + { \ + public: \ + name##Processor(sql::ObExecContext &ctx, sql::ParamStore ¶ms, common::ObObj &result) \ + : ObUDRProcessor(ctx, params, result) \ + {} \ + virtual ~name##Processor() {} \ + virtual int parse_request_param(); \ + virtual int generate_exec_arg(); \ + virtual int execute(); \ + private: \ + DISALLOW_COPY_AND_ASSIGN(name##Processor); \ + }; + +class ObUDRProcessor +{ +public: + ObUDRProcessor(sql::ObExecContext &ctx, sql::ParamStore ¶ms, common::ObObj &result) + : is_inited_(false), + ctx_(ctx), + params_(params), + result_(result), + arg_() + {} + virtual ~ObUDRProcessor() {} + int process(); + virtual int init(); + virtual int parse_request_param() = 0; + virtual int generate_exec_arg() = 0; + virtual int execute() = 0; + +protected: + int pre_execution_check(); + int sync_rule_from_inner_table(); + +protected: + bool is_inited_; + sql::ObExecContext &ctx_; + sql::ParamStore ¶ms_; + common::ObObj &result_; + ObUDRInfo arg_; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRProcessor); +}; + +DEF_UDR_PROCESSOR(ObCreateRule); +DEF_UDR_PROCESSOR(ObRemoveRule); +DEF_UDR_PROCESSOR(ObEnableRule); +DEF_UDR_PROCESSOR(ObDisableRule); + +class ObDBMSUserDefineRule +{ +public: + ObDBMSUserDefineRule() {} + virtual ~ObDBMSUserDefineRule() {} +#define DEF_UDR_FUNC(name) \ + static int name(sql::ObExecContext &ctx, sql::ParamStore ¶ms, common::ObObj &result); + DEF_UDR_FUNC(create_rule); + DEF_UDR_FUNC(remove_rule); + DEF_UDR_FUNC(enable_rule); + DEF_UDR_FUNC(disable_rule); +}; + +} // end of pl +} // end of oceanbase +#endif /* OCEANBASE_SRC_PL_SYS_PACKAGE_DBMS_QUERY_REWRITE_H_ */ \ No newline at end of file diff --git a/src/rootserver/ob_ddl_operator.cpp b/src/rootserver/ob_ddl_operator.cpp index eb8e4109f..fdecd72a4 100644 --- a/src/rootserver/ob_ddl_operator.cpp +++ b/src/rootserver/ob_ddl_operator.cpp @@ -112,7 +112,8 @@ ObSysStat::ObSysStat() ob_max_used_ls_id_(item_list_, MAX_ID_NAME_INFO(OB_MAX_USED_LS_ID_TYPE)), ob_max_used_ls_group_id_(item_list_, MAX_ID_NAME_INFO(OB_MAX_USED_LS_GROUP_ID_TYPE)), ob_max_used_sys_pl_object_id_(item_list_, MAX_ID_NAME_INFO(OB_MAX_USED_SYS_PL_OBJECT_ID_TYPE)), - ob_max_used_object_id_(item_list_, MAX_ID_NAME_INFO(OB_MAX_USED_OBJECT_ID_TYPE)) + ob_max_used_object_id_(item_list_, MAX_ID_NAME_INFO(OB_MAX_USED_OBJECT_ID_TYPE)), + ob_max_used_rewrite_rule_version_(item_list_, MAX_ID_NAME_INFO(OB_MAX_USED_REWRITE_RULE_VERSION_TYPE)) { } @@ -146,6 +147,7 @@ int ObSysStat::set_initial_values(const uint64_t tenant_id) ob_max_used_sys_pl_object_id_.value_.set_int(OB_MIN_SYS_PL_OBJECT_ID); // Use OB_INITIAL_TEST_DATABASE_ID to avoid confict when create tenant with initial user schema objects. ob_max_used_object_id_.value_.set_int(OB_INITIAL_TEST_DATABASE_ID); + ob_max_used_rewrite_rule_version_.value_.set_int(OB_INIT_REWRITE_RULE_VERSION); } return ret; } diff --git a/src/rootserver/ob_ddl_operator.h b/src/rootserver/ob_ddl_operator.h index 86b76d188..20b73e9f8 100644 --- a/src/rootserver/ob_ddl_operator.h +++ b/src/rootserver/ob_ddl_operator.h @@ -131,6 +131,7 @@ struct ObSysStat Item ob_max_used_ls_group_id_; Item ob_max_used_sys_pl_object_id_; Item ob_max_used_object_id_; + Item ob_max_used_rewrite_rule_version_; }; class ObDDLOperator diff --git a/src/rootserver/ob_root_service.cpp b/src/rootserver/ob_root_service.cpp index 0ecf29959..34cb74273 100644 --- a/src/rootserver/ob_root_service.cpp +++ b/src/rootserver/ob_root_service.cpp @@ -6250,6 +6250,27 @@ int ObRootService::drop_synonym(const obrpc::ObDropSynonymArg &arg) //-----End of functions for managing synonyms----- +int ObRootService::admin_sync_rewrite_rules(const obrpc::ObSyncRewriteRuleArg &arg) +{ + int ret = OB_SUCCESS; + if (!inited_) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else { + ObSystemAdminCtx ctx; + if (OB_FAIL(init_sys_admin_ctx(ctx))) { + LOG_WARN("init_sys_admin_ctx failed", K(ret)); + } else { + ObAdminSyncRewriteRules admin_util(ctx); + if (OB_FAIL(admin_util.execute(arg))) { + LOG_WARN("dispatch sync rewrite rules failed", K(arg), K(ret)); + } + ROOTSERVICE_EVENT_ADD("root_service", "admin_sync_rewrite_rules", K(ret), K(arg)); + } + } + return ret; +} + int ObRootService::create_package(const obrpc::ObCreatePackageArg &arg) { int ret = OB_SUCCESS; diff --git a/src/rootserver/ob_root_service.h b/src/rootserver/ob_root_service.h index 12dd9c1d7..8bd049685 100644 --- a/src/rootserver/ob_root_service.h +++ b/src/rootserver/ob_root_service.h @@ -587,6 +587,10 @@ public: //----End of functions for managing synonyms---- + //----Functions for sync rewrite rules---- + int admin_sync_rewrite_rules(const obrpc::ObSyncRewriteRuleArg &arg); + //----End of functions for sync rewrite rules---- + //----Functions for managing package---- int create_package(const obrpc::ObCreatePackageArg &arg); int alter_package(const obrpc::ObAlterPackageArg &arg); diff --git a/src/rootserver/ob_rs_rpc_processor.h b/src/rootserver/ob_rs_rpc_processor.h index 483b3c5ee..2d8d2dce0 100644 --- a/src/rootserver/ob_rs_rpc_processor.h +++ b/src/rootserver/ob_rs_rpc_processor.h @@ -487,6 +487,8 @@ DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_DROP_DIRECTORY, ObRpcDropDirectoryP, drop_ // context object DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_DO_CONTEXT_DDL, ObRpcDoContextDDLP, do_context_ddl(arg_)); +DEFINE_RS_RPC_PROCESSOR(obrpc::OB_ADMIN_SYNC_REWRITE_RULES, ObRpcAdminSyncRewriteRulesP, admin_sync_rewrite_rules(arg_)); + #undef DEFINE_RS_RPC_PROCESSOR_ #undef DEFINE_RS_RPC_PROCESSOR #undef DEFINE_LIMITED_RS_RPC_PROCESSOR diff --git a/src/rootserver/ob_system_admin_util.cpp b/src/rootserver/ob_system_admin_util.cpp index 19f785efd..d5b6b5824 100644 --- a/src/rootserver/ob_system_admin_util.cpp +++ b/src/rootserver/ob_system_admin_util.cpp @@ -1943,5 +1943,48 @@ int ObAdminSetTP::call_server(const ObAddr &server) return ret; } +int ObAdminSyncRewriteRules::execute(const obrpc::ObSyncRewriteRuleArg &arg) +{ + int ret = OB_SUCCESS; + ObSEArray server_list; + if (OB_FAIL(get_tenant_servers(arg.tenant_id_, server_list))) { + LOG_WARN("fail to get tenant servers", "tenant_id", arg.tenant_id_, KR(ret)); + } else { + //call tenant servers; + for (int64_t j = 0; OB_SUCC(ret) && j < server_list.count(); ++j) { + if (OB_FAIL(call_server(server_list.at(j), arg))) { + LOG_WARN("fail to call tenant server", + "tenant_id", arg.tenant_id_, + "server addr", server_list.at(j), + KR(ret)); + } + } + } + server_list.reset(); + return ret; +} + +int ObAdminSyncRewriteRules::call_server(const common::ObAddr &server, + const obrpc::ObSyncRewriteRuleArg &arg) +{ + int ret = OB_SUCCESS; + if (!ctx_.is_inited()) { + ret = OB_NOT_INIT; + LOG_WARN("not init", KR(ret)); + } else if (!server.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid server", K(server), KR(ret)); + } else if (OB_ISNULL(ctx_.rpc_proxy_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ctx_.rpc_proxy_->to(server) + .by(arg.tenant_id_) + .as(arg.tenant_id_) + .sync_rewrite_rules(arg))) { + LOG_WARN("request server sync rewrite rules failed", KR(ret), K(server)); + } + return ret; +} + } // end namespace rootserver } // end namespace oceanbase diff --git a/src/rootserver/ob_system_admin_util.h b/src/rootserver/ob_system_admin_util.h index 0fb6a67ca..d94ae13e8 100644 --- a/src/rootserver/ob_system_admin_util.h +++ b/src/rootserver/ob_system_admin_util.h @@ -545,6 +545,23 @@ private: DISALLOW_COPY_AND_ASSIGN(ObAdminSetTP); }; +class ObAdminSyncRewriteRules : public ObTenantServerAdminUtil +{ +public: + explicit ObAdminSyncRewriteRules(const ObSystemAdminCtx &ctx) + : ObTenantServerAdminUtil(ctx) + {} + virtual ~ObAdminSyncRewriteRules() {} + + int call_server(const common::ObAddr &server, + const obrpc::ObSyncRewriteRuleArg &arg); + + int execute(const obrpc::ObSyncRewriteRuleArg &arg); + +private: + DISALLOW_COPY_AND_ASSIGN(ObAdminSyncRewriteRules); +}; + } // end namespace rootserver } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp b/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp index 72c5a9aa2..b6e5c17e8 100644 --- a/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.11001_11050.cpp @@ -3050,6 +3050,21 @@ int ObInnerTableSchema::all_virtual_plan_stat_schema(ObTableSchema &table_schema false, //is_nullable false); //is_autoincrement } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("rule_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + 256, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } if (OB_SUCC(ret)) { table_schema.get_part_option().set_part_num(1); table_schema.set_part_level(PARTITION_LEVEL_ONE); @@ -10223,6 +10238,21 @@ int ObInnerTableSchema::all_virtual_sql_audit_schema(ObTableSchema &table_schema false, //is_nullable false); //is_autoincrement } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("rule_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + 256, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } if (OB_SUCC(ret)) { table_schema.get_part_option().set_part_num(1); table_schema.set_part_level(PARTITION_LEVEL_ONE); @@ -11853,7 +11883,24 @@ int ObInnerTableSchema::all_virtual_sql_audit_all_virtual_sql_audit_i1_schema(Ob true);//is_storing_column } - table_schema.set_max_used_column_id(column_id + 92); + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_WITH_COLUMN_FLAGS("rule_name", //column_name + column_id + 93, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + 256, //column_length + -1, //column_precision + -1, //column_scale + false,//is_nullable + false,//is_autoincrement + false,//is_hidden + true);//is_storing_column + } + + table_schema.set_max_used_column_id(column_id + 93); return ret; } diff --git a/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp b/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp index c677fc506..06644ff2c 100644 --- a/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.15001_15050.cpp @@ -1437,6 +1437,21 @@ int ObInnerTableSchema::all_virtual_sql_audit_ora_schema(ObTableSchema &table_sc false, //is_nullable false); //is_autoincrement } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RULE_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + 256, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } if (OB_SUCC(ret)) { table_schema.get_part_option().set_part_num(1); table_schema.set_part_level(PARTITION_LEVEL_ONE); @@ -3067,7 +3082,24 @@ int ObInnerTableSchema::all_virtual_sql_audit_ora_all_virtual_sql_audit_i1_schem true);//is_storing_column } - table_schema.set_max_used_column_id(column_id + 92); + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_WITH_COLUMN_FLAGS("RULE_NAME", //column_name + column_id + 93, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + 256, //column_length + 2, //column_precision + -1, //column_scale + false,//is_nullable + false,//is_autoincrement + false,//is_hidden + true);//is_storing_column + } + + table_schema.set_max_used_column_id(column_id + 93); return ret; } @@ -4094,6 +4126,21 @@ int ObInnerTableSchema::all_virtual_plan_stat_ora_schema(ObTableSchema &table_sc false, //is_nullable false); //is_autoincrement } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RULE_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + 256, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } if (OB_SUCC(ret)) { table_schema.get_part_option().set_part_num(1); table_schema.set_part_level(PARTITION_LEVEL_ONE); diff --git a/src/share/inner_table/ob_inner_table_schema.15251_15300.cpp b/src/share/inner_table/ob_inner_table_schema.15251_15300.cpp index c587f0e3f..067276d59 100644 --- a/src/share/inner_table/ob_inner_table_schema.15251_15300.cpp +++ b/src/share/inner_table/ob_inner_table_schema.15251_15300.cpp @@ -5758,6 +5758,275 @@ int ObInnerTableSchema::all_virtual_database_privilege_real_agent_ora_schema(ObT return ret; } +int ObInnerTableSchema::all_virtual_tenant_rewrite_rules_real_agent_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TENANT_ID", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RULE_NAME", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_ORIGINAL_NANE_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RULE_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("PATTERN", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DB_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REPLACEMENT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("NORMALIZED_PATTERN", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("STATUS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("VERSION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("PATTERN_DIGEST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FIXED_PARAM_INFOS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DYNAMIC_PARAM_INFOS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DEF_NAME_CTX_STR", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.20001_20050.cpp b/src/share/inner_table/ob_inner_table_schema.20001_20050.cpp index ddb712b40..c5e600b2a 100644 --- a/src/share/inner_table/ob_inner_table_schema.20001_20050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.20001_20050.cpp @@ -109,7 +109,7 @@ int ObInnerTableSchema::gv_ob_plan_cache_plan_stat_schema(ObTableSchema &table_s table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); if (OB_SUCC(ret)) { - if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID,SVR_IP,SVR_PORT,PLAN_ID,SQL_ID,TYPE,IS_BIND_SENSITIVE,IS_BIND_AWARE,DB_ID,STATEMENT,QUERY_SQL,SPECIAL_PARAMS,PARAM_INFOS, SYS_VARS, CONFIGS, PLAN_HASH,FIRST_LOAD_TIME,SCHEMA_VERSION,LAST_ACTIVE_TIME,AVG_EXE_USEC,SLOWEST_EXE_TIME,SLOWEST_EXE_USEC,SLOW_COUNT,HIT_COUNT,PLAN_SIZE,EXECUTIONS,DISK_READS,DIRECT_WRITES,BUFFER_GETS,APPLICATION_WAIT_TIME,CONCURRENCY_WAIT_TIME,USER_IO_WAIT_TIME,ROWS_PROCESSED,ELAPSED_TIME,CPU_TIME,LARGE_QUERYS,DELAYED_LARGE_QUERYS,DELAYED_PX_QUERYS,OUTLINE_VERSION,OUTLINE_ID,OUTLINE_DATA,ACS_SEL_INFO,TABLE_SCAN,EVOLUTION, EVO_EXECUTIONS, EVO_CPU_TIME, TIMEOUT_COUNT, PS_STMT_ID, SESSID, TEMP_TABLES, IS_USE_JIT,OBJECT_TYPE,HINTS_INFO,HINTS_ALL_WORKED, PL_SCHEMA_ID, IS_BATCHED_MULTI_STMT FROM oceanbase.__all_virtual_plan_stat WHERE OBJECT_STATUS = 0 )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID,SVR_IP,SVR_PORT,PLAN_ID,SQL_ID,TYPE,IS_BIND_SENSITIVE,IS_BIND_AWARE, DB_ID,STATEMENT,QUERY_SQL,SPECIAL_PARAMS,PARAM_INFOS, SYS_VARS, CONFIGS, PLAN_HASH, FIRST_LOAD_TIME,SCHEMA_VERSION,LAST_ACTIVE_TIME,AVG_EXE_USEC,SLOWEST_EXE_TIME,SLOWEST_EXE_USEC, SLOW_COUNT,HIT_COUNT,PLAN_SIZE,EXECUTIONS,DISK_READS,DIRECT_WRITES,BUFFER_GETS,APPLICATION_WAIT_TIME, CONCURRENCY_WAIT_TIME,USER_IO_WAIT_TIME,ROWS_PROCESSED,ELAPSED_TIME,CPU_TIME,LARGE_QUERYS, DELAYED_LARGE_QUERYS,DELAYED_PX_QUERYS,OUTLINE_VERSION,OUTLINE_ID,OUTLINE_DATA,ACS_SEL_INFO, TABLE_SCAN,EVOLUTION, EVO_EXECUTIONS, EVO_CPU_TIME, TIMEOUT_COUNT, PS_STMT_ID, SESSID, TEMP_TABLES, IS_USE_JIT,OBJECT_TYPE,HINTS_INFO,HINTS_ALL_WORKED, PL_SCHEMA_ID, IS_BATCHED_MULTI_STMT, RULE_NAME FROM oceanbase.__all_virtual_plan_stat WHERE OBJECT_STATUS = 0 )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp b/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp index 0f3847331..5e72f58b7 100644 --- a/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp @@ -759,7 +759,7 @@ int ObInnerTableSchema::gv_ob_sql_audit_schema(ObTableSchema &table_schema) table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); if (OB_SUCC(ret)) { - if (OB_FAIL(table_schema.set_view_definition(R"__(select svr_ip as SVR_IP, svr_port as SVR_PORT, request_id as REQUEST_ID, execution_id as SQL_EXEC_ID, trace_id as TRACE_ID, session_id as SID, client_ip as CLIENT_IP, client_port as CLIENT_PORT, tenant_id as TENANT_ID, tenant_name as TENANT_NAME, effective_tenant_id as EFFECTIVE_TENANT_ID, user_id as USER_ID, user_name as USER_NAME, user_group as USER_GROUP, user_client_ip as USER_CLIENT_IP, db_id as DB_ID, db_name as DB_NAME, sql_id as SQL_ID, query_sql as QUERY_SQL, plan_id as PLAN_ID, affected_rows as AFFECTED_ROWS, return_rows as RETURN_ROWS, partition_cnt as PARTITION_CNT, ret_code as RET_CODE, qc_id as QC_ID, dfo_id as DFO_ID, sqc_id as SQC_ID, worker_id as WORKER_ID, event as EVENT, p1text as P1TEXT, p1 as P1, p2text as P2TEXT, p2 as P2, p3text as P3TEXT, p3 as P3, `level` as `LEVEL`, wait_class_id as WAIT_CLASS_ID, `wait_class#` as `WAIT_CLASS#`, wait_class as WAIT_CLASS, state as STATE, wait_time_micro as WAIT_TIME_MICRO, total_wait_time_micro as TOTAL_WAIT_TIME_MICRO, total_waits as TOTAL_WAITS, rpc_count as RPC_COUNT, plan_type as PLAN_TYPE, is_inner_sql as IS_INNER_SQL, is_executor_rpc as IS_EXECUTOR_RPC, is_hit_plan as IS_HIT_PLAN, request_time as REQUEST_TIME, elapsed_time as ELAPSED_TIME, net_time as NET_TIME, net_wait_time as NET_WAIT_TIME, queue_time as QUEUE_TIME, decode_time as DECODE_TIME, get_plan_time as GET_PLAN_TIME, execute_time as EXECUTE_TIME, application_wait_time as APPLICATION_WAIT_TIME, concurrency_wait_time as CONCURRENCY_WAIT_TIME, user_io_wait_time as USER_IO_WAIT_TIME, schedule_time as SCHEDULE_TIME, row_cache_hit as ROW_CACHE_HIT, bloom_filter_cache_hit as BLOOM_FILTER_CACHE_HIT, block_cache_hit as BLOCK_CACHE_HIT, disk_reads as DISK_READS, retry_cnt as RETRY_CNT, table_scan as TABLE_SCAN, consistency_level as CONSISTENCY_LEVEL, memstore_read_row_count as MEMSTORE_READ_ROW_COUNT, ssstore_read_row_count as SSSTORE_READ_ROW_COUNT, data_block_read_cnt as DATA_BLOCK_READ_CNT, data_block_cache_hit as DATA_BLOCK_CACHE_HIT, index_block_read_cnt as INDEX_BLOCK_READ_CNT, index_block_cache_hit as INDEX_BLOCK_CACHE_HIT, blockscan_block_cnt as BLOCKSCAN_BLOCK_CNT, blockscan_row_cnt as BLOCKSCAN_ROW_CNT, pushdown_storage_filter_row_cnt as PUSHDOWN_STORAGE_FILTER_ROW_CNT, request_memory_used as REQUEST_MEMORY_USED, expected_worker_count as EXPECTED_WORKER_COUNT, used_worker_count as USED_WORKER_COUNT, sched_info as SCHED_INFO, fuse_row_cache_hit as FUSE_ROW_CACHE_HIT, ps_client_stmt_id as PS_CLIENT_STMT_ID, ps_inner_stmt_id as PS_INNER_STMT_ID, transaction_id as TX_ID, snapshot_version as SNAPSHOT_VERSION, request_type as REQUEST_TYPE, is_batched_multi_stmt as IS_BATCHED_MULTI_STMT, ob_trace_info as OB_TRACE_INFO, plan_hash as PLAN_HASH, lock_for_read_time as LOCK_FOR_READ_TIME, params_value as PARAMS_VALUE from oceanbase.__all_virtual_sql_audit )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__(select svr_ip as SVR_IP, svr_port as SVR_PORT, request_id as REQUEST_ID, execution_id as SQL_EXEC_ID, trace_id as TRACE_ID, session_id as SID, client_ip as CLIENT_IP, client_port as CLIENT_PORT, tenant_id as TENANT_ID, tenant_name as TENANT_NAME, effective_tenant_id as EFFECTIVE_TENANT_ID, user_id as USER_ID, user_name as USER_NAME, user_group as USER_GROUP, user_client_ip as USER_CLIENT_IP, db_id as DB_ID, db_name as DB_NAME, sql_id as SQL_ID, query_sql as QUERY_SQL, plan_id as PLAN_ID, affected_rows as AFFECTED_ROWS, return_rows as RETURN_ROWS, partition_cnt as PARTITION_CNT, ret_code as RET_CODE, qc_id as QC_ID, dfo_id as DFO_ID, sqc_id as SQC_ID, worker_id as WORKER_ID, event as EVENT, p1text as P1TEXT, p1 as P1, p2text as P2TEXT, p2 as P2, p3text as P3TEXT, p3 as P3, `level` as `LEVEL`, wait_class_id as WAIT_CLASS_ID, `wait_class#` as `WAIT_CLASS#`, wait_class as WAIT_CLASS, state as STATE, wait_time_micro as WAIT_TIME_MICRO, total_wait_time_micro as TOTAL_WAIT_TIME_MICRO, total_waits as TOTAL_WAITS, rpc_count as RPC_COUNT, plan_type as PLAN_TYPE, is_inner_sql as IS_INNER_SQL, is_executor_rpc as IS_EXECUTOR_RPC, is_hit_plan as IS_HIT_PLAN, request_time as REQUEST_TIME, elapsed_time as ELAPSED_TIME, net_time as NET_TIME, net_wait_time as NET_WAIT_TIME, queue_time as QUEUE_TIME, decode_time as DECODE_TIME, get_plan_time as GET_PLAN_TIME, execute_time as EXECUTE_TIME, application_wait_time as APPLICATION_WAIT_TIME, concurrency_wait_time as CONCURRENCY_WAIT_TIME, user_io_wait_time as USER_IO_WAIT_TIME, schedule_time as SCHEDULE_TIME, row_cache_hit as ROW_CACHE_HIT, bloom_filter_cache_hit as BLOOM_FILTER_CACHE_HIT, block_cache_hit as BLOCK_CACHE_HIT, disk_reads as DISK_READS, retry_cnt as RETRY_CNT, table_scan as TABLE_SCAN, consistency_level as CONSISTENCY_LEVEL, memstore_read_row_count as MEMSTORE_READ_ROW_COUNT, ssstore_read_row_count as SSSTORE_READ_ROW_COUNT, data_block_read_cnt as DATA_BLOCK_READ_CNT, data_block_cache_hit as DATA_BLOCK_CACHE_HIT, index_block_read_cnt as INDEX_BLOCK_READ_CNT, index_block_cache_hit as INDEX_BLOCK_CACHE_HIT, blockscan_block_cnt as BLOCKSCAN_BLOCK_CNT, blockscan_row_cnt as BLOCKSCAN_ROW_CNT, pushdown_storage_filter_row_cnt as PUSHDOWN_STORAGE_FILTER_ROW_CNT, request_memory_used as REQUEST_MEMORY_USED, expected_worker_count as EXPECTED_WORKER_COUNT, used_worker_count as USED_WORKER_COUNT, sched_info as SCHED_INFO, fuse_row_cache_hit as FUSE_ROW_CACHE_HIT, ps_client_stmt_id as PS_CLIENT_STMT_ID, ps_inner_stmt_id as PS_INNER_STMT_ID, transaction_id as TX_ID, snapshot_version as SNAPSHOT_VERSION, request_type as REQUEST_TYPE, is_batched_multi_stmt as IS_BATCHED_MULTI_STMT, ob_trace_info as OB_TRACE_INFO, plan_hash as PLAN_HASH, lock_for_read_time as LOCK_FOR_READ_TIME, params_value as PARAMS_VALUE, rule_name as RULE_NAME from oceanbase.__all_virtual_sql_audit )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp b/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp index d17a7a73a..58dda1423 100644 --- a/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp +++ b/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp @@ -1281,6 +1281,56 @@ int ObInnerTableSchema::cdb_ob_database_privilege_schema(ObTableSchema &table_sc return ret; } +int ObInnerTableSchema::dba_ob_user_defined_rules_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_USER_DEFINED_RULES_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_USER_DEFINED_RULES_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT CAST(T.DB_NAME AS CHAR(128)) AS DB_NAME, CAST(T.RULE_NAME AS CHAR(256)) AS RULE_NAME, CAST(T.RULE_ID AS SIGNED) AS RULE_ID, PATTERN, REPLACEMENT, NORMALIZED_PATTERN, CAST(CASE STATUS WHEN 1 THEN 'ENABLE' WHEN 2 THEN 'DISABLE' ELSE NULL END AS CHAR(10)) AS STATUS, CAST(T.VERSION AS SIGNED) AS VERSION, CAST(T.PATTERN_DIGEST AS UNSIGNED) AS PATTERN_DIGEST FROM oceanbase.__all_tenant_rewrite_rules T WHERE T.STATUS != 3 )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::dba_ob_cluster_event_history_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.28001_28050.cpp b/src/share/inner_table/ob_inner_table_schema.28001_28050.cpp index 8bc27d56a..96069b23a 100644 --- a/src/share/inner_table/ob_inner_table_schema.28001_28050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.28001_28050.cpp @@ -59,7 +59,7 @@ int ObInnerTableSchema::gv_ob_sql_audit_ora_schema(ObTableSchema &table_schema) table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); if (OB_SUCC(ret)) { - if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT svr_ip SVR_IP, svr_port SVR_PORT, request_id REQUEST_ID, execution_id SQL_EXEC_ID, trace_id TRACE_ID, session_id SID, client_ip CLIENT_IP, client_port CLIENT_PORT, tenant_id TENANT_ID, effective_tenant_id EFFECTIVE_TENANT_ID, tenant_name TENANT_NAME, user_id USER_ID, user_name USER_NAME, user_group as USER_GROUP, user_client_ip as USER_CLIENT_IP, db_id DB_ID, db_name DB_NAME, sql_id SQL_ID, query_sql QUERY_SQL, plan_id PLAN_ID, affected_rows AFFECTED_ROWS, return_rows RETURN_ROWS, partition_cnt PARTITION_CNT, ret_code RET_CODE, qc_id QC_ID, dfo_id DFO_ID, sqc_id SQC_ID, worker_id WORKER_ID, event EVENT, p1text P1TEXT, p1 P1, p2text P2TEXT, p2 P2, p3text P3TEXT, p3 P3, "LEVEL" "LEVEL", wait_class_id WAIT_CLASS_ID, "WAIT_CLASS#" "WAIT_CLASS#", wait_class WAIT_CLASS, state STATE, wait_time_micro WAIT_TIME_MICRO, total_wait_time_micro TOTAL_WAIT_TIME_MICRO, total_waits TOTAL_WAITS, rpc_count RPC_COUNT, plan_type PLAN_TYPE, is_inner_sql IS_INNER_SQL, is_executor_rpc IS_EXECUTOR_RPC, is_hit_plan IS_HIT_PLAN, request_time REQUEST_TIME, elapsed_time ELAPSED_TIME, net_time NET_TIME, net_wait_time NET_WAIT_TIME, queue_time QUEUE_TIME, decode_time DECODE_TIME, get_plan_time GET_PLAN_TIME, execute_time EXECUTE_TIME, application_wait_time APPLICATION_WAIT_TIME, concurrency_wait_time CONCURRENCY_WAIT_TIME, user_io_wait_time USER_IO_WAIT_TIME, schedule_time SCHEDULE_TIME, row_cache_hit ROW_CACHE_HIT, bloom_filter_cache_hit BLOOM_FILTER_CACHE_HIT, block_cache_hit BLOCK_CACHE_HIT, disk_reads DISK_READS, retry_cnt RETRY_CNT, table_scan TABLE_SCAN, consistency_level CONSISTENCY_LEVEL, memstore_read_row_count MEMSTORE_READ_ROW_COUNT, ssstore_read_row_count SSSTORE_READ_ROW_COUNT, data_block_read_cnt as DATA_BLOCK_READ_CNT, data_block_cache_hit as DATA_BLOCK_CACHE_HIT, index_block_read_cnt as INDEX_BLOCK_READ_CNT, index_block_cache_hit as INDEX_BLOCK_CACHE_HIT, blockscan_block_cnt as BLOCKSCAN_BLOCK_CNT, blockscan_row_cnt as BLOCKSCAN_ROW_CNT, pushdown_storage_filter_row_cnt as PUSHDOWN_STORAGE_FILTER_ROW_CNT, request_memory_used REQUEST_MEMORY_USED, expected_worker_count EXPECTED_WORKER_COUNT, used_worker_count USED_WORKER_COUNT, sched_info SCHED_INFO, ps_client_stmt_id PS_CLIENT_STMT_ID, ps_inner_stmt_id PS_INNER_STMT_ID, transaction_id as TX_ID, snapshot_version as SNAPSHOT_VERSION, request_type as REQUEST_TYPE, is_batched_multi_stmt as IS_BATCHED_MULTI_STMT, ob_trace_info as OB_TRACE_INFO, plan_hash as PLAN_HASH, params_value as PARAMS_VALUE FROM SYS.ALL_VIRTUAL_SQL_AUDIT )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT svr_ip SVR_IP, svr_port SVR_PORT, request_id REQUEST_ID, execution_id SQL_EXEC_ID, trace_id TRACE_ID, session_id SID, client_ip CLIENT_IP, client_port CLIENT_PORT, tenant_id TENANT_ID, effective_tenant_id EFFECTIVE_TENANT_ID, tenant_name TENANT_NAME, user_id USER_ID, user_name USER_NAME, user_group as USER_GROUP, user_client_ip as USER_CLIENT_IP, db_id DB_ID, db_name DB_NAME, sql_id SQL_ID, query_sql QUERY_SQL, plan_id PLAN_ID, affected_rows AFFECTED_ROWS, return_rows RETURN_ROWS, partition_cnt PARTITION_CNT, ret_code RET_CODE, qc_id QC_ID, dfo_id DFO_ID, sqc_id SQC_ID, worker_id WORKER_ID, event EVENT, p1text P1TEXT, p1 P1, p2text P2TEXT, p2 P2, p3text P3TEXT, p3 P3, "LEVEL" "LEVEL", wait_class_id WAIT_CLASS_ID, "WAIT_CLASS#" "WAIT_CLASS#", wait_class WAIT_CLASS, state STATE, wait_time_micro WAIT_TIME_MICRO, total_wait_time_micro TOTAL_WAIT_TIME_MICRO, total_waits TOTAL_WAITS, rpc_count RPC_COUNT, plan_type PLAN_TYPE, is_inner_sql IS_INNER_SQL, is_executor_rpc IS_EXECUTOR_RPC, is_hit_plan IS_HIT_PLAN, request_time REQUEST_TIME, elapsed_time ELAPSED_TIME, net_time NET_TIME, net_wait_time NET_WAIT_TIME, queue_time QUEUE_TIME, decode_time DECODE_TIME, get_plan_time GET_PLAN_TIME, execute_time EXECUTE_TIME, application_wait_time APPLICATION_WAIT_TIME, concurrency_wait_time CONCURRENCY_WAIT_TIME, user_io_wait_time USER_IO_WAIT_TIME, schedule_time SCHEDULE_TIME, row_cache_hit ROW_CACHE_HIT, bloom_filter_cache_hit BLOOM_FILTER_CACHE_HIT, block_cache_hit BLOCK_CACHE_HIT, disk_reads DISK_READS, retry_cnt RETRY_CNT, table_scan TABLE_SCAN, consistency_level CONSISTENCY_LEVEL, memstore_read_row_count MEMSTORE_READ_ROW_COUNT, ssstore_read_row_count SSSTORE_READ_ROW_COUNT, data_block_read_cnt as DATA_BLOCK_READ_CNT, data_block_cache_hit as DATA_BLOCK_CACHE_HIT, index_block_read_cnt as INDEX_BLOCK_READ_CNT, index_block_cache_hit as INDEX_BLOCK_CACHE_HIT, blockscan_block_cnt as BLOCKSCAN_BLOCK_CNT, blockscan_row_cnt as BLOCKSCAN_ROW_CNT, pushdown_storage_filter_row_cnt as PUSHDOWN_STORAGE_FILTER_ROW_CNT, request_memory_used REQUEST_MEMORY_USED, expected_worker_count EXPECTED_WORKER_COUNT, used_worker_count USED_WORKER_COUNT, sched_info SCHED_INFO, ps_client_stmt_id PS_CLIENT_STMT_ID, ps_inner_stmt_id PS_INNER_STMT_ID, transaction_id as TX_ID, snapshot_version as SNAPSHOT_VERSION, request_type as REQUEST_TYPE, is_batched_multi_stmt as IS_BATCHED_MULTI_STMT, ob_trace_info as OB_TRACE_INFO, plan_hash as PLAN_HASH, params_value as PARAMS_VALUE, rule_name as RULE_NAME FROM SYS.ALL_VIRTUAL_SQL_AUDIT )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp b/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp index 716c01d00..3702176aa 100644 --- a/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp +++ b/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp @@ -174,6 +174,56 @@ int ObInnerTableSchema::dba_ob_table_locations_ora_schema(ObTableSchema &table_s return ret; } +int ObInnerTableSchema::dba_ob_user_defined_rules_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_USER_DEFINED_RULES_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_USER_DEFINED_RULES_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT CAST(T.DB_NAME AS VARCHAR2(128)) AS DB_NAME, CAST(T.RULE_NAME AS VARCHAR2(256)) AS RULE_NAME, CAST(T.RULE_ID AS NUMBER) AS RULE_ID, PATTERN, REPLACEMENT, NORMALIZED_PATTERN, CAST(CASE WHEN STATUS = 1 THEN 'ENABLE' WHEN STATUS = 2 THEN 'DISABLE' ELSE NULL END AS VARCHAR2(8)) AS STATUS, CAST(T.VERSION AS NUMBER) AS VERSION, CAST(T.PATTERN_DIGEST AS NUMBER) AS PATTERN_DIGEST FROM SYS.ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT T WHERE T.STATUS != 3 )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_table_idx_data_table_id_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.401_450.cpp b/src/share/inner_table/ob_inner_table_schema.401_450.cpp index 41ac8a987..51555c388 100644 --- a/src/share/inner_table/ob_inner_table_schema.401_450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.401_450.cpp @@ -2856,6 +2856,305 @@ int ObInnerTableSchema::all_column_checksum_error_info_schema(ObTableSchema &tab return ret; } +int ObInnerTableSchema::all_tenant_rewrite_rules_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_TENANT_REWRITE_RULES_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_TENANT_REWRITE_RULES_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ObObj gmt_create_default; + ObObj gmt_create_default_null; + + gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_create_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + false, //is_on_update_for_timestamp + gmt_create_default_null, + gmt_create_default) + } + + if (OB_SUCC(ret)) { + ObObj gmt_modified_default; + ObObj gmt_modified_default_null; + + gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_modified_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + true, //is_on_update_for_timestamp + gmt_modified_default_null, + gmt_modified_default) + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("rule_name", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_ORIGINAL_NANE_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("rule_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("pattern", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("db_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("replacement", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("normalized_pattern", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("version", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("pattern_digest", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj fixed_param_infos_default; + fixed_param_infos_default.set_lob_value(ObLongTextType, "", strlen("")); + ADD_COLUMN_SCHEMA_T("fixed_param_infos", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + fixed_param_infos_default, + fixed_param_infos_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj dynamic_param_infos_default; + dynamic_param_infos_default.set_lob_value(ObLongTextType, "", strlen("")); + ADD_COLUMN_SCHEMA_T("dynamic_param_infos", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + dynamic_param_infos_default, + dynamic_param_infos_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj def_name_ctx_str_default; + def_name_ctx_str_default.set_lob_value(ObLongTextType, "", strlen("")); + ADD_COLUMN_SCHEMA_T("def_name_ctx_str", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + def_name_ctx_str_default, + def_name_ctx_str_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_TENANT_REWRITE_RULES_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_cluster_event_history_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp b/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp index 57aeb134e..1f7074360 100644 --- a/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.50401_50450.cpp @@ -1779,6 +1779,141 @@ int ObInnerTableSchema::all_column_checksum_error_info_aux_lob_meta_schema(ObTab return ret; } +int ObInnerTableSchema::all_tenant_rewrite_rules_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_TENANT_REWRITE_RULES_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_cluster_event_history_aux_lob_meta_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp b/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp index 43a5fd6be..415aa9620 100644 --- a/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.60401_60450.cpp @@ -1194,6 +1194,96 @@ int ObInnerTableSchema::all_column_checksum_error_info_aux_lob_piece_schema(ObTa return ret; } +int ObInnerTableSchema::all_tenant_rewrite_rules_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_TENANT_REWRITE_RULES_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_cluster_event_history_aux_lob_piece_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.h b/src/share/inner_table/ob_inner_table_schema.h index a175ea9e1..75e825413 100644 --- a/src/share/inner_table/ob_inner_table_schema.h +++ b/src/share/inner_table/ob_inner_table_schema.h @@ -97,7 +97,8 @@ struct ALL_VIRTUAL_PLAN_STAT_CDE { HINTS_ALL_WORKED, PL_SCHEMA_ID, IS_BATCHED_MULTI_STMT, - OBJECT_STATUS + OBJECT_STATUS, + RULE_NAME }; }; @@ -220,7 +221,8 @@ struct ALL_VIRTUAL_PLAN_STAT_ORA_CDE { HINTS_ALL_WORKED, PL_SCHEMA_ID, IS_BATCHED_MULTI_STMT, - OBJECT_STATUS + OBJECT_STATUS, + RULE_NAME }; }; @@ -496,6 +498,7 @@ public: static int all_service_epoch_schema(share::schema::ObTableSchema &table_schema); static int all_spatial_reference_systems_schema(share::schema::ObTableSchema &table_schema); static int all_column_checksum_error_info_schema(share::schema::ObTableSchema &table_schema); + static int all_tenant_rewrite_rules_schema(share::schema::ObTableSchema &table_schema); static int all_cluster_event_history_schema(share::schema::ObTableSchema &table_schema); static int tenant_virtual_all_table_schema(share::schema::ObTableSchema &table_schema); static int tenant_virtual_table_column_schema(share::schema::ObTableSchema &table_schema); @@ -1033,6 +1036,7 @@ public: static int all_virtual_ls_replica_task_ora_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_ls_replica_task_plan_ora_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_database_privilege_real_agent_ora_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_tenant_rewrite_rules_real_agent_ora_schema(share::schema::ObTableSchema &table_schema); static int gv_ob_plan_cache_stat_schema(share::schema::ObTableSchema &table_schema); static int gv_ob_plan_cache_plan_stat_schema(share::schema::ObTableSchema &table_schema); static int schemata_schema(share::schema::ObTableSchema &table_schema); @@ -1293,6 +1297,7 @@ public: static int cdb_ob_users_schema(share::schema::ObTableSchema &table_schema); static int dba_ob_database_privilege_schema(share::schema::ObTableSchema &table_schema); static int cdb_ob_database_privilege_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_user_defined_rules_schema(share::schema::ObTableSchema &table_schema); static int dba_ob_cluster_event_history_schema(share::schema::ObTableSchema &table_schema); static int dba_synonyms_schema(share::schema::ObTableSchema &table_schema); static int dba_objects_ora_schema(share::schema::ObTableSchema &table_schema); @@ -1637,6 +1642,7 @@ public: static int v_ob_global_transaction_ora_schema(share::schema::ObTableSchema &table_schema); static int dba_ob_ls_ora_schema(share::schema::ObTableSchema &table_schema); static int dba_ob_table_locations_ora_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_user_defined_rules_ora_schema(share::schema::ObTableSchema &table_schema); static int all_table_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_column_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_ddl_operation_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); @@ -1853,6 +1859,7 @@ public: static int all_service_epoch_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_spatial_reference_systems_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_column_checksum_error_info_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_tenant_rewrite_rules_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_cluster_event_history_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_table_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_column_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); @@ -2070,6 +2077,7 @@ public: static int all_service_epoch_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_spatial_reference_systems_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_column_checksum_error_info_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_tenant_rewrite_rules_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_cluster_event_history_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_plan_cache_stat_all_virtual_plan_cache_stat_i1_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_session_event_all_virtual_session_event_i1_schema(share::schema::ObTableSchema &table_schema); @@ -2406,6 +2414,7 @@ const schema_create_func sys_table_schema_creators [] = { ObInnerTableSchema::all_service_epoch_schema, ObInnerTableSchema::all_spatial_reference_systems_schema, ObInnerTableSchema::all_column_checksum_error_info_schema, + ObInnerTableSchema::all_tenant_rewrite_rules_schema, ObInnerTableSchema::all_cluster_event_history_schema, NULL,}; @@ -2946,6 +2955,7 @@ const schema_create_func virtual_table_schema_creators [] = { ObInnerTableSchema::all_virtual_ls_replica_task_ora_schema, ObInnerTableSchema::all_virtual_ls_replica_task_plan_ora_schema, ObInnerTableSchema::all_virtual_database_privilege_real_agent_ora_schema, + ObInnerTableSchema::all_virtual_tenant_rewrite_rules_real_agent_ora_schema, ObInnerTableSchema::all_virtual_plan_cache_stat_all_virtual_plan_cache_stat_i1_schema, ObInnerTableSchema::all_virtual_session_event_all_virtual_session_event_i1_schema, ObInnerTableSchema::all_virtual_session_wait_all_virtual_session_wait_i1_schema, @@ -3226,6 +3236,7 @@ const schema_create_func sys_view_schema_creators [] = { ObInnerTableSchema::cdb_ob_users_schema, ObInnerTableSchema::dba_ob_database_privilege_schema, ObInnerTableSchema::cdb_ob_database_privilege_schema, + ObInnerTableSchema::dba_ob_user_defined_rules_schema, ObInnerTableSchema::dba_ob_cluster_event_history_schema, ObInnerTableSchema::dba_synonyms_schema, ObInnerTableSchema::dba_objects_ora_schema, @@ -3570,6 +3581,7 @@ const schema_create_func sys_view_schema_creators [] = { ObInnerTableSchema::v_ob_global_transaction_ora_schema, ObInnerTableSchema::dba_ob_ls_ora_schema, ObInnerTableSchema::dba_ob_table_locations_ora_schema, + ObInnerTableSchema::dba_ob_user_defined_rules_ora_schema, NULL,}; const schema_create_func core_index_table_schema_creators [] = { @@ -3870,6 +3882,7 @@ const uint64_t tenant_space_tables [] = { OB_ALL_SERVICE_EPOCH_TID, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TID, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TID, + OB_ALL_TENANT_REWRITE_RULES_TID, OB_TENANT_VIRTUAL_ALL_TABLE_TID, OB_TENANT_VIRTUAL_TABLE_COLUMN_TID, OB_TENANT_VIRTUAL_TABLE_INDEX_TID, @@ -4232,6 +4245,7 @@ const uint64_t tenant_space_tables [] = { OB_ALL_VIRTUAL_LS_REPLICA_TASK_ORA_TID, OB_ALL_VIRTUAL_LS_REPLICA_TASK_PLAN_ORA_TID, OB_ALL_VIRTUAL_DATABASE_PRIVILEGE_REAL_AGENT_ORA_TID, + OB_ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA_TID, OB_GV_OB_PLAN_CACHE_STAT_TID, OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID, OB_SCHEMATA_TID, @@ -4419,6 +4433,7 @@ const uint64_t tenant_space_tables [] = { OB_DBA_SCHEDULER_WINDOWS_TID, OB_DBA_OB_USERS_TID, OB_DBA_OB_DATABASE_PRIVILEGE_TID, + OB_DBA_OB_USER_DEFINED_RULES_TID, OB_DBA_SYNONYMS_TID, OB_DBA_OBJECTS_ORA_TID, OB_ALL_OBJECTS_TID, @@ -4762,6 +4777,7 @@ const uint64_t tenant_space_tables [] = { OB_V_OB_GLOBAL_TRANSACTION_ORA_TID, OB_DBA_OB_LS_ORA_TID, OB_DBA_OB_TABLE_LOCATIONS_ORA_TID, + OB_DBA_OB_USER_DEFINED_RULES_ORA_TID, OB_ALL_TABLE_IDX_DATA_TABLE_ID_TID, OB_ALL_TABLE_IDX_DB_TB_NAME_TID, OB_ALL_TABLE_IDX_TB_NAME_TID, @@ -5040,6 +5056,7 @@ const uint64_t tenant_space_tables [] = { OB_ALL_SERVICE_EPOCH_AUX_LOB_META_TID, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TID, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TID, + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID, OB_ALL_TABLE_AUX_LOB_PIECE_TID, OB_ALL_COLUMN_AUX_LOB_PIECE_TID, OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TID, @@ -5236,7 +5253,8 @@ const uint64_t tenant_space_tables [] = { OB_ALL_LOG_RESTORE_SOURCE_AUX_LOB_PIECE_TID, OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TID, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TID, - OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TID, }; + OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TID, + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID, }; const uint64_t all_ora_mapping_virtual_table_org_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_TID, @@ -5662,6 +5680,7 @@ const char* const tenant_space_table_names [] = { OB_ALL_SERVICE_EPOCH_TNAME, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TNAME, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TNAME, + OB_ALL_TENANT_REWRITE_RULES_TNAME, OB_TENANT_VIRTUAL_ALL_TABLE_TNAME, OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME, OB_TENANT_VIRTUAL_TABLE_INDEX_TNAME, @@ -6024,6 +6043,7 @@ const char* const tenant_space_table_names [] = { OB_ALL_VIRTUAL_LS_REPLICA_TASK_ORA_TNAME, OB_ALL_VIRTUAL_LS_REPLICA_TASK_PLAN_ORA_TNAME, OB_ALL_VIRTUAL_DATABASE_PRIVILEGE_REAL_AGENT_ORA_TNAME, + OB_ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA_TNAME, OB_GV_OB_PLAN_CACHE_STAT_TNAME, OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME, OB_SCHEMATA_TNAME, @@ -6211,6 +6231,7 @@ const char* const tenant_space_table_names [] = { OB_DBA_SCHEDULER_WINDOWS_TNAME, OB_DBA_OB_USERS_TNAME, OB_DBA_OB_DATABASE_PRIVILEGE_TNAME, + OB_DBA_OB_USER_DEFINED_RULES_TNAME, OB_DBA_SYNONYMS_TNAME, OB_DBA_OBJECTS_ORA_TNAME, OB_ALL_OBJECTS_TNAME, @@ -6554,6 +6575,7 @@ const char* const tenant_space_table_names [] = { OB_V_OB_GLOBAL_TRANSACTION_ORA_TNAME, OB_DBA_OB_LS_ORA_TNAME, OB_DBA_OB_TABLE_LOCATIONS_ORA_TNAME, + OB_DBA_OB_USER_DEFINED_RULES_ORA_TNAME, OB_ALL_TABLE_IDX_DATA_TABLE_ID_TNAME, OB_ALL_TABLE_IDX_DB_TB_NAME_TNAME, OB_ALL_TABLE_IDX_TB_NAME_TNAME, @@ -6832,6 +6854,7 @@ const char* const tenant_space_table_names [] = { OB_ALL_SERVICE_EPOCH_AUX_LOB_META_TNAME, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TNAME, OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TNAME, + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TNAME, OB_ALL_TABLE_AUX_LOB_PIECE_TNAME, OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME, OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TNAME, @@ -7028,7 +7051,8 @@ const char* const tenant_space_table_names [] = { OB_ALL_LOG_RESTORE_SOURCE_AUX_LOB_PIECE_TNAME, OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TNAME, OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TNAME, - OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TNAME, }; + OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TNAME, + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TNAME, }; const uint64_t only_rs_vtables [] = { OB_ALL_VIRTUAL_CORE_META_TABLE_TID, @@ -9156,6 +9180,14 @@ LOBMapping const lob_aux_table_mappings [] = { ObInnerTableSchema::all_column_checksum_error_info_aux_lob_piece_schema }, + { + OB_ALL_TENANT_REWRITE_RULES_TID, + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID, + OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_tenant_rewrite_rules_aux_lob_meta_schema, + ObInnerTableSchema::all_tenant_rewrite_rules_aux_lob_piece_schema + }, + { OB_ALL_CLUSTER_EVENT_HISTORY_TID, OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_META_TID, @@ -9201,12 +9233,12 @@ static inline int get_sys_table_lob_aux_schema(const uint64_t tid, } const int64_t OB_CORE_TABLE_COUNT = 4; -const int64_t OB_SYS_TABLE_COUNT = 214; -const int64_t OB_VIRTUAL_TABLE_COUNT = 553; -const int64_t OB_SYS_VIEW_COUNT = 604; -const int64_t OB_SYS_TENANT_TABLE_COUNT = 1376; +const int64_t OB_SYS_TABLE_COUNT = 215; +const int64_t OB_VIRTUAL_TABLE_COUNT = 554; +const int64_t OB_SYS_VIEW_COUNT = 606; +const int64_t OB_SYS_TENANT_TABLE_COUNT = 1380; const int64_t OB_CORE_SCHEMA_VERSION = 1; -const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1379; +const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1383; } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.lob.cpp b/src/share/inner_table/ob_inner_table_schema.lob.cpp index c8f31caf0..6f71df9fd 100644 --- a/src/share/inner_table/ob_inner_table_schema.lob.cpp +++ b/src/share/inner_table/ob_inner_table_schema.lob.cpp @@ -21,7 +21,7 @@ inner_lob_map_t inner_lob_map; bool lob_mapping_init() { int ret = OB_SUCCESS; - if (OB_FAIL(inner_lob_map.create(217, ObModIds::OB_INNER_LOB_HASH_SET))) { + if (OB_FAIL(inner_lob_map.create(218, ObModIds::OB_INNER_LOB_HASH_SET))) { SERVER_LOG(WARN, "fail to create inner lob map", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < ARRAYSIZEOF(lob_aux_table_mappings); ++i) { diff --git a/src/share/inner_table/ob_inner_table_schema.vt.cpp b/src/share/inner_table/ob_inner_table_schema.vt.cpp index 7017b8f95..aeb2e74e9 100644 --- a/src/share/inner_table/ob_inner_table_schema.vt.cpp +++ b/src/share/inner_table/ob_inner_table_schema.vt.cpp @@ -441,6 +441,13 @@ bool vt_mapping_init() tmp_vt_mapping.is_real_vt_ = true; } + { + int64_t idx = OB_ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA_TID - start_idx; + VTMapping &tmp_vt_mapping = vt_mappings[idx]; + tmp_vt_mapping.mapping_tid_ = OB_ALL_TENANT_REWRITE_RULES_TID; + tmp_vt_mapping.is_real_vt_ = true; + } + { int64_t idx = OB_ALL_VIRTUAL_TENANT_ROLE_GRANTEE_MAP_REAL_AGENT_ORA_TID - start_idx; VTMapping &tmp_vt_mapping = vt_mappings[idx]; diff --git a/src/share/inner_table/ob_inner_table_schema_constants.h b/src/share/inner_table/ob_inner_table_schema_constants.h index 703cd49eb..ea712d9bb 100644 --- a/src/share/inner_table/ob_inner_table_schema_constants.h +++ b/src/share/inner_table/ob_inner_table_schema_constants.h @@ -244,6 +244,7 @@ const uint64_t OB_ALL_LOG_RESTORE_SOURCE_TID = 409; // "__all_log_restore_source const uint64_t OB_ALL_SERVICE_EPOCH_TID = 412; // "__all_service_epoch" const uint64_t OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TID = 413; // "__all_spatial_reference_systems" const uint64_t OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TID = 416; // "__all_column_checksum_error_info" +const uint64_t OB_ALL_TENANT_REWRITE_RULES_TID = 443; // "__all_tenant_rewrite_rules" const uint64_t OB_ALL_CLUSTER_EVENT_HISTORY_TID = 445; // "__all_cluster_event_history" const uint64_t OB_TENANT_VIRTUAL_ALL_TABLE_TID = 10001; // "__tenant_virtual_all_table" const uint64_t OB_TENANT_VIRTUAL_TABLE_COLUMN_TID = 10002; // "__tenant_virtual_table_column" @@ -781,6 +782,7 @@ const uint64_t OB_ALL_VIRTUAL_FREEZE_INFO_REAL_AGENT_ORA_TID = 15270; // "ALL_VI const uint64_t OB_ALL_VIRTUAL_LS_REPLICA_TASK_ORA_TID = 15272; // "ALL_VIRTUAL_LS_REPLICA_TASK_ORA" const uint64_t OB_ALL_VIRTUAL_LS_REPLICA_TASK_PLAN_ORA_TID = 15273; // "ALL_VIRTUAL_LS_REPLICA_TASK_PLAN_ORA" const uint64_t OB_ALL_VIRTUAL_DATABASE_PRIVILEGE_REAL_AGENT_ORA_TID = 15275; // "ALL_VIRTUAL_DATABASE_PRIVILEGE_REAL_AGENT_ORA" +const uint64_t OB_ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA_TID = 15281; // "ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA" const uint64_t OB_GV_OB_PLAN_CACHE_STAT_TID = 20001; // "GV$OB_PLAN_CACHE_STAT" const uint64_t OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID = 20002; // "GV$OB_PLAN_CACHE_PLAN_STAT" const uint64_t OB_SCHEMATA_TID = 20003; // "SCHEMATA" @@ -1041,6 +1043,7 @@ const uint64_t OB_DBA_OB_USERS_TID = 21336; // "DBA_OB_USERS" const uint64_t OB_CDB_OB_USERS_TID = 21337; // "CDB_OB_USERS" const uint64_t OB_DBA_OB_DATABASE_PRIVILEGE_TID = 21338; // "DBA_OB_DATABASE_PRIVILEGE" const uint64_t OB_CDB_OB_DATABASE_PRIVILEGE_TID = 21339; // "CDB_OB_DATABASE_PRIVILEGE" +const uint64_t OB_DBA_OB_USER_DEFINED_RULES_TID = 21340; // "DBA_OB_USER_DEFINED_RULES" const uint64_t OB_DBA_OB_CLUSTER_EVENT_HISTORY_TID = 21345; // "DBA_OB_CLUSTER_EVENT_HISTORY" const uint64_t OB_DBA_SYNONYMS_TID = 25001; // "DBA_SYNONYMS" const uint64_t OB_DBA_OBJECTS_ORA_TID = 25002; // "DBA_OBJECTS_ORA" @@ -1385,6 +1388,7 @@ const uint64_t OB_GV_OB_GLOBAL_TRANSACTION_ORA_TID = 28149; // "GV$OB_GLOBAL_TRA const uint64_t OB_V_OB_GLOBAL_TRANSACTION_ORA_TID = 28150; // "V$OB_GLOBAL_TRANSACTION_ORA" const uint64_t OB_DBA_OB_LS_ORA_TID = 28151; // "DBA_OB_LS_ORA" const uint64_t OB_DBA_OB_TABLE_LOCATIONS_ORA_TID = 28152; // "DBA_OB_TABLE_LOCATIONS_ORA" +const uint64_t OB_DBA_OB_USER_DEFINED_RULES_ORA_TID = 28171; // "DBA_OB_USER_DEFINED_RULES_ORA" const uint64_t OB_ALL_TABLE_AUX_LOB_META_TID = 50003; // "__all_table_aux_lob_meta" const uint64_t OB_ALL_COLUMN_AUX_LOB_META_TID = 50004; // "__all_column_aux_lob_meta" const uint64_t OB_ALL_DDL_OPERATION_AUX_LOB_META_TID = 50005; // "__all_ddl_operation_aux_lob_meta" @@ -1601,6 +1605,7 @@ const uint64_t OB_ALL_LOG_RESTORE_SOURCE_AUX_LOB_META_TID = 50409; // "__all_log const uint64_t OB_ALL_SERVICE_EPOCH_AUX_LOB_META_TID = 50412; // "__all_service_epoch_aux_lob_meta" const uint64_t OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TID = 50413; // "__all_spatial_reference_systems_aux_lob_meta" const uint64_t OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TID = 50416; // "__all_column_checksum_error_info_aux_lob_meta" +const uint64_t OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TID = 50443; // "__all_tenant_rewrite_rules_aux_lob_meta" const uint64_t OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_META_TID = 50445; // "__all_cluster_event_history_aux_lob_meta" const uint64_t OB_ALL_TABLE_AUX_LOB_PIECE_TID = 60003; // "__all_table_aux_lob_piece" const uint64_t OB_ALL_COLUMN_AUX_LOB_PIECE_TID = 60004; // "__all_column_aux_lob_piece" @@ -1818,6 +1823,7 @@ const uint64_t OB_ALL_LOG_RESTORE_SOURCE_AUX_LOB_PIECE_TID = 60409; // "__all_lo const uint64_t OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TID = 60412; // "__all_service_epoch_aux_lob_piece" const uint64_t OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TID = 60413; // "__all_spatial_reference_systems_aux_lob_piece" const uint64_t OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TID = 60416; // "__all_column_checksum_error_info_aux_lob_piece" +const uint64_t OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TID = 60443; // "__all_tenant_rewrite_rules_aux_lob_piece" const uint64_t OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_PIECE_TID = 60445; // "__all_cluster_event_history_aux_lob_piece" const uint64_t OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TID = 14999; // "__all_virtual_plan_cache_stat" const uint64_t OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TID = 14998; // "__all_virtual_session_event" @@ -2141,6 +2147,7 @@ const char *const OB_ALL_LOG_RESTORE_SOURCE_TNAME = "__all_log_restore_source"; const char *const OB_ALL_SERVICE_EPOCH_TNAME = "__all_service_epoch"; const char *const OB_ALL_SPATIAL_REFERENCE_SYSTEMS_TNAME = "__all_spatial_reference_systems"; const char *const OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_TNAME = "__all_column_checksum_error_info"; +const char *const OB_ALL_TENANT_REWRITE_RULES_TNAME = "__all_tenant_rewrite_rules"; const char *const OB_ALL_CLUSTER_EVENT_HISTORY_TNAME = "__all_cluster_event_history"; const char *const OB_TENANT_VIRTUAL_ALL_TABLE_TNAME = "__tenant_virtual_all_table"; const char *const OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME = "__tenant_virtual_table_column"; @@ -2678,6 +2685,7 @@ const char *const OB_ALL_VIRTUAL_FREEZE_INFO_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL const char *const OB_ALL_VIRTUAL_LS_REPLICA_TASK_ORA_TNAME = "ALL_VIRTUAL_LS_REPLICA_TASK"; const char *const OB_ALL_VIRTUAL_LS_REPLICA_TASK_PLAN_ORA_TNAME = "ALL_VIRTUAL_LS_REPLICA_TASK_PLAN"; const char *const OB_ALL_VIRTUAL_DATABASE_PRIVILEGE_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL_DATABASE_PRIVILEGE_REAL_AGENT"; +const char *const OB_ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT"; const char *const OB_GV_OB_PLAN_CACHE_STAT_TNAME = "GV$OB_PLAN_CACHE_STAT"; const char *const OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME = "GV$OB_PLAN_CACHE_PLAN_STAT"; const char *const OB_SCHEMATA_TNAME = "SCHEMATA"; @@ -2938,6 +2946,7 @@ const char *const OB_DBA_OB_USERS_TNAME = "DBA_OB_USERS"; const char *const OB_CDB_OB_USERS_TNAME = "CDB_OB_USERS"; const char *const OB_DBA_OB_DATABASE_PRIVILEGE_TNAME = "DBA_OB_DATABASE_PRIVILEGE"; const char *const OB_CDB_OB_DATABASE_PRIVILEGE_TNAME = "CDB_OB_DATABASE_PRIVILEGE"; +const char *const OB_DBA_OB_USER_DEFINED_RULES_TNAME = "DBA_OB_USER_DEFINED_RULES"; const char *const OB_DBA_OB_CLUSTER_EVENT_HISTORY_TNAME = "DBA_OB_CLUSTER_EVENT_HISTORY"; const char *const OB_DBA_SYNONYMS_TNAME = "DBA_SYNONYMS"; const char *const OB_DBA_OBJECTS_ORA_TNAME = "DBA_OBJECTS"; @@ -3282,6 +3291,7 @@ const char *const OB_GV_OB_GLOBAL_TRANSACTION_ORA_TNAME = "GV$OB_GLOBAL_TRANSACT const char *const OB_V_OB_GLOBAL_TRANSACTION_ORA_TNAME = "V$OB_GLOBAL_TRANSACTION"; const char *const OB_DBA_OB_LS_ORA_TNAME = "DBA_OB_LS"; const char *const OB_DBA_OB_TABLE_LOCATIONS_ORA_TNAME = "DBA_OB_TABLE_LOCATIONS"; +const char *const OB_DBA_OB_USER_DEFINED_RULES_ORA_TNAME = "DBA_OB_USER_DEFINED_RULES"; const char *const OB_ALL_TABLE_AUX_LOB_META_TNAME = "__all_table_aux_lob_meta"; const char *const OB_ALL_COLUMN_AUX_LOB_META_TNAME = "__all_column_aux_lob_meta"; const char *const OB_ALL_DDL_OPERATION_AUX_LOB_META_TNAME = "__all_ddl_operation_aux_lob_meta"; @@ -3498,6 +3508,7 @@ const char *const OB_ALL_LOG_RESTORE_SOURCE_AUX_LOB_META_TNAME = "__all_log_rest const char *const OB_ALL_SERVICE_EPOCH_AUX_LOB_META_TNAME = "__all_service_epoch_aux_lob_meta"; const char *const OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_META_TNAME = "__all_spatial_reference_systems_aux_lob_meta"; const char *const OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_META_TNAME = "__all_column_checksum_error_info_aux_lob_meta"; +const char *const OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_META_TNAME = "__all_tenant_rewrite_rules_aux_lob_meta"; const char *const OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_META_TNAME = "__all_cluster_event_history_aux_lob_meta"; const char *const OB_ALL_TABLE_AUX_LOB_PIECE_TNAME = "__all_table_aux_lob_piece"; const char *const OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME = "__all_column_aux_lob_piece"; @@ -3715,6 +3726,7 @@ const char *const OB_ALL_LOG_RESTORE_SOURCE_AUX_LOB_PIECE_TNAME = "__all_log_res const char *const OB_ALL_SERVICE_EPOCH_AUX_LOB_PIECE_TNAME = "__all_service_epoch_aux_lob_piece"; const char *const OB_ALL_SPATIAL_REFERENCE_SYSTEMS_AUX_LOB_PIECE_TNAME = "__all_spatial_reference_systems_aux_lob_piece"; const char *const OB_ALL_COLUMN_CHECKSUM_ERROR_INFO_AUX_LOB_PIECE_TNAME = "__all_column_checksum_error_info_aux_lob_piece"; +const char *const OB_ALL_TENANT_REWRITE_RULES_AUX_LOB_PIECE_TNAME = "__all_tenant_rewrite_rules_aux_lob_piece"; const char *const OB_ALL_CLUSTER_EVENT_HISTORY_AUX_LOB_PIECE_TNAME = "__all_cluster_event_history_aux_lob_piece"; const char *const OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TNAME = "__idx_11003_all_virtual_plan_cache_stat_i1"; const char *const OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TNAME = "__idx_11013_all_virtual_session_event_i1"; diff --git a/src/share/inner_table/ob_inner_table_schema_def.py b/src/share/inner_table/ob_inner_table_schema_def.py index f650824ed..95fd1bbae 100644 --- a/src/share/inner_table/ob_inner_table_schema_def.py +++ b/src/share/inner_table/ob_inner_table_schema_def.py @@ -5147,6 +5147,7 @@ def_table_schema( # 429 : __all_arbitration_service # 430 : __all_arbitration_service_replica_task # 431 : __all_meta_dictionary_location; + # 432 : __all_arbitration_service_replica_task_history # 433 : __all_tenant_rls_policy @@ -5159,7 +5160,36 @@ def_table_schema( # 440 : __all_tenant_rls_context_history # 441 : __all_tenant_rls_attribute # 442 : __all_tenant_rls_attribute_history -# 443 : __all_tenant_rewrite_rules + +def_table_schema( + owner = 'luofan.zp', + table_name = '__all_tenant_rewrite_rules', + table_id = '443', + table_type = 'SYSTEM_TABLE', + gm_columns = ['gmt_create', 'gmt_modified'], + rowkey_columns = [ + ('tenant_id', 'int'), + ('rule_name', 'varchar:OB_MAX_ORIGINAL_NANE_LENGTH'), + ], + + in_tenant_space = True, + is_cluster_private = False, + meta_record_in_sys = False, + + normal_columns = [ + ('rule_id', 'int'), + ('pattern', 'longtext'), + ('db_name', 'varchar:OB_MAX_DATABASE_NAME_LENGTH'), + ('replacement', 'longtext'), + ('normalized_pattern', 'longtext'), + ('status', 'int'), + ('version', 'int'), + ('pattern_digest', 'uint'), + ('fixed_param_infos', 'longtext', 'false', ''), + ('dynamic_param_infos', 'longtext', 'false', ''), + ('def_name_ctx_str', 'longtext', 'false', '') + ], +) # 444 : __all_reserved_snapshot def_table_schema( @@ -5754,6 +5784,7 @@ def_table_schema( ('pl_schema_id', 'uint'), ('is_batched_multi_stmt', 'bool'), ('object_status', 'int'), + ('rule_name', 'varchar:256'), ], vtable_route_policy = 'distributed', partition_columns = ['svr_ip', 'svr_port'], @@ -6379,7 +6410,8 @@ def_table_schema( ('plan_hash', 'uint'), ('user_group', 'int', 'true'), ('lock_for_read_time', 'bigint'), - ('params_value', 'longtext') + ('params_value', 'longtext'), + ('rule_name', 'varchar:256') ], partition_columns = ['svr_ip', 'svr_port'], vtable_route_policy = 'distributed', @@ -11223,7 +11255,8 @@ def_table_schema(**gen_oracle_mapping_real_virtual_table_def('15275', all_def_ke # 15278: all_virtual_tenant_rls_group_real_agent # 15279: all_virtual_tenant_rls_context_real_agent # 15280: all_virtual_tenant_rls_attribute_real_agent -# def_table_schema(**gen_oracle_mapping_real_virtual_table_def('15281', all_def_keywords['__all_tenant_rewrite_rules'])) +def_table_schema(**gen_oracle_mapping_real_virtual_table_def('15281', all_def_keywords['__all_tenant_rewrite_rules'])) + # 15282: ALL_VIRTUAL_TENANT_SYS_AGENT # 15283: __all_virtual_tenant_info_agent @@ -11273,7 +11306,15 @@ def_table_schema( in_tenant_space = True, rowkey_columns = [], view_definition = """ - SELECT TENANT_ID,SVR_IP,SVR_PORT,PLAN_ID,SQL_ID,TYPE,IS_BIND_SENSITIVE,IS_BIND_AWARE,DB_ID,STATEMENT,QUERY_SQL,SPECIAL_PARAMS,PARAM_INFOS, SYS_VARS, CONFIGS, PLAN_HASH,FIRST_LOAD_TIME,SCHEMA_VERSION,LAST_ACTIVE_TIME,AVG_EXE_USEC,SLOWEST_EXE_TIME,SLOWEST_EXE_USEC,SLOW_COUNT,HIT_COUNT,PLAN_SIZE,EXECUTIONS,DISK_READS,DIRECT_WRITES,BUFFER_GETS,APPLICATION_WAIT_TIME,CONCURRENCY_WAIT_TIME,USER_IO_WAIT_TIME,ROWS_PROCESSED,ELAPSED_TIME,CPU_TIME,LARGE_QUERYS,DELAYED_LARGE_QUERYS,DELAYED_PX_QUERYS,OUTLINE_VERSION,OUTLINE_ID,OUTLINE_DATA,ACS_SEL_INFO,TABLE_SCAN,EVOLUTION, EVO_EXECUTIONS, EVO_CPU_TIME, TIMEOUT_COUNT, PS_STMT_ID, SESSID, TEMP_TABLES, IS_USE_JIT,OBJECT_TYPE,HINTS_INFO,HINTS_ALL_WORKED, PL_SCHEMA_ID, IS_BATCHED_MULTI_STMT + SELECT TENANT_ID,SVR_IP,SVR_PORT,PLAN_ID,SQL_ID,TYPE,IS_BIND_SENSITIVE,IS_BIND_AWARE, + DB_ID,STATEMENT,QUERY_SQL,SPECIAL_PARAMS,PARAM_INFOS, SYS_VARS, CONFIGS, PLAN_HASH, + FIRST_LOAD_TIME,SCHEMA_VERSION,LAST_ACTIVE_TIME,AVG_EXE_USEC,SLOWEST_EXE_TIME,SLOWEST_EXE_USEC, + SLOW_COUNT,HIT_COUNT,PLAN_SIZE,EXECUTIONS,DISK_READS,DIRECT_WRITES,BUFFER_GETS,APPLICATION_WAIT_TIME, + CONCURRENCY_WAIT_TIME,USER_IO_WAIT_TIME,ROWS_PROCESSED,ELAPSED_TIME,CPU_TIME,LARGE_QUERYS, + DELAYED_LARGE_QUERYS,DELAYED_PX_QUERYS,OUTLINE_VERSION,OUTLINE_ID,OUTLINE_DATA,ACS_SEL_INFO, + TABLE_SCAN,EVOLUTION, EVO_EXECUTIONS, EVO_CPU_TIME, TIMEOUT_COUNT, PS_STMT_ID, SESSID, + TEMP_TABLES, IS_USE_JIT,OBJECT_TYPE,HINTS_INFO,HINTS_ALL_WORKED, PL_SCHEMA_ID, + IS_BATCHED_MULTI_STMT, RULE_NAME FROM oceanbase.__all_virtual_plan_stat WHERE OBJECT_STATUS = 0 """.replace("\n", " "), @@ -12178,7 +12219,8 @@ def_table_schema( ob_trace_info as OB_TRACE_INFO, plan_hash as PLAN_HASH, lock_for_read_time as LOCK_FOR_READ_TIME, - params_value as PARAMS_VALUE + params_value as PARAMS_VALUE, + rule_name as RULE_NAME from oceanbase.__all_virtual_sql_audit """.replace("\n", " "), @@ -23445,7 +23487,32 @@ def_table_schema( """.replace("\n", " ") ) -# 21340: DBA_OB_USER_DEFINED_RULES +def_table_schema( + owner = 'luofan.zp', + table_name = 'DBA_OB_USER_DEFINED_RULES', + table_id = '21340', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """SELECT + CAST(T.DB_NAME AS CHAR(128)) AS DB_NAME, + CAST(T.RULE_NAME AS CHAR(256)) AS RULE_NAME, + CAST(T.RULE_ID AS SIGNED) AS RULE_ID, + PATTERN, + REPLACEMENT, + NORMALIZED_PATTERN, + CAST(CASE STATUS WHEN 1 THEN 'ENABLE' + WHEN 2 THEN 'DISABLE' + ELSE NULL END AS CHAR(10)) AS STATUS, + CAST(T.VERSION AS SIGNED) AS VERSION, + CAST(T.PATTERN_DIGEST AS UNSIGNED) AS PATTERN_DIGEST + FROM + oceanbase.__all_tenant_rewrite_rules T + WHERE T.STATUS != 3 +""".replace("\n", " ") +) # 21341: GV$OB_SQL_PLAN # 21342: V$OB_SQL_PLAN @@ -40220,7 +40287,8 @@ def_table_schema( is_batched_multi_stmt as IS_BATCHED_MULTI_STMT, ob_trace_info as OB_TRACE_INFO, plan_hash as PLAN_HASH, - params_value as PARAMS_VALUE + params_value as PARAMS_VALUE, + rule_name as RULE_NAME FROM SYS.ALL_VIRTUAL_SQL_AUDIT """.replace("\n", " ") ) @@ -45121,7 +45189,36 @@ FROM ( # 28159: GV$OB_TRANSACTION_SCHEDULERS # 28160: V$OB_TRANSACTION_SCHEDULERS -# 28171: DBA_OB_USER_DEFINED_RULES +def_table_schema( + owner = 'luofan.zp', + table_name = 'DBA_OB_USER_DEFINED_RULES', + name_postfix = '_ORA', + database_id = 'OB_ORA_SYS_DATABASE_ID', + table_id = '28171', + table_type = 'SYSTEM_VIEW', + gm_columns = [], + rowkey_columns = [], + normal_columns = [], + in_tenant_space = True, + view_definition = + """ + SELECT + CAST(T.DB_NAME AS VARCHAR2(128)) AS DB_NAME, + CAST(T.RULE_NAME AS VARCHAR2(256)) AS RULE_NAME, + CAST(T.RULE_ID AS NUMBER) AS RULE_ID, + PATTERN, + REPLACEMENT, + NORMALIZED_PATTERN, + CAST(CASE WHEN STATUS = 1 THEN 'ENABLE' + WHEN STATUS = 2 THEN 'DISABLE' + ELSE NULL END AS VARCHAR2(8)) AS STATUS, + CAST(T.VERSION AS NUMBER) AS VERSION, + CAST(T.PATTERN_DIGEST AS NUMBER) AS PATTERN_DIGEST + FROM + SYS.ALL_VIRTUAL_TENANT_REWRITE_RULES_REAL_AGENT T + WHERE T.STATUS != 3 + """.replace("\n", " "), +) # 28172: GV$OB_SQL_PLAN # 28173: V$OB_SQL_PLAN diff --git a/src/share/inner_table/sys_package/dbms_udr.sql b/src/share/inner_table/sys_package/dbms_udr.sql new file mode 100644 index 000000000..5539fb6cf --- /dev/null +++ b/src/share/inner_table/sys_package/dbms_udr.sql @@ -0,0 +1,26 @@ +#package_name:dbms_udr +#author: luofan.zp + +CREATE OR REPLACE PACKAGE DBMS_UDR AUTHID CURRENT_USER AS + + PROCEDURE CREATE_RULE ( + rule_name IN VARCHAR, + pattern IN CLOB, + replacement IN CLOB, + enabled IN VARCHAR := 'YES' + ); + + PROCEDURE REMOVE_RULE ( + rule_name IN VARCHAR + ); + + PROCEDURE ENABLE_RULE ( + rule_name IN VARCHAR + ); + + PROCEDURE DISABLE_RULE ( + rule_name IN VARCHAR + ); + +END DBMS_UDR; +// diff --git a/src/share/inner_table/sys_package/dbms_udr_body.sql b/src/share/inner_table/sys_package/dbms_udr_body.sql new file mode 100644 index 000000000..7209db712 --- /dev/null +++ b/src/share/inner_table/sys_package/dbms_udr_body.sql @@ -0,0 +1,30 @@ +#package_name:dbms_udr +#author: luofan.zp + +CREATE OR REPLACE PACKAGE BODY DBMS_UDR AS + + PROCEDURE CREATE_RULE ( + rule_name IN VARCHAR, + pattern IN CLOB, + replacement IN CLOB, + enabled IN VARCHAR := 'YES' + ); + PRAGMA INTERFACE(c, CREATE_RULE); + + PROCEDURE REMOVE_RULE ( + rule_name IN VARCHAR + ); + PRAGMA INTERFACE(c, REMOVE_RULE); + + PROCEDURE ENABLE_RULE ( + rule_name IN VARCHAR + ); + PRAGMA INTERFACE(c, ENABLE_RULE); + + PROCEDURE DISABLE_RULE ( + rule_name IN VARCHAR + ); + PRAGMA INTERFACE(c, DISABLE_RULE); + +END DBMS_UDR; +// diff --git a/src/share/inner_table/sys_package/dbms_udr_body_mysql.sql b/src/share/inner_table/sys_package/dbms_udr_body_mysql.sql new file mode 100644 index 000000000..0668ffbd3 --- /dev/null +++ b/src/share/inner_table/sys_package/dbms_udr_body_mysql.sql @@ -0,0 +1,30 @@ +#package_name: dbms_udr +#author: luofan.zp + +CREATE OR REPLACE PACKAGE BODY DBMS_UDR + + PROCEDURE CREATE_RULE ( + rule_name VARCHAR(256), + rule_owner_name VARCHAR(128), + pattern LONGTEXT, + replacement LONGTEXT, + enabled VARCHAR(64) DEFAULT 'YES' + ); + PRAGMA INTERFACE(c, CREATE_RULE); + + PROCEDURE REMOVE_RULE ( + rule_name VARCHAR(256) + ); + PRAGMA INTERFACE(c, REMOVE_RULE); + + PROCEDURE ENABLE_RULE ( + rule_name VARCHAR(256) + ); + PRAGMA INTERFACE(c, ENABLE_RULE); + + PROCEDURE DISABLE_RULE ( + rule_name VARCHAR(256) + ); + PRAGMA INTERFACE(c, DISABLE_RULE); + +END dbms_udr; diff --git a/src/share/inner_table/sys_package/dbms_udr_mysql.sql b/src/share/inner_table/sys_package/dbms_udr_mysql.sql new file mode 100644 index 000000000..956906956 --- /dev/null +++ b/src/share/inner_table/sys_package/dbms_udr_mysql.sql @@ -0,0 +1,26 @@ +#package_name:dbms_udr +#author: luofan.zp + +CREATE OR REPLACE PACKAGE DBMS_UDR AUTHID CURRENT_USER + + PROCEDURE CREATE_RULE ( + rule_name VARCHAR(256), + rule_owner_name VARCHAR(128), + pattern LONGTEXT, + replacement LONGTEXT, + enabled VARCHAR(64) DEFAULT 'YES' + ); + + PROCEDURE REMOVE_RULE ( + rule_name VARCHAR(256) + ); + + PROCEDURE ENABLE_RULE ( + rule_name VARCHAR(256) + ); + + PROCEDURE DISABLE_RULE ( + rule_name VARCHAR(256) + ); + +END DBMS_UDR; diff --git a/src/share/ob_common_rpc_proxy.h b/src/share/ob_common_rpc_proxy.h index 7a9db4ab9..9c0fefb14 100644 --- a/src/share/ob_common_rpc_proxy.h +++ b/src/share/ob_common_rpc_proxy.h @@ -291,6 +291,10 @@ public: //----Definitions for Application Context---- RPC_S(PRD do_context_ddl, obrpc::OB_DO_CONTEXT_DDL, (ObContextDDLArg)); //----End of definitions for Application Context---- + + //----Definitions for sync rewrite rules---- + RPC_S(PR5 admin_sync_rewrite_rules, obrpc::OB_ADMIN_SYNC_REWRITE_RULES, (ObSyncRewriteRuleArg)); + //----End of Definitions for sync rewrite rules---- public: void set_rs_mgr(share::ObRsMgr &rs_mgr) { diff --git a/src/share/ob_max_id_fetcher.cpp b/src/share/ob_max_id_fetcher.cpp index f42cf2691..db75bfb9b 100644 --- a/src/share/ob_max_id_fetcher.cpp +++ b/src/share/ob_max_id_fetcher.cpp @@ -49,6 +49,7 @@ const char *ObMaxIdFetcher::max_id_name_info_[OB_MAX_ID_TYPE][2] = { { "ob_max_used_logstrema_group_id", "max used log stream group id"}, { "ob_max_used_sys_pl_object_id", "max used sys pl object id"}, { "ob_max_used_object_id", "max used object id"}, + { "ob_max_used_rewrite_rule_version", "max used rewrite rule version"}, /* the following id_type will be changed to ob_max_used_object_id and won't be persisted. */ { "ob_max_used_table_id", "max used table id"}, { "ob_max_used_database_id", "max used database id"}, @@ -105,7 +106,8 @@ int ObMaxIdFetcher::convert_id_type( case OB_MAX_USED_LS_ID_TYPE: case OB_MAX_USED_LS_GROUP_ID_TYPE: case OB_MAX_USED_SYS_PL_OBJECT_ID_TYPE: - case OB_MAX_USED_OBJECT_ID_TYPE: { + case OB_MAX_USED_OBJECT_ID_TYPE: + case OB_MAX_USED_REWRITE_RULE_VERSION_TYPE: { dst = src; break; } @@ -286,7 +288,8 @@ int ObMaxIdFetcher::fetch_new_max_id(const uint64_t tenant_id, case OB_MAX_USED_DDL_TASK_ID_TYPE: case OB_MAX_USED_UNIT_GROUP_ID_TYPE: case OB_MAX_USED_LS_ID_TYPE: - case OB_MAX_USED_LS_GROUP_ID_TYPE: { + case OB_MAX_USED_LS_GROUP_ID_TYPE: + case OB_MAX_USED_REWRITE_RULE_VERSION_TYPE: { // won't check other id break; } diff --git a/src/share/ob_max_id_fetcher.h b/src/share/ob_max_id_fetcher.h index 97c8f7973..f6941a8ee 100644 --- a/src/share/ob_max_id_fetcher.h +++ b/src/share/ob_max_id_fetcher.h @@ -41,6 +41,7 @@ enum ObMaxIdType OB_MAX_USED_LS_GROUP_ID_TYPE, OB_MAX_USED_SYS_PL_OBJECT_ID_TYPE, /* used for sys package object id */ OB_MAX_USED_OBJECT_ID_TYPE, /* used for all kinds of user schema objects */ + OB_MAX_USED_REWRITE_RULE_VERSION_TYPE, /* the following ObMaxIdType will be changed to OB_MAX_USED_OBJECT_ID_TYPE and won't be persisted. */ OB_MAX_USED_TABLE_ID_TYPE, diff --git a/src/share/ob_rpc_struct.cpp b/src/share/ob_rpc_struct.cpp index 2d38273ec..dd89c46b9 100644 --- a/src/share/ob_rpc_struct.cpp +++ b/src/share/ob_rpc_struct.cpp @@ -7228,6 +7228,8 @@ int ObFetchLocationResult::set_servers( return servers_.assign(servers); } +OB_SERIALIZE_MEMBER(ObSyncRewriteRuleArg, tenant_id_); + OB_SERIALIZE_MEMBER(ObInitTenantConfigArg, tenant_configs_); int ObInitTenantConfigArg::assign(const ObInitTenantConfigArg &other) diff --git a/src/share/ob_rpc_struct.h b/src/share/ob_rpc_struct.h index 8fe3f0cb0..baa061ac5 100644 --- a/src/share/ob_rpc_struct.h +++ b/src/share/ob_rpc_struct.h @@ -8092,6 +8092,24 @@ private: common::ObSArray servers_; }; +struct ObSyncRewriteRuleArg +{ + OB_UNIS_VERSION(1); +public: + ObSyncRewriteRuleArg(): tenant_id_(common::OB_INVALID_TENANT_ID) + {} + ~ObSyncRewriteRuleArg() {} + int assign(const ObSyncRewriteRuleArg &other) + { + tenant_id_ = other.tenant_id_; + return common::OB_SUCCESS; + } + bool is_valid() const { return common::OB_INVALID_TENANT_ID != tenant_id_; } + TO_STRING_KV(K_(tenant_id)); + + uint64_t tenant_id_; +}; + struct ObInitTenantConfigArg { OB_UNIS_VERSION(1); diff --git a/src/share/ob_srv_rpc_proxy.h b/src/share/ob_srv_rpc_proxy.h index 9dd7a4c4f..9485c2260 100644 --- a/src/share/ob_srv_rpc_proxy.h +++ b/src/share/ob_srv_rpc_proxy.h @@ -177,6 +177,7 @@ public: RPC_AP(PR5 get_ls_access_mode, OB_GET_LS_ACCESS_MODE, (obrpc::ObGetLSAccessModeInfoArg), obrpc::ObLSAccessModeInfo); RPC_AP(PR5 change_ls_access_mode, OB_CHANGE_LS_ACCESS_MODE, (obrpc::ObLSAccessModeInfo), obrpc::ObChangeLSAccessModeRes); RPC_S(PR5 estimate_tablet_block_count, OB_ESTIMATE_TABLET_BLOCK_COUNT, (ObEstBlockArg), ObEstBlockRes); + RPC_S(PR5 sync_rewrite_rules, OB_SYNC_REWRITE_RULES, (ObSyncRewriteRuleArg)); }; // end of class ObSrvRpcProxy } // end of namespace rpc diff --git a/src/share/ob_upgrade_utils.cpp b/src/share/ob_upgrade_utils.cpp index 4ac858462..18dbaae92 100644 --- a/src/share/ob_upgrade_utils.cpp +++ b/src/share/ob_upgrade_utils.cpp @@ -781,12 +781,35 @@ int ObBaseUpgradeProcessor::init( int ObUpgradeFor4100Processor::post_upgrade() { int ret = OB_SUCCESS; - if (OB_FAIL(post_upgrade_for_srs())) { + const uint64_t tenant_id = get_tenant_id(); + if (OB_FAIL(check_inner_stat())) { + LOG_WARN("fail to check inner stat", KR(ret)); + } else if (OB_FAIL(post_upgrade_for_srs())) { LOG_WARN("post upgrade for srs failed", K(ret)); + } else if (OB_FAIL(init_rewrite_rule_version(tenant_id))) { + LOG_WARN("fail to init rewrite rule version", K(ret), K(tenant_id)); } return ret; } +int ObUpgradeFor4100Processor::init_rewrite_rule_version(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = 0; + OZ (sql.append_fmt( + "insert ignore into %s " + "(tenant_id, zone, name, data_type, value, info) values " + "(%lu, '', 'ob_max_used_rewrite_rule_version', %lu, %lu, 'max used rewrite rule version')", + OB_ALL_SYS_STAT_TNAME, + OB_INVALID_TENANT_ID, + static_cast(ObIntType), + OB_INIT_REWRITE_RULE_VERSION)); + CK (sql_proxy_ != NULL); + OZ (sql_proxy_->write(tenant_id, sql.ptr(), affected_rows)); + return ret; +} + int ObUpgradeFor4100Processor::post_upgrade_for_srs() { int ret = OB_SUCCESS; @@ -815,6 +838,7 @@ int ObUpgradeFor4100Processor::post_upgrade_for_srs() return ret; } +/* =========== 4100 upgrade processor end ============= */ /* =========== special upgrade processor end ============= */ } // end share } // end oceanbase diff --git a/src/share/ob_upgrade_utils.h b/src/share/ob_upgrade_utils.h index cdb0616dd..454165f45 100644 --- a/src/share/ob_upgrade_utils.h +++ b/src/share/ob_upgrade_utils.h @@ -184,6 +184,7 @@ public: virtual int post_upgrade() override; private: int post_upgrade_for_srs(); + int init_rewrite_rule_version(const uint64_t tenant_id); }; /* =========== special upgrade processor end ============= */ diff --git a/src/share/parameter/ob_parameter_seed.ipp b/src/share/parameter/ob_parameter_seed.ipp index 3f474621d..f66d61e86 100644 --- a/src/share/parameter/ob_parameter_seed.ipp +++ b/src/share/parameter/ob_parameter_seed.ipp @@ -1315,6 +1315,10 @@ ERRSIM_DEF_STR(errsim_migration_src_server_addr, OB_CLUSTER_PARAMETER, "", DEF_BOOL(enable_cgroup, OB_CLUSTER_PARAMETER, "True", "when set to false, cgroup will not init; when set to true but cgroup root dir is not ready, print ERROR", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::STATIC_EFFECTIVE)); +DEF_BOOL(enable_user_defined_rewrite_rules, OB_TENANT_PARAMETER, "False", + "specify whether the user defined rewrite rules are enabled. " + "Value: True: enable False: disable", + ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); DEF_TIME(_ob_plan_cache_auto_flush_interval, OB_CLUSTER_PARAMETER, "0s", "[0s,)", "time interval for auto periodic flush plan cache. Range: [0s, +∞)", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); diff --git a/src/share/rc/ob_tenant_base.h b/src/share/rc/ob_tenant_base.h index 01d751e78..3c326605b 100644 --- a/src/share/rc/ob_tenant_base.h +++ b/src/share/rc/ob_tenant_base.h @@ -41,6 +41,7 @@ namespace sql { class ObPlanBaselineMgr; class ObDataAccessService; class ObDASIDService; + class ObUDRMgr; } namespace blocksstable { class ObSharedMacroBlockMgr; @@ -209,6 +210,7 @@ namespace detector storage::ObTenantFreezeInfoMgr*, \ transaction::ObTxLoopWorker *, \ storage::ObAccessService*, \ + sql::ObUDRMgr*, \ ObTestModule* \ ) diff --git a/src/sql/CMakeLists.txt b/src/sql/CMakeLists.txt index 3f07f2478..838b7122d 100644 --- a/src/sql/CMakeLists.txt +++ b/src/sql/CMakeLists.txt @@ -1126,6 +1126,17 @@ ob_set_subtarget(ob_sql session ) +ob_set_subtarget(ob_sql udr + udr/ob_udr_struct.cpp + udr/ob_udr_sql_service.cpp + udr/ob_udr_mgr.cpp + udr/ob_udr_item.cpp + udr/ob_udr_item_mgr.cpp + udr/ob_udr_analyzer.cpp + udr/ob_udr_callback.cpp + udr/ob_udr_utils.cpp +) + ob_add_new_object_target(ob_sql ob_sql) ob_set_subtarget(ob_sql_simd common diff --git a/src/sql/engine/ob_physical_plan.h b/src/sql/engine/ob_physical_plan.h index 68d983088..3f5253be4 100644 --- a/src/sql/engine/ob_physical_plan.h +++ b/src/sql/engine/ob_physical_plan.h @@ -335,6 +335,17 @@ public: inline int64_t get_ddl_execution_id() const { return ddl_execution_id_; } inline void set_ddl_task_id(const int64_t ddl_task_id) { ddl_task_id_ = ddl_task_id; } inline int64_t get_ddl_task_id() const { return ddl_task_id_; } + const common::ObString &get_rule_name() const { return stat_.rule_name_; } + inline void set_is_rewrite_sql(bool v) { stat_.is_rewrite_sql_ = v; } + inline bool is_rewrite_sql() const { return stat_.is_rewrite_sql_; } + inline void set_rule_version(int64_t version) { stat_.rule_version_ = version; } + inline int64_t get_rule_version() const { return stat_.rule_version_; } + inline void set_is_enable_udr(const bool v) { stat_.enable_udr_ = v; } + inline bool is_enable_udr() const { return stat_.enable_udr_; } + inline int set_rule_name(const common::ObString &rule_name) + { + return ob_write_string(allocator_, rule_name, stat_.rule_name_); + } public: int inc_concurrent_num(); void dec_concurrent_num(); diff --git a/src/sql/monitor/ob_exec_stat.h b/src/sql/monitor/ob_exec_stat.h index bac5ab248..2ae7306fa 100644 --- a/src/sql/monitor/ob_exec_stat.h +++ b/src/sql/monitor/ob_exec_stat.h @@ -422,6 +422,8 @@ struct ObAuditRecordData { int64_t trx_lock_for_read_elapse_; int64_t params_value_len_; char *params_value_; + char *rule_name_; + int64_t rule_name_len_; struct StmtSnapshot { share::SCN version_; // snapshot version int64_t tx_id_; // snapshot inner which txn diff --git a/src/sql/ob_sql.cpp b/src/sql/ob_sql.cpp index d7496e7a8..0462ae7ec 100644 --- a/src/sql/ob_sql.cpp +++ b/src/sql/ob_sql.cpp @@ -73,6 +73,9 @@ #include "observer/omt/ob_tenant_config_mgr.h" #include "sql/executor/ob_executor_rpc_impl.h" #include "sql/executor/ob_remote_executor_processor.h" +#include "sql/udr/ob_udr_utils.h" +#include "sql/udr/ob_udr_mgr.h" +#include "sql/udr/ob_udr_analyzer.h" #include "common/ob_smart_call.h" namespace oceanbase @@ -819,17 +822,9 @@ int ObSql::fill_result_set(const ObPsStmtId stmt_id, const ObPsStmtInfo &stmt_in return ret; } -int ObSql::do_add_ps_cache(const ObString &sql, - const ObString &no_param_sql, - const ObIArray &raw_params, - const ObIArray &fixed_param_idx, - int64_t param_cnt, +int ObSql::do_add_ps_cache(const PsCacheInfoCtx &info_ctx, ObSchemaGetterGuard &schema_guard, - stmt::StmtType stmt_type, - ObResultSet &result, - bool is_inner_sql, - bool is_sensitive_sql, - int32_t returning_into_parm_num) + ObResultSet &result) { int ret = OB_SUCCESS; ObSQLSessionInfo &session = result.get_session(); @@ -845,27 +840,21 @@ int ObSql::do_add_ps_cache(const ObString &sql, bool duplicate_prepare = false; // add stmt item if (OB_FAIL(ps_cache->get_or_add_stmt_item(db_id, - sql, + info_ctx.normalized_sql_, ps_stmt_item))) { - LOG_WARN("get or create stmt item faield", K(ret), K(db_id), K(sql)); - } else if (OB_FAIL(ps_cache->get_or_add_stmt_info(result, - sql, - no_param_sql, - raw_params, - fixed_param_idx, - param_cnt, + LOG_WARN("get or create stmt item faield", K(ret), K(db_id), K(info_ctx.normalized_sql_)); + } else if (OB_FAIL(ps_cache->get_or_add_stmt_info(info_ctx, + result, schema_guard, - stmt_type, ps_stmt_item, - ref_stmt_info, - returning_into_parm_num))) { - LOG_WARN("get or create stmt info failed", K(ret), K(ps_stmt_item), K(db_id), K(sql)); + ref_stmt_info))) { + LOG_WARN("get or create stmt info failed", K(ret), K(ps_stmt_item), K(db_id), K(info_ctx)); } else if (OB_ISNULL(ps_stmt_item) || OB_ISNULL(ref_stmt_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("stmt_item or stmt_info is NULL", K(ret), KP(ps_stmt_item), KP(ref_stmt_info)); } if (NULL != ref_stmt_info) { - ref_stmt_info->set_is_sensitive_sql(is_sensitive_sql); + ref_stmt_info->set_is_sensitive_sql(info_ctx.is_sensitive_sql_); } //add session info if (OB_SUCC(ret)) { @@ -875,11 +864,11 @@ int ObSql::do_add_ps_cache(const ObString &sql, ref_stmt_info, client_stmt_id, duplicate_prepare, - is_inner_sql))) { + info_ctx.is_inner_sql_))) { LOG_WARN("prepare_ps_stmt failed", K(ret), K(inner_stmt_id), K(client_stmt_id)); } else { result.set_statement_id(client_stmt_id); - result.set_stmt_type(stmt_type); + result.set_stmt_type(info_ctx.stmt_type_); LOG_TRACE("add ps session info", K(ret), K(*ref_stmt_info), K(client_stmt_id), K(*ps_stmt_item), K(session.get_sessid())); } @@ -907,9 +896,8 @@ int ObSql::do_real_prepare(const ObString &sql, ObStmt *basic_stmt = NULL; stmt::StmtType stmt_type = stmt::T_NONE; int64_t param_cnt = 0; - ObString normalized_sql; - int32_t returning_into_parm_num = OB_INVALID_COUNT; - ObString no_param_sql; + PsCacheInfoCtx info_ctx; + ObUDRItemMgr::UDRItemRefGuard item_guard; ObIAllocator &allocator = result.get_mem_pool(); ObSQLSessionInfo &session = result.get_session(); ObExecContext &ectx = result.get_exec_context(); @@ -926,11 +914,15 @@ int ObSql::do_real_prepare(const ObString &sql, ObPlanCacheCtx pc_ctx(sql, true, /*is_ps_mode*/ allocator, context, ectx, session.get_effective_tenant_id()); ParamStore param_store( (ObWrapperAllocator(&allocator)) ); + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(session.get_effective_tenant_id())); pc_ctx.set_is_inner_sql(is_inner_sql); CHECK_COMPATIBILITY_MODE(context.session_info_); - if (OB_ISNULL(context.session_info_) || OB_ISNULL(context.schema_guard_)) { + if (!tenant_config.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant config is invalid", K(ret)); + } else if (OB_ISNULL(context.session_info_) || OB_ISNULL(context.schema_guard_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("session info is NULL", K(ret)); } else if (OB_FAIL(parser.parse(sql, @@ -954,7 +946,7 @@ int ObSql::do_real_prepare(const ObString &sql, && context.is_prepare_stage_ && context.is_pre_execute_)) { param_cnt = parse_result.question_mark_ctx_.count_; - normalized_sql = sql; + info_ctx.normalized_sql_ = sql; if (stmt::T_ANONYMOUS_BLOCK == stmt_type && context.is_prepare_protocol_ && context.is_prepare_stage_ @@ -974,7 +966,13 @@ int ObSql::do_real_prepare(const ObString &sql, } if (OB_FAIL(generate_stmt(parse_result, NULL, context, allocator, result, basic_stmt))) { LOG_WARN("generate stmt failed", K(ret)); + } else if (!is_from_pl + && !is_inner_sql + && tenant_config->enable_user_defined_rewrite_rules + && OB_FAIL(ObUDRUtils::match_udr_item(sql, session, allocator, item_guard))) { + LOG_WARN("failed to match rewrite rule", K(ret)); } else if (ObStmt::is_dml_stmt(stmt_type) + && NULL == item_guard.get_ref_obj() && !ObStmt::is_show_stmt(stmt_type) && !is_inner_sql && !is_from_pl @@ -992,7 +990,7 @@ int ObSql::do_real_prepare(const ObString &sql, pc_ctx.fixed_param_idx_.reset(); pc_ctx.fp_result_.raw_params_.reset(); } else { - no_param_sql = pc_ctx.sql_ctx_.spm_ctx_.bl_key_.constructed_sql_; + info_ctx.no_param_sql_ = pc_ctx.sql_ctx_.spm_ctx_.bl_key_.constructed_sql_; } } @@ -1019,40 +1017,37 @@ int ObSql::do_real_prepare(const ObString &sql, //需要确保prepare的文本与客户端发过来的一致 if (is_inner_sql) { // pl - normalized_sql = basic_stmt->get_query_ctx()->get_sql_stmt(); + info_ctx.normalized_sql_ = basic_stmt->get_query_ctx()->get_sql_stmt(); } else if (result.is_returning() && ObStmt::is_dml_write_stmt(stmt_type)) { - normalized_sql = sql; - no_param_sql = basic_stmt->get_query_ctx()->get_sql_stmt(); + info_ctx.normalized_sql_ = sql; + info_ctx.no_param_sql_ = basic_stmt->get_query_ctx()->get_sql_stmt(); } else { - normalized_sql = sql; + info_ctx.normalized_sql_ = sql; } } if (OB_SUCC(ret)) { if (basic_stmt->is_insert_stmt() || basic_stmt->is_update_stmt() || basic_stmt->is_delete_stmt()) { ObDelUpdStmt *dml_stmt = static_cast(basic_stmt); if (dml_stmt->get_returning_into_exprs().count() != 0 && dml_stmt->is_returning()) { - returning_into_parm_num = dml_stmt->get_returning_into_exprs().count(); + info_ctx.num_of_returning_into_ = dml_stmt->get_returning_into_exprs().count(); } } } - LOG_INFO("generate new stmt", K(ret), K(param_cnt), K(stmt_type), K(no_param_sql), - K(normalized_sql), K(sql), K(returning_into_parm_num)); + LOG_INFO("generate new stmt", K(ret), K(param_cnt), K(stmt_type), K(info_ctx.no_param_sql_), + K(info_ctx.normalized_sql_), K(sql), K(info_ctx.num_of_returning_into_)); } if (OB_SUCC(ret)) { - if (OB_FAIL(do_add_ps_cache(normalized_sql, - no_param_sql, - pc_ctx.fp_result_.raw_params_, - pc_ctx.fixed_param_idx_, - param_cnt, - *context.schema_guard_, - stmt_type, - result, - is_inner_sql, - context.is_sensitive_, - returning_into_parm_num))) { + info_ctx.param_cnt_ = param_cnt; + info_ctx.stmt_type_ = stmt_type; + info_ctx.is_inner_sql_ = is_inner_sql; + info_ctx.is_sensitive_sql_ = context.is_sensitive_; + info_ctx.raw_params_ = &pc_ctx.fp_result_.raw_params_; + info_ctx.fixed_param_idx_ = &pc_ctx.fixed_param_idx_; + info_ctx.raw_sql_.assign_ptr(sql.ptr(), sql.length()); + if (OB_FAIL(do_add_ps_cache(info_ctx, *context.schema_guard_, result))) { LOG_WARN("add to ps plan cache failed", - K(ret), K(normalized_sql), K(param_cnt)); + K(ret), K(info_ctx.normalized_sql_), K(param_cnt)); } } //if the error code is ob_timeout, we add more error info msg for dml query. @@ -1065,7 +1060,7 @@ int ObSql::do_real_prepare(const ObString &sql, IS_SHOW_STMT(parse_result.result_tree_->children_[0]->type_))) { LOG_USER_ERROR(OB_TIMEOUT, THIS_WORKER.get_timeout_ts() - session.get_query_start_time()); } - LOG_INFO("add ps cache", K(normalized_sql), K(param_cnt), K(ret)); + LOG_INFO("add ps cache", K(info_ctx.normalized_sql_), K(param_cnt), K(ret)); return ret; } @@ -1335,6 +1330,7 @@ int ObSql::construct_ps_param(const ParamStore ¶ms, ObPlanCacheCtx &phy_ctx) { int ret = OB_SUCCESS; + phy_ctx.fp_result_.ps_params_.reset(); phy_ctx.fp_result_.ps_params_.set_allocator(&phy_ctx.allocator_); phy_ctx.fp_result_.ps_params_.set_capacity(params.count()); for (int i = 0; OB_SUCC(ret) && i < params.count(); ++i) { @@ -1533,8 +1529,7 @@ int ObSql::handle_ps_execute(const ObPsStmtId client_stmt_id, } else if (OB_FAIL(construct_param_store(ps_params, pctx->get_param_store_for_update()))) { LOG_WARN("construct param store failed", K(ret)); } else { - const ObString &sql = - !ps_info->get_no_param_sql().empty() ? ps_info->get_no_param_sql() : ps_info->get_ps_sql(); + const ObString &sql = !ps_info->get_no_param_sql().empty() ? ps_info->get_no_param_sql() : ps_info->get_ps_sql(); context.cur_sql_ = sql; #ifndef NDEBUG LOG_INFO("Begin to handle execute stmtement", K(session.get_sessid()), K(sql)); @@ -1670,6 +1665,7 @@ int ObSql::handle_remote_query(const ObRemoteSqlInfo &remote_sql_info, const uint64_t tenant_id = session->get_effective_tenant_id(); bool use_plan_cache = session->get_local_ob_enable_plan_cache(); context.self_add_plan_ = false; + context.cur_sql_ = trimed_stmt; pc_ctx = new (pc_ctx) ObPlanCacheCtx(trimed_stmt, remote_sql_info.use_ps_, /*is_ps_mode*/ allocator, @@ -1698,7 +1694,8 @@ int ObSql::handle_remote_query(const ObRemoteSqlInfo &remote_sql_info, //切割出来的query最后走batched multi stmt的逻辑去查询plan cache和生成计划 ObParser parser(allocator, session->get_sql_mode(), - session->get_local_collation_connection()); + session->get_local_collation_connection(), + pc_ctx->def_name_ctx_); ObMPParseStat parse_stat; if (OB_FAIL(parser.split_multiple_stmt(remote_sql_info.remote_sql_, queries, parse_stat))) { LOG_WARN("split multiple stmt failed", K(ret), K(remote_sql_info)); @@ -3084,7 +3081,7 @@ int ObSql::get_outline_data(ObSqlCtx &context, } if (OB_SUCC(ret) && !outline_content.empty()) { - ObParser parser(pc_ctx.allocator_, session->get_sql_mode(), session->get_local_collation_connection()); + ObParser parser(pc_ctx.allocator_, session->get_sql_mode(), session->get_local_collation_connection(), pc_ctx.def_name_ctx_); ObSqlString sql_helper; ObString temp_outline_sql; if (OB_FAIL(sql_helper.assign_fmt("select %.*s 1 from dual", outline_content.length(), @@ -3209,9 +3206,9 @@ int ObSql::parser_and_check(const ObString &outlined_stmt, } else { pctx->reset_datum_param_store(); pctx->get_param_store_for_update().reuse(); - ObParser parser(allocator, session->get_sql_mode(), session->get_local_collation_connection()); + ObParser parser(allocator, session->get_sql_mode(), session->get_local_collation_connection(), pc_ctx.def_name_ctx_); if (OB_FAIL(parser.parse(outlined_stmt, parse_result, - STD_MODE, + pc_ctx.is_rewrite_sql_ ? UDR_SQL_MODE : STD_MODE, pc_ctx.sql_ctx_.handle_batched_multi_stmt()))) { LOG_WARN("Generate syntax tree failed", K(outlined_stmt), K(ret)); } else if (pc_ctx.is_ps_mode_ @@ -3436,9 +3433,13 @@ int ObSql::pc_add_plan(ObPlanCacheCtx &pc_ctx, pc_ctx.fp_result_.pc_key_.namespace_ = ObLibCacheNameSpace::NS_CRSR; plan_added = false; bool is_batch_exec = pc_ctx.sql_ctx_.multi_stmt_item_.is_batched_multi_stmt(); + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); if (OB_ISNULL(phy_plan) || OB_ISNULL(plan_cache)) { ret = OB_NOT_INIT; LOG_WARN("Fail to generate plan", K(phy_plan), K(plan_cache)); + } else if (!tenant_config.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant config is invalid", K(ret)); } else if (OB_USE_PLAN_CACHE_NONE == phy_plan->get_phy_plan_hint().plan_cache_policy_) { LOG_DEBUG("Hint not use plan cache"); } else if (OB_FAIL(result.to_plan(pc_ctx.is_ps_mode_, phy_plan))) { @@ -3452,9 +3453,15 @@ int ObSql::pc_add_plan(ObPlanCacheCtx &pc_ctx, pc_ctx.sql_ctx_.spm_ctx_.bl_key_.sql_id_, phy_plan->stat_.sql_id_))) { LOG_WARN("failed to ob write string", K(ret)); + } else if (pc_ctx.is_rewrite_sql_ && OB_FAIL(phy_plan->set_rule_name(pc_ctx.rule_name_))) { + LOG_WARN("failed to ob write string", K(ret)); } else { + sql::ObUDRMgr *rule_mgr = MTL(sql::ObUDRMgr*); phy_plan->set_outline_state(outline_state); phy_plan->stat_.db_id_ = pc_ctx.sql_ctx_.spm_ctx_.bl_key_.db_id_; + phy_plan->stat_.is_rewrite_sql_ = pc_ctx.is_rewrite_sql_; + phy_plan->stat_.rule_version_ = rule_mgr->get_rule_version(); + phy_plan->stat_.enable_udr_ = tenant_config->enable_user_defined_rewrite_rules; if (pc_ctx.is_ps_mode_) { //远程SQL第二次进入plan,将raw_sql作为pc_key存入plan cache中, @@ -3527,7 +3534,8 @@ void ObSql::check_template_sql_can_be_prepare(ObPlanCacheCtx &pc_ctx, ObPhysical ParseResult parse_result; ObParser parser(pc_ctx.allocator_, session->get_sql_mode(), - session->get_local_collation_connection()); + session->get_local_collation_connection(), + pc_ctx.def_name_ctx_); if (OB_FAIL(parser.parse(temp_sql, parse_result))) { LOG_DEBUG("generate syntax tree failed", K(temp_sql), K(ret)); } else { @@ -3725,6 +3733,58 @@ int ObSql::need_add_plan(const ObPlanCacheCtx &pc_ctx, return ret; } +int ObSql::pc_add_udr_plan(const ObUDRItemMgr::UDRItemRefGuard &item_guard, + ObPlanCacheCtx &pc_ctx, + ObResultSet &result, + ObOutlineState &outline_state, + bool& plan_added) +{ + int ret = OB_SUCCESS; + int get_plan_err = OB_SUCCESS; + bool add_plan_to_pc = false; + ParseResult parse_result; + ObIAllocator &allocator = result.get_mem_pool(); + ObSQLSessionInfo &session = result.get_session(); + ObPlanCache *plan_cache = session.get_plan_cache(); + bool is_enable_transform_tree = !session.get_enable_exact_mode(); + ObExecContext &ectx = result.get_exec_context(); + ObPhysicalPlanCtx *pctx = ectx.get_physical_plan_ctx(); + ParamStore param_store( (ObWrapperAllocator(&allocator)) ); + const ObString &raw_sql = pc_ctx.raw_sql_; + ObPlanCacheCtx tmp_pc_ctx(raw_sql, pc_ctx.is_ps_mode_, + allocator, pc_ctx.sql_ctx_, ectx, session.get_effective_tenant_id()); + tmp_pc_ctx.fp_result_ = pc_ctx.fp_result_; + tmp_pc_ctx.normal_parse_const_cnt_ = pc_ctx.normal_parse_const_cnt_; + tmp_pc_ctx.set_is_rewrite_sql(true); + tmp_pc_ctx.rule_name_ = pc_ctx.rule_name_; + const ObUDRItem *rule_item = item_guard.get_ref_obj(); + ObParser parser(allocator, session.get_sql_mode(), + session.get_local_collation_connection(), + pc_ctx.def_name_ctx_); + if (OB_ISNULL(rule_item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rule item is null", K(ret)); + } else if (OB_FAIL(tmp_pc_ctx.fixed_param_info_list_.assign(rule_item->get_fixed_param_value_array()))) { + LOG_WARN("failed to assign fixed param info list", K(ret)); + } else if (OB_FAIL(tmp_pc_ctx.dynamic_param_info_list_.assign(rule_item->get_dynamic_param_info_array()))) { + LOG_WARN("failed to assign dynamic param info list", K(ret)); + } else if (OB_FAIL(tmp_pc_ctx.tpl_sql_const_cons_.assign(pc_ctx.tpl_sql_const_cons_))) { + LOG_WARN("failed to assign tpl sql const cons", K(ret)); + } else if (OB_FAIL(parser.parse(raw_sql, parse_result))) { + LOG_WARN("failed to parse sql", K(ret), K(raw_sql)); + } else if (OB_FAIL(ObSqlParameterization::parameterize_syntax_tree(allocator, + false/*is_transform_outline*/, + tmp_pc_ctx, + parse_result.result_tree_, + param_store, + session.get_local_collation_connection()))) { + LOG_WARN("parameterize syntax tree failed", K(ret)); + } else if (OB_FAIL(pc_add_plan(tmp_pc_ctx, result, outline_state, plan_cache, plan_added))) { + LOG_WARN("failed to add plan", K(ret)); + } + return ret; +} + OB_NOINLINE int ObSql::handle_physical_plan(const ObString &trimed_stmt, ObSqlCtx &context, ObResultSet &result, @@ -3741,6 +3801,9 @@ OB_NOINLINE int ObSql::handle_physical_plan(const ObString &trimed_stmt, ParseResult parse_result; ParseResult outline_parse_result; bool add_plan_to_pc = false; + bool is_match_udr = false; + ObUDRItemMgr::UDRItemRefGuard item_guard; + UDRBackupRecoveryGuard backup_recovery_guard(context, pc_ctx); ObSQLSessionInfo &session = result.get_session(); ObPlanCache *plan_cache = session.get_plan_cache(); ObSpmCacheCtx &spm_ctx = context.spm_ctx_; @@ -3760,6 +3823,15 @@ OB_NOINLINE int ObSql::handle_physical_plan(const ObString &trimed_stmt, context.multi_stmt_item_.is_batched_multi_stmt() && OB_FAIL(get_first_batched_multi_stmt(context.multi_stmt_item_, outlined_stmt))) { LOG_WARN("failed to get first batched stmt item", K(ret)); + } else if (OB_FAIL(ObUDRUtils::match_udr_and_refill_ctx(outlined_stmt, + context, + result, + pc_ctx, + is_match_udr, + item_guard))) { + LOG_WARN("failed to match udr and refill ctx", K(ret)); + } else if (is_match_udr + && FALSE_IT(outlined_stmt = item_guard.get_ref_obj()->get_replacement())) { } else if (OB_FAIL(handle_parser(outlined_stmt, result.get_exec_context(), pc_ctx, @@ -3797,6 +3869,7 @@ OB_NOINLINE int ObSql::handle_physical_plan(const ObString &trimed_stmt, } else { LOG_WARN("Failed to generate plan", K(ret), K(result.get_exec_context().need_disconnect())); } + } else if (OB_FALSE_IT(backup_recovery_guard.recovery())) { } else if (OB_FAIL(need_add_plan(pc_ctx, result, use_plan_cache, @@ -3804,7 +3877,13 @@ OB_NOINLINE int ObSql::handle_physical_plan(const ObString &trimed_stmt, LOG_WARN("get need_add_plan failed", K(ret)); } else if (!add_plan_to_pc) { // do nothing - } else if (OB_FAIL(pc_add_plan(pc_ctx, result, outline_state, plan_cache, plan_added))) { + } else if (is_match_udr && OB_FAIL(pc_add_udr_plan(item_guard, + pc_ctx, + result, + outline_state, + plan_added))) { + LOG_WARN("fail to add plan to plan cache", K(ret)); + } else if (!is_match_udr && OB_FAIL(pc_add_plan(pc_ctx, result, outline_state, plan_cache, plan_added))) { LOG_WARN("fail to add plan to plan cache", K(ret)); } //if the error code is ob_timeout, we add more error info msg for dml query. diff --git a/src/sql/ob_sql.h b/src/sql/ob_sql.h index cfbbf49aa..34f53bc8d 100644 --- a/src/sql/ob_sql.h +++ b/src/sql/ob_sql.h @@ -26,6 +26,7 @@ #include "sql/optimizer/ob_optimizer.h" #include "sql/rewrite/ob_transform_rule.h" #include "sql/executor/ob_maintain_dependency_info_task.h" +#include "sql/udr/ob_udr_item_mgr.h" namespace test { @@ -237,8 +238,6 @@ private: ObSql &operator=(const ObSql &other); private: - int add_param_to_param_store(const ObObjParam ¶m, - ParamStore ¶m_store); int construct_param_store(const ParamStore ¶ms, ParamStore ¶m_store); int construct_ps_param_store(const ParamStore ¶ms, @@ -253,35 +252,18 @@ private: ObSqlCtx &context, ObResultSet &result, bool is_inner_sql); - /** - * @brief 把prepare的信息加入ps cache - * @param [in] sql - prepare的语句 - * @param [in] stmt_type - stmt类型 - * @param [inout] result - prepare的结果 - * @param [out] ps_stmt_info - 加入ps cache的结构 - * @retval OB_SUCCESS execute success - * @retval OB_SOME_ERROR special errno need to handle - * - * 两个add_to_ps_cache接口输入参数不同,一个使用parser的结果,供mysql模式使用; - * 一个使用resolver的结果,供oracle模式使用 - * 其主要内容是获取prepare语句的id和type,并构造出ObPsStmtInfo,加入ps cache - */ - int do_add_ps_cache(const ObString &sql, - const ObString &no_param_sql, - const ObIArray &raw_params, - const common::ObIArray &fixed_param_idx, - int64_t param_cnt, - share::schema::ObSchemaGetterGuard &schema_guard, - stmt::StmtType stmt_type, - ObResultSet &result, - bool is_inner_sql, - bool is_sensitive_sql, - int32_t returning_into_parm_num); - + int do_add_ps_cache(const PsCacheInfoCtx &info_ctx, + share::schema::ObSchemaGetterGuard &schema_guard, + ObResultSet &result); int fill_result_set(ObResultSet &result, ObSqlCtx *context, const bool is_ps_mode, ObStmt &stmt); int fill_select_result_set(ObResultSet &result_set, ObSqlCtx *context, const bool is_ps_mode, ObCollationType collation_type, const ObString &type_name, ObStmt &basic_stmt, ObField &field); + int pc_add_udr_plan(const ObUDRItemMgr::UDRItemRefGuard &item_guard, + ObPlanCacheCtx &pc_ctx, + ObResultSet &result, + ObOutlineState &outline_state, + bool& plan_added); int handle_ps_prepare(const common::ObString &stmt, ObSqlCtx &context, ObResultSet &result, @@ -454,6 +436,8 @@ private: int handle_text_execute(const ObStmt *basic_stmt, ObSqlCtx &sql_ctx, ObResultSet &result); int check_need_reroute(ObPlanCacheCtx &pc_ctx, ObPhysicalPlan *plan, bool &need_reroute); int get_first_batched_multi_stmt(ObMultiStmtItem& multi_stmt_item, ObString& sql); + static int add_param_to_param_store(const ObObjParam ¶m, + ParamStore ¶m_store); typedef hash::ObHashMap PlanCacheMap; friend class ::test::TestOptimizerUtils; diff --git a/src/sql/parser/ob_fast_parser.cpp b/src/sql/parser/ob_fast_parser.cpp index c8562a68a..c2149c42e 100644 --- a/src/sql/parser/ob_fast_parser.cpp +++ b/src/sql/parser/ob_fast_parser.cpp @@ -12,6 +12,7 @@ #define USING_LOG_PREFIX SQL_PARSER #include "ob_fast_parser.h" +#include "sql/udr/ob_udr_struct.h" #include "share/ob_define.h" #include "lib/ash/ob_active_session_guard.h" #include "lib/worker.h" @@ -30,24 +31,22 @@ do { \ } while (0) int ObFastParser::parse(const common::ObString &stmt, - const bool enable_batched_multi_stmt, + const FPContext &fp_ctx, + common::ObIAllocator &allocator, char *&no_param_sql, int64_t &no_param_sql_len, ParamList *¶m_list, - int64_t ¶m_num, - ObCollationType connection_collation, - common::ObIAllocator &allocator, - ObSQLMode sql_mode /* default = 0*/) + int64_t ¶m_num) { ObActiveSessionGuard::get_stat().in_parse_ = true; int ret = OB_SUCCESS; if (!lib::is_oracle_mode()) { - ObFastParserMysql fp(allocator, connection_collation, enable_batched_multi_stmt, sql_mode); + ObFastParserMysql fp(allocator, fp_ctx); if (OB_FAIL(fp.parse(stmt, no_param_sql, no_param_sql_len, param_list, param_num))) { LOG_WARN("failed to fast parser", K(stmt)); } } else { - ObFastParserOracle fp(allocator, connection_collation, enable_batched_multi_stmt); + ObFastParserOracle fp(allocator, fp_ctx); if (OB_FAIL(fp.parse(stmt, no_param_sql, no_param_sql_len, param_list, param_num))) { LOG_WARN("failed to fast parser", K(stmt)); } @@ -56,6 +55,36 @@ int ObFastParser::parse(const common::ObString &stmt, return ret; } +int ObFastParser::parse(const common::ObString &stmt, + const FPContext &fp_ctx, + common::ObIAllocator &allocator, + char *&no_param_sql, + int64_t &no_param_sql_len, + ParamList *¶m_list, + int64_t ¶m_num, + ObQuestionMarkCtx &ctx) +{ + ObActiveSessionGuard::get_stat().in_parse_ = true; + int ret = OB_SUCCESS; + if (!lib::is_oracle_mode()) { + ObFastParserMysql fp(allocator, fp_ctx); + if (OB_FAIL(fp.parse(stmt, no_param_sql, no_param_sql_len, param_list, param_num))) { + LOG_WARN("failed to fast parser", K(stmt)); + } else { + ctx = fp.get_question_mark_ctx(); + } + } else { + ObFastParserOracle fp(allocator, fp_ctx); + if (OB_FAIL(fp.parse(stmt, no_param_sql, no_param_sql_len, param_list, param_num))) { + LOG_WARN("failed to fast parser", K(stmt)); + } else { + ctx = fp.get_question_mark_ctx(); + } + } + ObActiveSessionGuard::get_stat().in_parse_ = false; + return ret; +} + inline int64_t ObFastParserBase::ObRawSql::strncasecmp( int64_t pos, const char *str, const int64_t size) { @@ -75,11 +104,12 @@ inline int64_t ObFastParserBase::ObRawSql::strncasecmp( ObFastParserBase::ObFastParserBase( ObIAllocator &allocator, - const ObCollationType connection_collation, - const bool enable_batched_multi_stmt) : + const FPContext fp_ctx) : no_param_sql_(nullptr), no_param_sql_len_(0), param_num_(0), is_oracle_mode_(false), - is_batched_multi_stmt_split_on_(enable_batched_multi_stmt), + is_batched_multi_stmt_split_on_(fp_ctx.enable_batched_multi_stmt_), + is_udr_mode_(fp_ctx.is_udr_mode_), + def_name_ctx_(fp_ctx.def_name_ctx_), is_mysql_compatible_comment_(false), cur_token_begin_pos_(0), copy_begin_pos_(0), copy_end_pos_(0), tmp_buf_(nullptr), tmp_buf_len_(0), last_escape_check_pos_(0), @@ -88,11 +118,12 @@ ObFastParserBase::ObFastParserBase( parse_next_token_func_(nullptr), process_idf_func_(nullptr) { question_mark_ctx_.count_ = 0; + question_mark_ctx_.capacity_ = 0; question_mark_ctx_.by_ordinal_ = false; question_mark_ctx_.by_name_ = false; question_mark_ctx_.name_ = nullptr; - charset_type_ = ObCharset::charset_type_by_coll(connection_collation); - charset_info_ = ObCharset::get_charset(connection_collation); + charset_type_ = ObCharset::charset_type_by_coll(fp_ctx.conn_coll_); + charset_info_ = ObCharset::get_charset(fp_ctx.conn_coll_); } int ObFastParserBase::parse(const ObString &stmt, @@ -850,6 +881,25 @@ int64_t ObFastParserBase::get_question_mark(ObQuestionMarkCtx *ctx, return idx; } +int64_t ObFastParserBase::get_question_mark_by_defined_name(QuestionMarkDefNameCtx *ctx, + const char *name, + const int64_t name_len) +{ + int64_t idx = -1; + if (OB_UNLIKELY(NULL == ctx || NULL == name)) { + (void)fprintf(stderr, "ERROR question mark ctx or name is NULL\n"); + } else if (ctx->name_ != NULL) { + for (int64_t i = 0; -1 == idx && i < ctx->count_; ++i) { + if (NULL == ctx->name_[i]) { + (void)fprintf(stderr, "ERROR name_ in question mark ctx is null\n"); + } else if (0 == STRNCASECMP(ctx->name_[i], name, name_len)) { + idx = i; + } + } + } + return idx; +} + inline char* ObFastParserBase::parse_strndup(const char *str, size_t nbyte, char *buf) { MEMMOVE(buf, str, nbyte); @@ -1293,8 +1343,11 @@ int ObFastParserBase::process_question_mark() cur_token_type_ = PARAM_TOKEN; int64_t need_mem_size = FIEXED_PARAM_NODE_SIZE; int64_t text_len = raw_sql_.cur_pos_ - cur_token_begin_pos_; - // allocate all the memory needed at once - if (OB_ISNULL(buf = static_cast(allocator_.alloc(need_mem_size)))) { + if (question_mark_ctx_.by_name_) { + ret = OB_ERR_PARSER_SYNTAX; + LOG_WARN("parser syntax error", K(ret), K(raw_sql_.to_string()), K_(raw_sql_.cur_pos)); + } else if (OB_ISNULL(buf = static_cast(allocator_.alloc(need_mem_size)))) { + // allocate all the memory needed at once ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail to alloc memory", K(ret), K(need_mem_size)); } else { @@ -1309,6 +1362,72 @@ int ObFastParserBase::process_question_mark() return ret; } +int ObFastParserBase::process_ps_statement() +{ + int ret = OB_SUCCESS; + char *buf = nullptr; + cur_token_type_ = PARAM_TOKEN; + char ch = raw_sql_.char_at(raw_sql_.cur_pos_); + bool is_num = is_digit(ch) ? true : false; + if (is_num) { // ":"{int_num} + ch = raw_sql_.scan(); + while (is_digit(ch)) { + ch = raw_sql_.scan(); + } + } else { + int64_t next_idf_pos = raw_sql_.cur_pos_; + while (-1 != (next_idf_pos = is_identifier_flags(next_idf_pos))) { + raw_sql_.cur_pos_ = next_idf_pos; + } + } + int64_t need_mem_size = FIEXED_PARAM_NODE_SIZE; + int64_t text_len = raw_sql_.cur_pos_ - cur_token_begin_pos_; + need_mem_size += (text_len + 1); + if (question_mark_ctx_.by_ordinal_) { + ret = OB_ERR_PARSER_SYNTAX; + LOG_WARN("parser syntax error", K(ret), K(raw_sql_.to_string()), K_(raw_sql_.cur_pos)); + } else if (OB_ISNULL(buf = static_cast(allocator_.alloc(need_mem_size)))) { + // allocate all the memory needed at once + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret), K(need_mem_size)); + } else { + ParseNode *node = new_node(buf, T_QUESTIONMARK); + node->text_len_ = text_len; + node->raw_text_ = raw_sql_.ptr(cur_token_begin_pos_); + if (is_num) { + if (is_udr_mode_) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "question mark by number"); + LOG_WARN("question mark by number not supported", K(ret)); + } else { + node->value_ = strtoll(&node->raw_text_[1], NULL, 10); + } + } else { + int64_t ind = -1; + if (is_udr_mode_ && nullptr != def_name_ctx_) { + ind = get_question_mark_by_defined_name(def_name_ctx_, node->raw_text_, text_len); + } else { + ind = get_question_mark(&question_mark_ctx_, &allocator_, + node->raw_text_, text_len, buf); + } + node->value_ = ind; + // buf points to the beginning of the next available memory + buf += text_len + 1; + question_mark_ctx_.by_name_ = true; + } + if (OB_SUCC(ret)) { + if (node->value_ < 0) { + ret = OB_ERR_PARSER_SYNTAX; + LOG_WARN("parser syntax error", K(ret), K(raw_sql_.to_string()), K_(raw_sql_.cur_pos)); + } else { + node->raw_sql_offset_ = cur_token_begin_pos_; + lex_store_param(node, buf); + } + } + } + return ret; +} + // Used to process '`' and keep all characters before the next '`' int ObFastParserBase::process_backtick() { @@ -1750,42 +1869,6 @@ inline void ObFastParserBase::append_no_param_sql() no_param_sql_[no_param_sql_len_] = '\0'; } -int ObFastParserMysql::process_ps_statement() -{ - int ret = OB_SUCCESS; - char *buf = nullptr; - cur_token_type_ = PARAM_TOKEN; - char ch = raw_sql_.char_at(raw_sql_.cur_pos_); - while (is_digit(ch)) { - ch = raw_sql_.scan(); - } - int64_t need_mem_size = FIEXED_PARAM_NODE_SIZE; - int64_t text_len = raw_sql_.cur_pos_ - cur_token_begin_pos_; - need_mem_size += (text_len + 1); - // allocate all the memory needed at once - if (OB_ISNULL(buf = static_cast(allocator_.alloc(need_mem_size)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to alloc memory", K(ret), K(need_mem_size)); - } else { - ParseNode *node = new_node(buf, T_QUESTIONMARK); - node->text_len_ = text_len; - node->raw_text_ = raw_sql_.ptr(cur_token_begin_pos_); - int64_t ind = get_question_mark(&question_mark_ctx_, &allocator_, - node->raw_text_, text_len, buf); - if (-1 == ind) { - ret = OB_ERR_PARSER_SYNTAX; - LOG_WARN("parser syntax error", K(ret), K(raw_sql_.to_string()), K_(raw_sql_.cur_pos)); - } else { - // buf points to the beginning of the next available memory - buf += text_len + 1; - node->value_ = ind; - node->raw_sql_offset_ = cur_token_begin_pos_; - lex_store_param(node, buf); - } - } - return ret; -} - int ObFastParserMysql::process_zero_identifier() { int ret = OB_SUCCESS; @@ -2174,7 +2257,7 @@ int ObFastParserMysql::parse_next_token() } case ':': { // [":"{int_num}] - if (is_digit(raw_sql_.peek())) { + if (-1 != is_first_identifier_flags(raw_sql_.cur_pos_ + 1) || is_digit(raw_sql_.peek())) { raw_sql_.scan(); OZ (process_ps_statement()); } else { @@ -2233,58 +2316,6 @@ int ObFastParserMysql::parse_next_token() return ret; } -int ObFastParserOracle::process_ps_statement() -{ - int ret = OB_SUCCESS; - char *buf = nullptr; - cur_token_type_ = PARAM_TOKEN; - char ch = raw_sql_.char_at(raw_sql_.cur_pos_); - bool is_num = is_digit(ch) ? true : false; - if (is_num) { // ":"{int_num} - ch = raw_sql_.scan(); - while (is_digit(ch)) { - ch = raw_sql_.scan(); - } - } else { - int64_t next_idf_pos = raw_sql_.cur_pos_; - while (-1 != (next_idf_pos = is_identifier_flags(next_idf_pos))) { - raw_sql_.cur_pos_ = next_idf_pos; - } - } - int64_t need_mem_size = FIEXED_PARAM_NODE_SIZE; - int64_t text_len = raw_sql_.cur_pos_ - cur_token_begin_pos_; - need_mem_size += (text_len + 1); - // allocate all the memory needed at once - if (OB_ISNULL(buf = static_cast(allocator_.alloc(need_mem_size)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to alloc memory", K(ret), K(need_mem_size)); - } else { - ParseNode *node = new_node(buf, T_QUESTIONMARK); - node->text_len_ = text_len; - node->raw_text_ = raw_sql_.ptr(cur_token_begin_pos_); - if (is_num) { - node->value_ = strtoll(&node->raw_text_[1], NULL, 10); - } else { - int64_t ind = get_question_mark(&question_mark_ctx_, &allocator_, - node->raw_text_, text_len, buf); - if (-1 == ind) { - ret = OB_ERR_PARSER_SYNTAX; - LOG_WARN("parser syntax error", K(ret), K(raw_sql_.to_string()), K_(raw_sql_.cur_pos)); - } else { - question_mark_ctx_.by_name_ = true; - node->value_ = ind; - // buf points to the beginning of the next available memory - buf += text_len + 1; - } - } - if (OB_SUCC(ret)) { - node->raw_sql_offset_ = cur_token_begin_pos_; - lex_store_param(node, buf); - } - } - return ret; -} - /** * @param [in] : if in_q_quote is true, means that the current token * starts with ("N"|"n")?("Q"|"q"){sqbegin} diff --git a/src/sql/parser/ob_fast_parser.h b/src/sql/parser/ob_fast_parser.h index 0922ad8e0..ff22199a4 100644 --- a/src/sql/parser/ob_fast_parser.h +++ b/src/sql/parser/ob_fast_parser.h @@ -19,23 +19,55 @@ #include "sql/parser/ob_parser_utils.h" #include "sql/parser/ob_char_type.h" #include "sql/parser/parse_malloc.h" +#include "sql/udr/ob_udr_struct.h" namespace oceanbase { namespace sql { +struct FPContext +{ +public: + bool enable_batched_multi_stmt_; + bool is_udr_mode_; + common::ObCollationType conn_coll_; + ObSQLMode sql_mode_; + QuestionMarkDefNameCtx *def_name_ctx_; + + FPContext() + : enable_batched_multi_stmt_(false), + is_udr_mode_(false), + conn_coll_(CS_TYPE_INVALID), + sql_mode_(0), + def_name_ctx_(nullptr) + {} + FPContext(common::ObCollationType conn_coll) + : enable_batched_multi_stmt_(false), + is_udr_mode_(false), + conn_coll_(conn_coll), + sql_mode_(0), + def_name_ctx_(nullptr) + {} +}; + struct ObFastParser final { public: static int parse(const common::ObString &stmt, - const bool enable_batched_multi_stmt, + const FPContext &fp_ctx, + common::ObIAllocator &allocator, + char *&no_param_sql, + int64_t &no_param_sql_len, + ParamList *¶m_list, + int64_t ¶m_num); + static int parse(const common::ObString &stmt, + const FPContext &fp_ctx, + common::ObIAllocator &allocator, char *&no_param_sql, int64_t &no_param_sql_len, ParamList *¶m_list, int64_t ¶m_num, - common::ObCollationType connection_collation, - common::ObIAllocator &allocator, - ObSQLMode sql_mode = 0); + ObQuestionMarkCtx &ctx); }; class ObFastParserBase @@ -47,14 +79,14 @@ public: typedef int (ObFastParserBase::*ProcessIdfFunc) (bool is_number_begin); explicit ObFastParserBase(common::ObIAllocator &allocator, - const common::ObCollationType connection_collation, - const bool enable_batched_multi_stmt); + const FPContext fp_ctx); ~ObFastParserBase() {} int parse(const common::ObString &stmt, char *&no_param_sql, int64_t &no_param_sql_len, ParamList *¶m_list, int64_t ¶m_num); + const ObQuestionMarkCtx &get_question_mark_ctx() const { return question_mark_ctx_; } protected: enum TokenType @@ -382,6 +414,9 @@ protected: const char *name, const int64_t name_len, char *buf); + int64_t get_question_mark_by_defined_name(QuestionMarkDefNameCtx *ctx, + const char *name, + const int64_t name_len); /** * The hexadecimal number in mysql mode has the following two representations: * x'([0-9A-F])*' or 0x([0-9A-F])+ @@ -398,6 +433,7 @@ protected: int process_binary(bool is_quote); int process_hint(); int process_question_mark(); + int process_ps_statement(); int process_number(bool has_minus); int process_negative(); int process_identifier_begin_with_l(bool &need_process_ws); @@ -489,6 +525,8 @@ protected: int param_num_; bool is_oracle_mode_; bool is_batched_multi_stmt_split_on_; + bool is_udr_mode_; + QuestionMarkDefNameCtx *def_name_ctx_; bool is_mysql_compatible_comment_; int64_t cur_token_begin_pos_; int64_t copy_begin_pos_; @@ -515,10 +553,9 @@ class ObFastParserMysql final : public ObFastParserBase public: explicit ObFastParserMysql( common::ObIAllocator &allocator, - const common::ObCollationType connection_collation, - const bool enable_batched_multi_stmt, - ObSQLMode sql_mode) - : ObFastParserBase(allocator, connection_collation, enable_batched_multi_stmt),sql_mode_(sql_mode) + const FPContext fp_ctx) + : ObFastParserBase(allocator, fp_ctx), + sql_mode_(fp_ctx.sql_mode_) { is_oracle_mode_ = false; set_callback_func( @@ -541,7 +578,6 @@ private: */ int process_string(const char quote); int process_zero_identifier(); - int process_ps_statement(); int process_identifier_begin_with_n(); private: @@ -553,9 +589,8 @@ class ObFastParserOracle final : public ObFastParserBase public: explicit ObFastParserOracle( common::ObIAllocator &allocator, - const common::ObCollationType connection_collation, - const bool enable_batched_multi_stmt) - : ObFastParserBase(allocator, connection_collation, enable_batched_multi_stmt) + const FPContext fp_ctx) + : ObFastParserBase(allocator, fp_ctx) { is_oracle_mode_ = true; set_callback_func( @@ -573,7 +608,6 @@ private: * else, means that the current token starts with ("N"|"n")?{sqbegin } */ int process_string(const bool in_q_quote); - int process_ps_statement(); int process_identifier_begin_with_n(); private: diff --git a/src/sql/parser/ob_parser.cpp b/src/sql/parser/ob_parser.cpp index ceca17968..7737799ed 100644 --- a/src/sql/parser/ob_parser.cpp +++ b/src/sql/parser/ob_parser.cpp @@ -23,10 +23,14 @@ using namespace oceanbase::pl; using namespace oceanbase::sql; using namespace oceanbase::common; -ObParser::ObParser(common::ObIAllocator &allocator, ObSQLMode mode, ObCollationType conn_collation) +ObParser::ObParser(common::ObIAllocator &allocator, + ObSQLMode mode, + ObCollationType conn_collation, + QuestionMarkDefNameCtx *ctx) :allocator_(&allocator), sql_mode_(mode), - connection_collation_(conn_collation) + connection_collation_(conn_collation), + def_name_ctx_(ctx) {} ObParser::~ObParser() @@ -982,6 +986,7 @@ int ObParser::parse(const ObString &query, parse_result.is_for_trigger_ = (TRIGGER_MODE == parse_mode); parse_result.is_dynamic_sql_ = (DYNAMIC_SQL_MODE == parse_mode); parse_result.is_dbms_sql_ = (DBMS_SQL_MODE == parse_mode); + parse_result.is_for_udr_ = (UDR_SQL_MODE == parse_mode); parse_result.is_batched_multi_enabled_split_ = is_batched_multi_stmt_split_on; parse_result.is_not_utf8_connection_ = ObCharset::is_valid_collation(connection_collation_) ? (ObCharset::charset_type_by_coll(connection_collation_) != CHARSET_UTF8MB4) : false; @@ -999,6 +1004,11 @@ int ObParser::parse(const ObString &query, parse_result.connection_collation_ = connection_collation_; parse_result.mysql_compatible_comment_ = false; parse_result.enable_compatible_comment_ = true; + if (nullptr != def_name_ctx_) { + parse_result.question_mark_ctx_.by_defined_name_ = true; + parse_result.question_mark_ctx_.name_ = def_name_ctx_->name_; + parse_result.question_mark_ctx_.count_ = def_name_ctx_->count_; + } if (INS_MULTI_VALUES == parse_mode) { void *buffer = nullptr; diff --git a/src/sql/parser/ob_parser.h b/src/sql/parser/ob_parser.h index f8e88cf7e..3e297b66e 100644 --- a/src/sql/parser/ob_parser.h +++ b/src/sql/parser/ob_parser.h @@ -19,6 +19,7 @@ #include "lib/string/ob_string.h" #include "lib/charset/ob_charset.h" #include "sql/parser/ob_parser_utils.h" +#include "sql/udr/ob_udr_struct.h" namespace oceanbase { @@ -49,7 +50,8 @@ class ObParser { public: explicit ObParser(common::ObIAllocator &allocator, ObSQLMode mode, - common::ObCollationType conn_collation = common::CS_TYPE_UTF8MB4_GENERAL_CI); + common::ObCollationType conn_collation = common::CS_TYPE_UTF8MB4_GENERAL_CI, + QuestionMarkDefNameCtx *ctx = nullptr); virtual ~ObParser(); /// @param queries Note that all three members of ObString is valid, size() is the length /// of the single statement, length() is the length of remainer statements @@ -193,6 +195,7 @@ private: // we don't use it in parser now ObSQLMode sql_mode_; common::ObCollationType connection_collation_; + QuestionMarkDefNameCtx *def_name_ctx_; }; } // end namespace sql diff --git a/src/sql/parser/parse_node.c b/src/sql/parser/parse_node.c index 9e353129f..b45c30790 100644 --- a/src/sql/parser/parse_node.c +++ b/src/sql/parser/parse_node.c @@ -751,6 +751,26 @@ int64_t get_question_mark(ObQuestionMarkCtx *ctx, void *malloc_pool, const char return idx; } +int64_t get_question_mark_by_defined_name(ObQuestionMarkCtx *ctx, const char *name) +{ + int64_t idx = -1; + if (OB_UNLIKELY(NULL == ctx || NULL == name)) { + (void)fprintf(stderr, "ERROR question mark ctx or name is NULL\n"); + } else if (ctx->name_ != NULL) { + for (int64_t i = 0; -1 == idx && i < ctx->count_; ++i) { + if (NULL == ctx->name_[i]) { + (void)fprintf(stderr, "ERROR name_ in question mark ctx is null\n"); + } else if (0 == STRCASECMP(ctx->name_[i], name)) { + idx = i; + break; + } + } + } else { + (void)fprintf(stderr, "ERROR name_ in question mark ctx is null\n"); + } + return idx; +} + ParserLinkNode *new_link_node(void *malloc) { ParserLinkNode *new_node = (ParserLinkNode *)parse_malloc(sizeof(ParserLinkNode), malloc); diff --git a/src/sql/parser/parse_node.h b/src/sql/parser/parse_node.h index ee84c789d..6c01e0d2e 100644 --- a/src/sql/parser/parse_node.h +++ b/src/sql/parser/parse_node.h @@ -91,6 +91,7 @@ enum ParseMode TRIGGER_MODE, /* treat ':xxx' as identifier */ DYNAMIC_SQL_MODE, /*解析动态sql过程中,:idx和:identifier要根据语句类型确定是否检查placeholder的名字*/ DBMS_SQL_MODE, + UDR_SQL_MODE, INS_MULTI_VALUES, }; @@ -227,6 +228,7 @@ typedef struct _ObQuestionMarkCtx int capacity_; bool by_ordinal_; bool by_name_; + bool by_defined_name_; } ObQuestionMarkCtx; @@ -301,7 +303,7 @@ typedef struct uint32_t is_include_old_new_in_trigger_ : 1; uint32_t is_normal_ps_prepare_ : 1; uint32_t is_multi_values_parser_ : 1; - uint32_t reserved_ : 1; + uint32_t is_for_udr_ : 1; }; ParseNode *result_tree_; @@ -394,6 +396,7 @@ extern uint64_t parsenode_hash(const ParseNode *node, int *ret); extern bool parsenode_equal(const ParseNode *node1, const ParseNode *node2, int *ret); extern int64_t get_question_mark(ObQuestionMarkCtx *ctx, void *malloc_pool, const char *name); +extern int64_t get_question_mark_by_defined_name(ObQuestionMarkCtx *ctx, const char *name); // compare ParseNode str_value_ to pattern // @param [in] node ParseNode diff --git a/src/sql/parser/sql_parser_base.c b/src/sql/parser/sql_parser_base.c index 2457642fa..1458d1fe6 100644 --- a/src/sql/parser/sql_parser_base.c +++ b/src/sql/parser/sql_parser_base.c @@ -73,7 +73,7 @@ int parse_sql(ParseResult *p, const char *buf, size_t len) p->comment_cnt_ = 0; p->stop_add_comment_ = false; #endif - if (false == p->pl_parse_info_.is_pl_parse_) {//如果是PLParse调用的该接口,不去重置 + if (false == p->pl_parse_info_.is_pl_parse_ && !p->is_for_udr_) {//如果是PLParse调用的该接口,不去重置 p->question_mark_ctx_.count_ = 0; } diff --git a/src/sql/parser/sql_parser_mysql_mode.l b/src/sql/parser/sql_parser_mysql_mode.l index 3f0f1a297..cdbdae036 100644 --- a/src/sql/parser/sql_parser_mysql_mode.l +++ b/src/sql/parser/sql_parser_mysql_mode.l @@ -1349,6 +1349,33 @@ BEGIN(in_c_comment); } } +":"{identifier} { + ParseResult *p = (ParseResult *)yyextra; + check_value(yylval); + malloc_new_node(yylval->node, p->malloc_pool_, T_QUESTIONMARK, 0); + if (p->question_mark_ctx_.by_defined_name_) { + yylval->node->value_ = get_question_mark_by_defined_name(&p->question_mark_ctx_, yytext); + if (yylval->node->value_ < 0) { + YY_UNEXPECTED_ERROR("Invalid question mark idx\n"); + } + } else { + yylval->node->value_ = get_question_mark(&p->question_mark_ctx_, p->malloc_pool_, yytext); + } + p->question_mark_ctx_.by_name_ = true; + if (OB_UNLIKELY(p->question_mark_ctx_.by_ordinal_)) { + YY_UNEXPECTED_ERROR("Ordinal binding and Named binding cannot be combined\n"); + } + copy_and_replace_questionmark(p, yylloc->first_column, yylloc->last_column, yylval->node->value_); + if (IS_FAST_PARAMETERIZE) { + yylval->node->raw_text_ = parse_strdup(yytext, p->malloc_pool_, &(yylval->node->text_len_)); + check_malloc(yylval->node->raw_text_); + STORE_PARAM_NODE(); + } else { + yylval->node->pos_ = yylloc->first_column - 1; + return QUESTIONMARK; + } +} + ":"{identifier}"."{identifier} { ParseResult *p = (ParseResult *)yyextra; check_value(yylval); diff --git a/src/sql/plan_cache/ob_plan_cache.cpp b/src/sql/plan_cache/ob_plan_cache.cpp index ce41cde2d..032ed2855 100644 --- a/src/sql/plan_cache/ob_plan_cache.cpp +++ b/src/sql/plan_cache/ob_plan_cache.cpp @@ -32,6 +32,7 @@ #include "sql/engine/ob_physical_plan.h" #include "sql/plan_cache/ob_plan_cache_callback.h" #include "sql/plan_cache/ob_cache_object_factory.h" +#include "sql/udr/ob_udr_mgr.h" #include "pl/ob_pl.h" #include "pl/ob_pl_package.h" #include "observer/ob_req_time_service.h" @@ -389,13 +390,16 @@ int ObPlanCache::check_after_get_plan(int tmp_ret, ObPhysicalPlan *plan = NULL; bool need_late_compilation = false; ObJITEnableMode jit_mode = OFF; + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); ObPlanCacheCtx &pc_ctx = static_cast(ctx); - if (cache_obj != NULL && ObLibCacheNameSpace::NS_CRSR == cache_obj->get_ns()) { plan = static_cast(cache_obj); } if (OB_SUCC(ret)) { - if (OB_ISNULL(pc_ctx.sql_ctx_.session_info_)) { + if (!tenant_config.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant config is invalid", K(ret)); + } else if (OB_ISNULL(pc_ctx.sql_ctx_.session_info_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected null session info", K(ret)); } else if (OB_FAIL(pc_ctx.sql_ctx_.session_info_->get_jit_enabled_mode(jit_mode))) { @@ -404,17 +408,39 @@ int ObPlanCache::check_after_get_plan(int tmp_ret, // do nothing } } - if (OB_SUCC(ret) && - plan != NULL && - AUTO == jit_mode && // only use late compilation when jit_mode is auto - OB_FAIL(need_late_compile(plan, need_late_compilation))) { - LOG_WARN("failed to check for late compilation", K(ret)); - } else { - // set context's need_late_compile_ for upper layer to proceed - pc_ctx.sql_ctx_.need_late_compile_ = need_late_compilation; + if (OB_SUCC(ret) && plan != NULL) { + bool is_exists = false; + sql::ObUDRMgr *rule_mgr = MTL(sql::ObUDRMgr*); + // when the global rule version changes or enable_user_defined_rewrite_rules changes + // it is necessary to check whether the physical plan are expired + if ((plan->get_rule_version() != rule_mgr->get_rule_version() + || plan->is_enable_udr() != tenant_config->enable_user_defined_rewrite_rules)) { + if (OB_FAIL(rule_mgr->fuzzy_check_by_pattern_digest(pc_ctx.get_normalized_pattern_digest(), is_exists))) { + LOG_WARN("failed to fuzzy check by pattern digest", K(ret)); + } else if (is_exists || plan->is_rewrite_sql()) { + ret = OB_OLD_SCHEMA_VERSION; + LOG_TRACE("Obsolete user-defined rewrite rules require eviction plan", K(ret), + K(is_exists), K(pc_ctx.raw_sql_), K(plan->is_enable_udr()), K(tenant_config->enable_user_defined_rewrite_rules), + K(plan->is_rewrite_sql()), K(plan->get_rule_version()), K(rule_mgr->get_rule_version())); + } else { + plan->set_rule_version(rule_mgr->get_rule_version()); + plan->set_is_enable_udr(tenant_config->enable_user_defined_rewrite_rules); + } + } + if (OB_SUCC(ret)) { + if (AUTO == jit_mode && // only use late compilation when jit_mode is auto + OB_FAIL(need_late_compile(plan, need_late_compilation))) { + LOG_WARN("failed to check for late compilation", K(ret)); + } else { + // set context's need_late_compile_ for upper layer to proceed + pc_ctx.sql_ctx_.need_late_compile_ = need_late_compilation; + } + } } // if schema expired, update pcv set; - if (OB_OLD_SCHEMA_VERSION == ret || (plan != NULL && plan->is_expired()) || need_late_compilation) { + if (OB_OLD_SCHEMA_VERSION == ret + || (plan != NULL && plan->is_expired()) + || need_late_compilation) { if (plan != NULL && plan->is_expired()) { LOG_INFO("the statistics of table is stale and evict plan.", K(plan->stat_)); } @@ -583,14 +609,17 @@ int ObPlanCache::construct_fast_parser_result(common::ObIAllocator &allocator, LOG_WARN("failed to construct plan cache key", K(ret)); } else if (enable_exact_mode) { (void)fp_result.pc_key_.name_.assign_ptr(raw_sql.ptr(), raw_sql.length()); - } else if (OB_FAIL(ObSqlParameterization::fast_parser(allocator, - sql_mode, - conn_coll, - raw_sql, - pc_ctx.sql_ctx_.handle_batched_multi_stmt(), - fp_result))) { - LOG_WARN("failed to fast parser", K(ret), K(sql_mode), K(pc_ctx.raw_sql_)); - } else { /*do nothing*/ } + } else { + FPContext fp_ctx(conn_coll); + fp_ctx.enable_batched_multi_stmt_ = pc_ctx.sql_ctx_.handle_batched_multi_stmt(); + fp_ctx.sql_mode_ = sql_mode; + if (OB_FAIL(ObSqlParameterization::fast_parser(allocator, + fp_ctx, + raw_sql, + fp_result))) { + LOG_WARN("failed to fast parser", K(ret), K(sql_mode), K(pc_ctx.raw_sql_)); + } else { /*do nothing*/ } + } } return ret; } @@ -1579,6 +1608,7 @@ int ObPlanCache::add_ps_plan(T *plan, ObPlanCacheCtx &pc_ctx) ret = OB_ERR_UNEXPECTED; SQL_PC_LOG(WARN, "pc_ctx.raw_sql_.ptr() is NULL, cannot add plan to plan cache by sql", K(ret)); } else { + pc_ctx.fp_result_.pc_key_.is_ps_mode_ = true; pc_ctx.fp_result_.pc_key_.name_ = pc_ctx.raw_sql_; uint64_t old_stmt_id = pc_ctx.fp_result_.pc_key_.key_id_; // the remote plan uses key_id is 0 to distinguish, so if key_id is 0, it cannot be set to OB_INVALID_ID diff --git a/src/sql/plan_cache/ob_plan_cache_struct.h b/src/sql/plan_cache/ob_plan_cache_struct.h index a7ae8c603..2c39bc129 100644 --- a/src/sql/plan_cache/ob_plan_cache_struct.h +++ b/src/sql/plan_cache/ob_plan_cache_struct.h @@ -26,6 +26,7 @@ #include "sql/plan_cache/ob_i_lib_cache_context.h" #include "sql/ob_sql_utils.h" #include "sql/plan_cache/ob_plan_cache_util.h" +#include "sql/udr/ob_udr_struct.h" namespace oceanbase { @@ -180,6 +181,11 @@ struct NotParamInfo TO_STRING_KV(K_(idx), K_(raw_text)); }; +typedef common::ObSEArray NotParamInfoList; + +// Template SQL Constant Constraints +typedef common::ObSEArray TplSqlConstCons; + struct PsNotParamInfo { int64_t idx_; @@ -198,16 +204,28 @@ public: raw_params_(&inner_alloc_), ps_params_(&inner_alloc_), cache_params_(NULL) - {} + { + reset_question_mark_ctx(); + } ObPlanCacheKey pc_key_; //plan cache key, parameterized by fast parser common::ObFixedArray raw_params_; common::ObFixedArray ps_params_; ParamStore *cache_params_; + ObQuestionMarkCtx question_mark_ctx_; void reset() { pc_key_.reset(); raw_params_.reuse(); ps_params_.reuse(); cache_params_ = NULL; + } + void reset_question_mark_ctx() + { + question_mark_ctx_.name_ = NULL; + question_mark_ctx_.count_ = 0; + question_mark_ctx_.capacity_ = 0; + question_mark_ctx_.by_ordinal_ = false; + question_mark_ctx_.by_name_ = false; + question_mark_ctx_.by_defined_name_ = false; } TO_STRING_KV(K(pc_key_), K(raw_params_), K(ps_params_), K(cache_params_)); }; @@ -294,7 +312,13 @@ struct ObPlanCacheCtx : public ObILibCacheCtx need_add_obj_stat_(true), is_inner_sql_(false), is_original_ps_mode_(false), - ab_params_(NULL) + ab_params_(NULL), + is_rewrite_sql_(false), + rule_name_(), + def_name_ctx_(NULL), + fixed_param_info_list_(allocator), + dynamic_param_info_list_(allocator), + tpl_sql_const_cons_(allocator) { fp_result_.pc_key_.is_ps_mode_ = is_ps_mode_; } @@ -329,6 +353,17 @@ struct ObPlanCacheCtx : public ObILibCacheCtx return ret; } + uint64_t get_normalized_pattern_digest() const + { + common::ObString normalized_pattern; + if (is_ps_mode_ || fp_result_.pc_key_.name_.empty()) { + normalized_pattern = raw_sql_; + } else { + normalized_pattern = fp_result_.pc_key_.name_; + } + return normalized_pattern.hash(); + } + int is_retry(bool &v) const; //是否在重试之中 int is_retry_for_dup_tbl(bool &v) const; //仅复制表原因的重试才会设置为true void set_begin_commit_stmt() { begin_commit_stmt_ = true; } @@ -337,6 +372,8 @@ struct ObPlanCacheCtx : public ObILibCacheCtx bool is_ps_execute_stage() { return is_ps_execute_stage_; } void set_is_inner_sql(bool v) { is_inner_sql_ = v; }; bool is_inner_sql() const { return is_inner_sql_; } + void set_is_rewrite_sql(bool v) { is_rewrite_sql_ = v; } + bool is_rewrite_sql() const { return is_rewrite_sql_; } TO_STRING_KV( K(is_ps_mode_), K(raw_sql_), @@ -353,6 +390,12 @@ struct ObPlanCacheCtx : public ObILibCacheCtx K(fixed_param_idx_), K(need_add_obj_stat_), K(is_inner_sql_), + K(is_rewrite_sql_), + K(rule_name_), + K(def_name_ctx_), + K(fixed_param_info_list_), + K(dynamic_param_info_list_), + K(tpl_sql_const_cons_), K(is_original_ps_mode_) ); bool is_ps_mode_; //control use which variables to do match @@ -402,6 +445,15 @@ struct ObPlanCacheCtx : public ObILibCacheCtx bool is_inner_sql_; bool is_original_ps_mode_; ParamStore *ab_params_; // arraybinding batch parameters, + + // ********** for rewrite rule ********** + bool is_rewrite_sql_; + common::ObString rule_name_; + QuestionMarkDefNameCtx *def_name_ctx_; + common::ObFixedArray fixed_param_info_list_; + common::ObFixedArray dynamic_param_info_list_; + common::ObFixedArray tpl_sql_const_cons_; + // ********** for rewrite end ********** }; struct ObPlanCacheStat diff --git a/src/sql/plan_cache/ob_plan_cache_util.h b/src/sql/plan_cache/ob_plan_cache_util.h index 308235866..8fb8fe20a 100644 --- a/src/sql/plan_cache/ob_plan_cache_util.h +++ b/src/sql/plan_cache/ob_plan_cache_util.h @@ -490,6 +490,10 @@ struct ObPlanStat common::ObString config_str_; common::ObString raw_sql_; //记录生成plan时的原始sql common::ObCollationType sql_cs_type_; + common::ObString rule_name_; + bool is_rewrite_sql_; + int64_t rule_version_; // the rule version when query rewrite generates a plan + bool enable_udr_; //******** for spm ****** //该计划是否正在演进过程中 bool is_evolution_; @@ -589,6 +593,10 @@ struct ObPlanStat outline_id_(common::OB_INVALID_ID), is_last_exec_succ_(true), sql_cs_type_(common::CS_TYPE_INVALID), + rule_name_(), + is_rewrite_sql_(false), + rule_version_(OB_INVALID_VERSION), + enable_udr_(false), is_evolution_(false), db_id_(common::OB_INVALID_ID), constructed_sql_(), @@ -660,6 +668,10 @@ struct ObPlanStat outline_id_(rhs.outline_id_), is_last_exec_succ_(rhs.is_last_exec_succ_), sql_cs_type_(rhs.sql_cs_type_), + rule_name_(), + is_rewrite_sql_(false), + rule_version_(OB_INVALID_VERSION), + enable_udr_(false), is_evolution_(rhs.is_evolution_), db_id_(rhs.db_id_), evolution_stat_(rhs.evolution_stat_), diff --git a/src/sql/plan_cache/ob_plan_cache_value.cpp b/src/sql/plan_cache/ob_plan_cache_value.cpp index d61ca5129..62c97d614 100644 --- a/src/sql/plan_cache/ob_plan_cache_value.cpp +++ b/src/sql/plan_cache/ob_plan_cache_value.cpp @@ -24,6 +24,8 @@ #include "sql/plan_cache/ob_plan_cache.h" #include "sql/plan_cache/ob_plan_set.h" #include "sql/session/ob_sql_session_info.h" +#include "sql/udr/ob_udr_mgr.h" +#include "sql/udr/ob_udr_utils.h" #include "share/ob_duplicate_scope_define.h" #include "pl/ob_pl_stmt.h" using namespace oceanbase::share::schema; @@ -154,6 +156,42 @@ ObPlanCacheValue::ObPlanCacheValue() MEMSET(sql_id_, 0, sizeof(sql_id_)); } +int ObPlanCacheValue::assign_udr_infos(ObPlanCacheCtx &pc_ctx) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(dynamic_param_list_.assign(pc_ctx.dynamic_param_info_list_))) { + LOG_WARN("fail to assign dynamic param info list", K(ret)); + } else if (OB_FAIL(tpl_sql_const_cons_.assign(pc_ctx.tpl_sql_const_cons_))) { + LOG_WARN("failed to assign tpl sql const cons", K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < tpl_sql_const_cons_.count(); ++i) { + NotParamInfoList ¬_param_list = tpl_sql_const_cons_.at(i); + std::sort(not_param_list.begin(), not_param_list.end(), + [](NotParamInfo &l, NotParamInfo &r) { return (l.idx_ < r.idx_); }); + for (int64_t j = 0; OB_SUCC(ret) && j < not_param_list.count(); ++j) { + if (OB_FAIL(ob_write_string(*pc_alloc_, + not_param_list.at(j).raw_text_, + not_param_list.at(j).raw_text_))) { + LOG_WARN("deep_copy_obj failed", K(i), K(not_param_list.at(j))); + } + } + } + } + return ret; +} + +void ObPlanCacheValue::reset_tpl_sql_const_cons() +{ + for (int64_t i = 0; i < tpl_sql_const_cons_.count(); ++i) { + NotParamInfoList ¬_param_list = tpl_sql_const_cons_.at(i); + for (int64_t j = 0; j < not_param_list.count(); ++j) { + pc_alloc_->free(not_param_list.at(j).raw_text_.ptr()); + not_param_list.at(j).raw_text_.reset(); + } + } + tpl_sql_const_cons_.reset(); +} + int ObPlanCacheValue::init(ObPCVSet *pcv_set, const ObILibCacheObject *cache_obj, ObPlanCacheCtx &pc_ctx) { int ret = OB_SUCCESS; @@ -207,6 +245,8 @@ int ObPlanCacheValue::init(ObPCVSet *pcv_set, const ObILibCacheObject *cache_obj pc_ctx.sql_ctx_.schema_guard_))) { LOG_WARN("failed to set stored schema objs", K(ret), K(plan->get_dependency_table()), K(pc_ctx.sql_ctx_.schema_guard_)); + } else if (OB_FAIL(assign_udr_infos(pc_ctx))) { + LOG_WARN("failed to assign user-defined rule infos", K(ret)); } else { //deep copy special param raw text if (pc_ctx.is_ps_mode_) { @@ -230,6 +270,10 @@ int ObPlanCacheValue::init(ObPCVSet *pcv_set, const ObILibCacheObject *cache_obj LOG_WARN("fail to deep copy param raw text", K(ret)); } } //for end + if (OB_SUCC(ret)) { + std::sort(not_param_info_.begin(), not_param_info_.end(), + [](NotParamInfo &l, NotParamInfo &r) { return (l.idx_ < r.idx_); }); + } } //deep copy constructed sql if (OB_SUCC(ret)) { @@ -430,6 +474,13 @@ int ObPlanCacheValue::choose_plan(ObPlanCacheCtx &pc_ctx, LOG_WARN("failed to resolver row params", K(ret)); } } + // cons user-defined rule param store + if (OB_SUCC(ret)) { + ParamStore param_store( (ObWrapperAllocator(pc_ctx.allocator_)) ); + if (OB_FAIL(ObUDRUtils::cons_udr_param_store(dynamic_param_list_, pc_ctx, param_store))) { + LOG_WARN("failed to construct user-defined rule param store", K(ret)); + } + } if (OB_SUCC(ret)) { ObPhysicalPlanCtx *phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx(); if (NULL != phy_ctx) { @@ -792,6 +843,54 @@ int ObPlanCacheValue::check_not_param_value(const ObFastParserResult &fp_result return ret; } +int ObPlanCacheValue::cmp_not_param_info(const NotParamInfoList &l_param_info_list, + const NotParamInfoList &r_param_info_list, + bool &is_equal) +{ + int ret = OB_SUCCESS; + is_equal = true; + if (l_param_info_list.count() != r_param_info_list.count()) { + is_equal = false; + } else { + for (int64_t i = 0; OB_SUCC(ret) && is_equal && i < l_param_info_list.count(); ++i) { + const NotParamInfo &l_param_info = l_param_info_list.at(i); + const NotParamInfo &r_param_info = r_param_info_list.at(i); + if (l_param_info.idx_ != r_param_info.idx_) { + is_equal = false; + LOG_DEBUG("compare not param info", K(l_param_info), K(r_param_info)); + } else if (0 != l_param_info.raw_text_.compare(r_param_info.raw_text_)) { + is_equal = false; + LOG_DEBUG("compare not param info", K(l_param_info), K(r_param_info)); + } + } + } + return ret; +} + +int ObPlanCacheValue::check_tpl_sql_const_cons(const ObFastParserResult &fp_result, + const TplSqlConstCons &tpl_cst_cons_list, + bool &is_same) +{ + int ret = OB_SUCCESS; + is_same = false; + bool is_match_tpl_cst_cons = false; + for (int64_t i = 0; OB_SUCC(ret) && !is_match_tpl_cst_cons && i < tpl_cst_cons_list.count(); ++i) { + const NotParamInfoList ¬_param_list = tpl_cst_cons_list.at(i); + if (OB_FAIL(check_not_param_value(fp_result, not_param_list, is_match_tpl_cst_cons))) { + LOG_WARN("failed to check not param value", K(ret)); + } else if (is_match_tpl_cst_cons + && OB_FAIL(cmp_not_param_info(not_param_list, not_param_info_, is_same))) { + LOG_WARN("failed to cmp not param info", K(ret)); + } + } + if (OB_SUCC(ret) && !is_match_tpl_cst_cons && !is_same) { + if (OB_FAIL(check_not_param_value(fp_result, not_param_info_, is_same))) { + LOG_WARN("failed to check not param value", K(ret)); + } + } + return ret; +} + int ObPlanCacheValue::get_outline_param_index(ObExecContext &exec_ctx, int64_t ¶m_idx) const { int ret = OB_SUCCESS; @@ -1091,6 +1190,7 @@ void ObPlanCacheValue::reset() neg_param_index_.reset(); param_charset_type_.reset(); sql_traits_.reset(); + reset_tpl_sql_const_cons(); if (OB_SUCCESS != outline_params_wrapper_.destroy()) { LOG_ERROR("fail to destroy ObOutlineParamWrapper"); @@ -1338,10 +1438,10 @@ int ObPlanCacheValue::match(ObPlanCacheCtx &pc_ctx, LOG_DEBUG("match", K(not_param_var_[i].idx_), K(not_param_var_[i].ps_param_), KPC(ps_param)); } } else { - if (OB_FAIL(check_not_param_value(pc_ctx.fp_result_, - not_param_info_, - is_same))) { - LOG_WARN("failed to check not param value", K(ret)); + if (OB_FAIL(check_tpl_sql_const_cons(pc_ctx.fp_result_, + tpl_sql_const_cons_, + is_same))) { + LOG_WARN("failed to check tpl sql const cons", K(ret)); } } diff --git a/src/sql/plan_cache/ob_plan_cache_value.h b/src/sql/plan_cache/ob_plan_cache_value.h index bc0955ed1..06adfe77e 100644 --- a/src/sql/plan_cache/ob_plan_cache_value.h +++ b/src/sql/plan_cache/ob_plan_cache_value.h @@ -332,6 +332,15 @@ private: static int rm_space_for_neg_num(ParseNode *param_node, ObIAllocator &allocator); + int assign_udr_infos(ObPlanCacheCtx &pc_ctx); + void reset_tpl_sql_const_cons(); + int check_tpl_sql_const_cons(const ObFastParserResult &fp_result, + const TplSqlConstCons &tpl_cst_cons_list, + bool &is_same); + int cmp_not_param_info(const NotParamInfoList &l_param_info_list, + const NotParamInfoList &r_param_info_list, + bool &is_equal); + friend class ::test::TestPlanSet_basic_Test; friend class ::test::TestPlanCacheValue_basic_Test; private: @@ -387,6 +396,35 @@ private: common::ObFixedArray stored_schema_objs_; common::ObBitSet<> must_be_positive_idx_; stmt::StmtType stmt_type_; + //*********** for user-defined rules ************** + DynamicParamInfoArray dynamic_param_list_; + /** + * call dbms_udr.create_rule('select ?, 1 from dual', 'select ? + 1, 1 from dual'); + * call dbms_udr.create_rule('select ?, 2 from dual', 'select ? + 2, 1 from dual'); + * + * template SQL: select ?, ? from dual has the following two constant constraints: + * tpl_sql_const_cons_ : {{idx:1, raw_text:"1"}, {idx:1, raw_text:"2"}} + * + * The following constraints are generated when executing a SQL that does not hit any of the rules: + * SQL: select 4, 5 from dual; + * not_param_info_ : {} + * tpl_sql_const_cons_ : {{idx:1, raw_text:"1"}, {idx:1, raw_text:"2"}} + * + * When executing a SQL that hits a rule, the following constraints are generated: + * SQL: select 4, 1 from dual; + * not_param_info_ : {idx:1, raw_text:"1"} + * tpl_sql_const_cons_ : {{idx:1, raw_text:"1"}, {idx:1, raw_text:"2"}} + * + * So the constant constraint matching rules are as follows: + * 1、First match tpl_sql_const_cons_constraint list + * 2、If it hits, use the hit rule to compare it with not_param_info_, if it is the same + * the match succeeds, otherwise it fails + * 3、If there is no hit, match not_param_info_. if the match is successful + * the result is successful, otherwise it fails + */ + TplSqlConstCons tpl_sql_const_cons_; + //*********** end user-defined rules ************** + DISALLOW_COPY_AND_ASSIGN(ObPlanCacheValue); }; diff --git a/src/sql/plan_cache/ob_prepare_stmt_struct.cpp b/src/sql/plan_cache/ob_prepare_stmt_struct.cpp index 3f7f68acc..430236eaa 100644 --- a/src/sql/plan_cache/ob_prepare_stmt_struct.cpp +++ b/src/sql/plan_cache/ob_prepare_stmt_struct.cpp @@ -292,6 +292,7 @@ ObPsStmtInfo::ObPsStmtInfo(ObIAllocator *inner_allocator) num_of_returning_into_(-1), no_param_sql_(), is_sensitive_sql_(false), + raw_sql_(), raw_params_(inner_allocator), raw_params_idx_(inner_allocator) @@ -321,6 +322,7 @@ ObPsStmtInfo::ObPsStmtInfo(ObIAllocator *inner_allocator, num_of_returning_into_(-1), no_param_sql_(), is_sensitive_sql_(false), + raw_sql_(), raw_params_(inner_allocator), raw_params_idx_(inner_allocator) { @@ -343,6 +345,18 @@ int ObPsStmtInfo::assign_no_param_sql(const common::ObString &no_param_sql) return ret; } +int ObPsStmtInfo::assign_raw_sql(const common::ObString &raw_sql) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(allocator_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("allocator is invalid", K(ret)); + } else if (OB_FAIL(ObPsSqlUtils::deep_copy_str(*allocator_, raw_sql, raw_sql_))) { + LOG_WARN("deep copy raw sql failed", K(raw_sql), K(ret)); + } + return ret; +} + int ObPsStmtInfo::add_fixed_raw_param(const ObPCParam &node) { int ret = OB_SUCCESS; @@ -473,6 +487,9 @@ int ObPsStmtInfo::deep_copy(const ObPsStmtInfo &other) } else if (OB_FAIL(ObPsSqlUtils::deep_copy_str(*allocator_, other.get_no_param_sql(), no_param_sql_))) { LOG_WARN("deep copy str failed", K(other), K(ret)); + } else if (OB_FAIL(ObPsSqlUtils::deep_copy_str(*allocator_, other.get_raw_sql(), + raw_sql_))) { + LOG_WARN("deep copy str failed", K(other), K(ret)); } else if (OB_FAIL(deep_copy_fixed_raw_params(other.raw_params_idx_, other.raw_params_))) { LOG_WARN("deep copy fixed raw params failed", K(other), K(ret)); } else if (OB_FAIL(ps_sql_meta_.deep_copy(other.get_ps_sql_meta()))) { @@ -507,6 +524,7 @@ int ObPsStmtInfo::get_convert_size(int64_t &cv_size) const int64_t convert_size = sizeof(ObPsStmtInfo); convert_size += ps_sql_.length() + 1; convert_size += no_param_sql_.length() + 1; + convert_size += raw_sql_.length() + 1; int64_t meta_convert_size = 0; int64_t raw_params_size = 0; if (OB_FAIL(ps_sql_meta_.get_convert_size(meta_convert_size))) { diff --git a/src/sql/plan_cache/ob_prepare_stmt_struct.h b/src/sql/plan_cache/ob_prepare_stmt_struct.h index e5a6b4d7c..2e7c9b712 100644 --- a/src/sql/plan_cache/ob_prepare_stmt_struct.h +++ b/src/sql/plan_cache/ob_prepare_stmt_struct.h @@ -166,6 +166,7 @@ public: inline int32_t get_num_of_returning_into() const { return num_of_returning_into_; } inline void set_is_sensitive_sql(const bool is_sensitive_sql) { is_sensitive_sql_ = is_sensitive_sql; } inline bool get_is_sensitive_sql() const { return is_sensitive_sql_; } + inline const common::ObString &get_raw_sql() const { return raw_sql_; } bool is_valid() const; bool check_erase_inc_ref_count(); @@ -174,6 +175,7 @@ public: int add_param_field(const common::ObField ¶m); int add_column_field(const common::ObField &column); int get_convert_size(int64_t &cv_size) const; + int assign_raw_sql(const common::ObString &raw_sql); int assign_no_param_sql(const common::ObString &no_param_sql); int assign_fixed_raw_params(const common::ObIArray ¶m_idxs, const common::ObIArray &raw_params); @@ -248,6 +250,7 @@ private: int32_t num_of_returning_into_; common::ObString no_param_sql_; bool is_sensitive_sql_; + common::ObString raw_sql_; // raw_params_ records constants other than question mark in raw prepare sql // raw_params_idx_ records the offset of the constants in raw_params_ in param_store // E.g: prepare stmt from 'select 3 + ? + 2 from dual'; @@ -400,6 +403,41 @@ private: DISALLOW_COPY_AND_ASSIGN(ObPsStmtInfoGuard); }; +struct PsCacheInfoCtx +{ + PsCacheInfoCtx() + : param_cnt_(0), + num_of_returning_into_(-1), + is_inner_sql_(false), + is_sensitive_sql_(false), + normalized_sql_(), + raw_sql_(), + no_param_sql_(), + raw_params_(NULL), + fixed_param_idx_(NULL), + stmt_type_(stmt::T_NONE) {} + + + TO_STRING_KV(K_(param_cnt), + K_(num_of_returning_into), + K_(is_inner_sql), + K_(is_sensitive_sql), + K_(normalized_sql), + K_(raw_sql), + K_(no_param_sql), + K_(stmt_type)); + + int64_t param_cnt_; + int32_t num_of_returning_into_; + bool is_inner_sql_; + bool is_sensitive_sql_; + common::ObString normalized_sql_; + common::ObString raw_sql_; + common::ObString no_param_sql_; + common::ObIArray *raw_params_; + common::ObIArray *fixed_param_idx_; + stmt::StmtType stmt_type_; +}; } //end of namespace sql } //end of namespace oceanbase diff --git a/src/sql/plan_cache/ob_ps_cache.cpp b/src/sql/plan_cache/ob_ps_cache.cpp index ae60877a4..cddf2a8b9 100644 --- a/src/sql/plan_cache/ob_ps_cache.cpp +++ b/src/sql/plan_cache/ob_ps_cache.cpp @@ -15,6 +15,7 @@ #include "sql/plan_cache/ob_ps_sql_utils.h" #include "sql/plan_cache/ob_ps_cache_callback.h" #include "sql/resolver/cmd/ob_call_procedure_stmt.h" +#include "sql/udr/ob_udr_mgr.h" #include "share/schema/ob_schema_getter_guard.h" namespace oceanbase @@ -381,25 +382,22 @@ int ObPsCache::ref_stmt_item(const uint64_t db_id, return ret; } -int ObPsCache::get_or_add_stmt_info(const ObResultSet &result, - const ObString &origin_sql, - const ObString &no_param_sql, - const ObIArray &raw_params, - const common::ObIArray &raw_params_idx, - int64_t param_cnt, +int ObPsCache::get_or_add_stmt_info(const PsCacheInfoCtx &info_ctx, + const ObResultSet &result, ObSchemaGetterGuard &schema_guard, - stmt::StmtType stmt_type, ObPsStmtItem *ps_item, - ObPsStmtInfo *&ref_ps_info, - int32_t returning_into_parm_num) + ObPsStmtInfo *&ref_ps_info) { int ret = OB_SUCCESS; if (OB_ISNULL(ps_item)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(ps_item)); - } else if (OB_ISNULL(origin_sql.ptr())) { + } else if (OB_ISNULL(info_ctx.normalized_sql_.ptr())) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("origin_sql is null", K(ret), K(ps_item)); + LOG_WARN("normalized sql is null", K(ret), K(ps_item)); + } else if (OB_ISNULL(info_ctx.raw_params_) || OB_ISNULL(info_ctx.fixed_param_idx_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("param is null", K(ret)); } else if (OB_FAIL(ref_stmt_info(ps_item->get_ps_stmt_id(), ref_ps_info))) { if (OB_HASH_NOT_EXIST == ret) { ret = OB_SUCCESS; @@ -407,21 +405,25 @@ int ObPsCache::get_or_add_stmt_info(const ObResultSet &result, ObArenaAllocator allocator; ObPsStmtInfo tmp_stmt_info(&allocator); tmp_stmt_info.assign_sql_key(*ps_item); - tmp_stmt_info.set_stmt_type(stmt_type); + tmp_stmt_info.set_stmt_type(info_ctx.stmt_type_); tmp_stmt_info.set_ps_item(ps_item); - // calc check_sum with origin_sql - uint64_t ps_stmt_checksum = ob_crc64(origin_sql.ptr(), - origin_sql.length()); // actual is crc32 + // calc check_sum with normalized sql + uint64_t ps_stmt_checksum = ob_crc64(info_ctx.normalized_sql_.ptr(), + info_ctx.normalized_sql_.length()); // actual is crc32 tmp_stmt_info.set_ps_stmt_checksum(ps_stmt_checksum); if (OB_FAIL(schema_guard.get_schema_version(tenant_id_, tenant_version))) { LOG_WARN("fail to get tenant version", K(ret), K(tenant_id_)); } else if (FALSE_IT(tmp_stmt_info.set_tenant_version(tenant_version))) { // do nothing - } else if (OB_FAIL(tmp_stmt_info.assign_no_param_sql(no_param_sql))) { - LOG_WARN("fail to assign no param sql", K(ret), K(no_param_sql)); - } else if (OB_FAIL(tmp_stmt_info.assign_fixed_raw_params(raw_params_idx, raw_params))) { - LOG_WARN("fail to assign raw params failed", K(raw_params_idx), K(ret)); - } else if (OB_FAIL(fill_ps_stmt_info(result, param_cnt, tmp_stmt_info, returning_into_parm_num))) { + } else if (OB_FAIL(tmp_stmt_info.assign_no_param_sql(info_ctx.no_param_sql_))) { + LOG_WARN("fail to assign no param sql", K(ret), K(info_ctx.no_param_sql_)); + } else if (OB_FAIL(tmp_stmt_info.assign_raw_sql(info_ctx.raw_sql_))) { + LOG_WARN("fail to assign rule name", K(ret), K(info_ctx.raw_sql_)); + } else if (OB_FAIL(tmp_stmt_info.assign_fixed_raw_params(*info_ctx.fixed_param_idx_, + *info_ctx.raw_params_))) { + LOG_WARN("fail to assign raw params failed", KPC(info_ctx.fixed_param_idx_), K(ret)); + } else if (OB_FAIL(fill_ps_stmt_info(result, info_ctx.param_cnt_, + tmp_stmt_info, info_ctx.num_of_returning_into_))) { LOG_WARN("fill ps stmt info failed", K(ret)); } else if (OB_FAIL(add_stmt_info(*ps_item, tmp_stmt_info, ref_ps_info))) { LOG_WARN("add stmt info failed", K(ret), K(*ps_item), K(tmp_stmt_info)); diff --git a/src/sql/plan_cache/ob_ps_cache.h b/src/sql/plan_cache/ob_ps_cache.h index 7ba1937ae..afd9b3657 100644 --- a/src/sql/plan_cache/ob_ps_cache.h +++ b/src/sql/plan_cache/ob_ps_cache.h @@ -84,17 +84,11 @@ public: int get_or_add_stmt_item(const uint64_t db_id, const common::ObString &ps_sql, ObPsStmtItem *&ps_item_value); - int get_or_add_stmt_info(const ObResultSet &result, - const ObString &origin_sql, - const common::ObString &no_param_sql, - const ObIArray &raw_params, - const common::ObIArray &raw_params_idx, - int64_t param_cnt, + int get_or_add_stmt_info(const PsCacheInfoCtx &info_ctx, + const ObResultSet &result, ObSchemaGetterGuard &schema_guard, - stmt::StmtType stmt_type, ObPsStmtItem *ps_item, - ObPsStmtInfo *&ref_ps_info, - int32_t returning_into_parm_num); + ObPsStmtInfo *&ref_ps_info); int cache_evict(); int cache_evict_all_ps(); diff --git a/src/sql/plan_cache/ob_sql_parameterization.cpp b/src/sql/plan_cache/ob_sql_parameterization.cpp index 8e1e93a50..495562483 100644 --- a/src/sql/plan_cache/ob_sql_parameterization.cpp +++ b/src/sql/plan_cache/ob_sql_parameterization.cpp @@ -63,6 +63,7 @@ struct TransformTreeCtx SQL_EXECUTION_MODE mode_; bool is_project_list_scope_; int64_t assign_father_level_; + const ObIArray *udr_fixed_params_; bool ignore_scale_check_; TransformTreeCtx(); }; @@ -123,6 +124,7 @@ TransformTreeCtx::TransformTreeCtx() : mode_(INVALID_MODE), is_project_list_scope_(false), assign_father_level_(ObSqlParameterization::NO_VALUES), + udr_fixed_params_(NULL), ignore_scale_check_(false) { } @@ -138,7 +140,8 @@ int ObSqlParameterization::transform_syntax_tree(ObIAllocator &allocator, SelectItemParamInfoArray *select_item_param_infos, ObMaxConcurrentParam::FixParamStore &fixed_param_store, bool is_transform_outline, - SQL_EXECUTION_MODE execution_mode) + SQL_EXECUTION_MODE execution_mode, + const ObIArray *udr_fixed_params) { int ret = OB_SUCCESS; ObCollationType collation_connection = CS_TYPE_INVALID; @@ -173,6 +176,7 @@ int ObSqlParameterization::transform_syntax_tree(ObIAllocator &allocator, ctx.paramlized_questionmask_count_ = 0;//used for outline sql限流, ctx.is_transform_outline_ = is_transform_outline;//used for outline sql限流 ctx.raw_params_ = raw_params; + ctx.udr_fixed_params_ = udr_fixed_params; ctx.is_project_list_scope_ = false; ctx.mode_ = execution_mode; ctx.assign_father_level_ = NO_VALUES; @@ -255,6 +259,23 @@ int ObSqlParameterization::is_fast_parse_const(TransformTreeCtx &ctx) return ret; } +bool ObSqlParameterization::is_udr_not_param(TransformTreeCtx &ctx) +{ + bool b_ret = false; + if (OB_ISNULL(ctx.tree_) || (NULL == ctx.udr_fixed_params_)) { + b_ret = false; + } else { + for (int64_t i = 0; !b_ret && i < ctx.udr_fixed_params_->count(); ++i) { + const FixedParamValue &fixed_param = ctx.udr_fixed_params_->at(i); + if (fixed_param.idx_ == ctx.tree_->raw_param_idx_) { + b_ret = true; + break; + } + } + } + return b_ret; +} + //判断该node是否为不能参数化的node bool ObSqlParameterization::is_node_not_param(TransformTreeCtx &ctx) { @@ -598,9 +619,11 @@ int ObSqlParameterization::transform_tree(TransformTreeCtx &ctx, //do nothing } else if (!is_execute_mode(ctx.mode_) && OB_FAIL(ctx.params_->push_back(value))) { SQL_PC_LOG(WARN, "fail to push into params", K(ret)); + } else if (is_udr_not_param(ctx) && OB_FAIL(add_not_param_flag(ctx.tree_, *ctx.sql_info_))) { + SQL_PC_LOG(WARN, "fail to add not param flag", K(ret)); } } - } else if (add_not_param_flag(ctx.tree_, *ctx.sql_info_)) { //not param + } else if (OB_FAIL(add_not_param_flag(ctx.tree_, *ctx.sql_info_))) { //not param SQL_PC_LOG(WARN, "fail to add not param flag", K(ret)); } } //if is_fast_parse_const end @@ -906,11 +929,14 @@ int ObSqlParameterization::parameterize_syntax_tree(common::ObIAllocator &alloca // if so, faster parser is needed // otherwise, fast parser has been done before pc_ctx.fp_result_.reset(); + FPContext fp_ctx(cs_type); + fp_ctx.enable_batched_multi_stmt_ = pc_ctx.sql_ctx_.handle_batched_multi_stmt(); + fp_ctx.sql_mode_ = session->get_sql_mode(); + fp_ctx.is_udr_mode_ = pc_ctx.is_rewrite_sql_; + fp_ctx.def_name_ctx_ = pc_ctx.def_name_ctx_; if (OB_FAIL(fast_parser(allocator, - session->get_sql_mode(), - cs_type, + fp_ctx, pc_ctx.raw_sql_, - pc_ctx.sql_ctx_.handle_batched_multi_stmt(), pc_ctx.fp_result_))) { SQL_PC_LOG(WARN, "fail to fast parser", K(ret)); } @@ -925,14 +951,15 @@ int ObSqlParameterization::parameterize_syntax_tree(common::ObIAllocator &alloca LOG_WARN("failed to reserve array", K(ret)); } else if (OB_FAIL(transform_syntax_tree(allocator, *session, - &pc_ctx.fp_result_.raw_params_, + is_execute_mode(mode) ? NULL : &pc_ctx.fp_result_.raw_params_, tree, sql_info, params, is_prepare_mode(mode) ? NULL : &pc_ctx.select_item_param_infos_, fix_param_store, is_transform_outline, - mode))) { + mode, + &pc_ctx.fixed_param_info_list_))) { if (OB_NOT_SUPPORTED != ret) { SQL_PC_LOG(WARN, "fail to normal parameterized parser tree", K(ret)); } @@ -1224,10 +1251,8 @@ bool ObSqlParameterization::need_fast_parser(const ObString &sql) } int ObSqlParameterization::fast_parser(ObIAllocator &allocator, - ObSQLMode sql_mode, - ObCollationType connection_collation, + const FPContext &fp_ctx, const ObString &sql, - const bool enable_batched_multi_stmt, ObFastParserResult &fp_result) { //UNUSED(sql_mode); @@ -1242,8 +1267,8 @@ int ObSqlParameterization::fast_parser(ObIAllocator &allocator, || (ObParser::is_pl_stmt(sql, nullptr, &is_call_procedure) && !is_call_procedure))) { (void)fp_result.pc_key_.name_.assign_ptr(sql.ptr(), sql.length()); } else if (GCONF._ob_enable_fast_parser) { - if (OB_FAIL(ObFastParser::parse(sql, enable_batched_multi_stmt, no_param_sql_ptr, - no_param_sql_len, p_list, param_num, connection_collation, allocator, sql_mode))) { + if (OB_FAIL(ObFastParser::parse(sql, fp_ctx, allocator, no_param_sql_ptr, + no_param_sql_len, p_list, param_num, fp_result.question_mark_ctx_))) { LOG_WARN("fast parse error", K(param_num), K(ObString(no_param_sql_len, no_param_sql_ptr)), K(sql)); } @@ -1252,6 +1277,7 @@ int ObSqlParameterization::fast_parser(ObIAllocator &allocator, if (param_num > 0) { ObPCParam *pc_param = NULL; char *ptr = (char *)allocator.alloc(param_num * sizeof(ObPCParam)); + fp_result.raw_params_.reset(); fp_result.raw_params_.set_allocator(&allocator); fp_result.raw_params_.set_capacity(param_num); if (OB_ISNULL(ptr)) { @@ -1271,9 +1297,9 @@ int ObSqlParameterization::fast_parser(ObIAllocator &allocator, } else { /*do nothing*/} } } else { - ObParser parser(allocator, sql_mode, connection_collation); + ObParser parser(allocator, fp_ctx.sql_mode_, fp_ctx.conn_coll_); SMART_VAR(ParseResult, parse_result) { - if (OB_FAIL(parser.parse(sql, parse_result, FP_MODE, enable_batched_multi_stmt))) { + if (OB_FAIL(parser.parse(sql, parse_result, FP_MODE, fp_ctx.enable_batched_multi_stmt_))) { SQL_PC_LOG(WARN, "fail to fast parser", K(sql), K(ret)); } else { (void)fp_result.pc_key_.name_.assign_ptr(parse_result.no_param_sql_, parse_result.no_param_sql_len_); diff --git a/src/sql/plan_cache/ob_sql_parameterization.h b/src/sql/plan_cache/ob_sql_parameterization.h index 963f23630..d2c5f030a 100644 --- a/src/sql/plan_cache/ob_sql_parameterization.h +++ b/src/sql/plan_cache/ob_sql_parameterization.h @@ -21,6 +21,7 @@ #include "lib/utility/ob_print_utils.h" #include "common/object/ob_object.h" #include "sql/parser/ob_parser.h" +#include "sql/parser/ob_fast_parser.h" #include "sql/session/ob_sql_session_info.h" #include "sql/plan_cache/ob_id_manager_allocator.h" #include "sql/plan_cache/ob_plan_cache_util.h" @@ -115,10 +116,8 @@ public: ObSqlParameterization() {} virtual ~ObSqlParameterization() {} static int fast_parser(common::ObIAllocator &allocator, - ObSQLMode sql_mode, - common::ObCollationType connection_collation, + const FPContext &fp_ctx, const common::ObString &sql, - const bool enable_batched_multi_stmt, ObFastParserResult &fp_result); static int transform_syntax_tree(common::ObIAllocator &allocator, @@ -130,7 +129,8 @@ public: SelectItemParamInfoArray *select_item_param_infos, share::schema::ObMaxConcurrentParam::FixParamStore &fixed_param_store, bool is_transform_outline, - SQL_EXECUTION_MODE execution_mode = INVALID_MODE); + SQL_EXECUTION_MODE execution_mode = INVALID_MODE, + const ObIArray *udr_fixed_params = NULL); static int raw_fast_parameterize_sql(common::ObIAllocator &allocator, const ObSQLSessionInfo &session, const common::ObString &sql, @@ -170,6 +170,7 @@ private: static int is_fast_parse_const(TransformTreeCtx &ctx); static bool is_node_not_param(TransformTreeCtx &ctx); + static bool is_udr_not_param(TransformTreeCtx &ctx); static int transform_tree(TransformTreeCtx &ctx, const ObSQLSessionInfo &session_info); static int add_param_flag(const ParseNode *node, SqlInfo &sql_info); static int add_not_param_flag(const ParseNode *node, SqlInfo &sql_info); diff --git a/src/sql/udr/ob_udr_analyzer.cpp b/src/sql/udr/ob_udr_analyzer.cpp new file mode 100644 index 000000000..a2b297e0f --- /dev/null +++ b/src/sql/udr/ob_udr_analyzer.cpp @@ -0,0 +1,439 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#define USING_LOG_PREFIX SQL_QRR +#include "lib/utility/ob_print_utils.h" +#include "sql/resolver/ob_resolver_utils.h" +#include "common/ob_smart_call.h" +#include "sql/plan_cache/ob_sql_parameterization.h" +#include "sql/udr/ob_udr_analyzer.h" + +namespace oceanbase +{ +namespace sql +{ + +bool ObUDRAnalyzer::check_is_allow_stmt_type(stmt::StmtType stmt_type) +{ + return (stmt_type == stmt::T_SELECT + || stmt_type == stmt::T_INSERT + || stmt_type == stmt::T_REPLACE + || stmt_type == stmt::T_MERGE + || stmt_type == stmt::T_DELETE + || stmt_type == stmt::T_UPDATE + || stmt_type == stmt::T_VARIABLE_SET); +} + +int ObUDRAnalyzer::parse_and_resolve_stmt_type(const common::ObString &sql, + ParseResult &parse_result, + stmt::StmtType &stmt_type) +{ + int ret = OB_SUCCESS; + stmt_type = stmt::T_NONE; + ObParser parser(allocator_, sql_mode_, connection_collation_); + if (OB_FAIL(multiple_query_check(sql))) { + LOG_WARN("failed to check multiple query check", K(ret)); + } else if (OB_FAIL(parser.parse(sql, parse_result))) { + LOG_WARN("generate syntax tree failed", K(sql), K(ret)); + } else if (OB_FAIL(ObResolverUtils::resolve_stmt_type(parse_result, stmt_type))) { + LOG_WARN("failed to resolve stmt type", K(ret)); + } + return ret; +} + +int ObUDRAnalyzer::find_leftest_const_node(ParseNode &cur_node, ParseNode *&const_node) +{ + int ret = OB_SUCCESS; + if (T_QUESTIONMARK == cur_node.type_) { + const_node = &cur_node; + } else if (T_OP_MUL == cur_node.type_ || T_OP_DIV == cur_node.type_ || T_OP_MOD == cur_node.type_) { + // syntax tree of '1 - (?-3)/4' is + // - + // / \ + // 1 div + // / \ + // - 4 + // / \ + // 2 3 + // at this time '-' cannot be combined with 2, that is, the syntax tree cannot be converted + if (OB_ISNULL(cur_node.children_) || 2 != cur_node.num_child_ + || OB_ISNULL(cur_node.children_[0]) || OB_ISNULL(cur_node.children_[1])) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument"); + } else if (OB_FAIL(SMART_CALL(find_leftest_const_node(*cur_node.children_[0], const_node)))) { + LOG_WARN("failed to find leftest const node", K(ret)); + } else { + // do nothing + } + } + return ret; +} + +int ObUDRAnalyzer::check_transform_minus_op(ParseNode *tree) +{ + int ret = OB_SUCCESS; + if (T_OP_MINUS != tree->type_) { + // do nothing + } else if (2 != tree->num_child_ + || OB_ISNULL(tree->children_) + || OB_ISNULL(tree->children_[1])) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid minus tree", K(ret)); + } else if (1 == tree->children_[1]->is_assigned_from_child_) { + // select 1 - (2) from dual; + // select 1 - (2/3/4) from dual; + // do nothing + } else if (T_QUESTIONMARK == tree->children_[1]->type_) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "template statement type"); + LOG_WARN("template statement type not supported", K(ret)); + } else if (T_OP_MUL == tree->children_[1]->type_ + || T_OP_DIV == tree->children_[1]->type_ + || (lib::is_mysql_mode() && T_OP_MOD == tree->children_[1]->type_)) { + // '0 - 2 * 3' should be transformed to '0 + (-2) * 3' + // '0 - 2 / 3' should be transformed to '0 + (-2) / 3' + // '0 - 4 mod 3' should be transformed to '0 + (-4 mod 3)' + // '0 - 2/3/4' => '0 + (-2/3/4)' + // notify that, syntax tree of '0 - 2/3/4' is + // - + // / \ + // 0 div + // / \ + // div 4 + // / \ + // 2 3 + // so, we need to find the leftest leave node and change its value and str + // same for '%','*', mod + ParseNode *const_node = NULL; + ParseNode *op_node = tree->children_[1]; + if (OB_FAIL(find_leftest_const_node(*op_node, const_node))) { + LOG_WARN("failed to find leftest const node", K(ret)); + } else if (OB_ISNULL(const_node)) { + // do nothing + } else { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "template statement type"); + LOG_WARN("template statement type not supported", K(ret)); + } + } + return ret; +} + +int ObUDRAnalyzer::traverse_and_check(ParseNode *tree) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(check_stack_overflow())) { + LOG_WARN("failed to check stack overflow", K(ret)); + } else if (OB_ISNULL(tree)) { + // do nothing + } else if (OB_FAIL(check_transform_minus_op(tree))) { + LOG_WARN("failed to check transform minus op", K(ret)); + } else { + for (int32_t i = 0; OB_SUCC(ret) && i < tree->num_child_; ++i) { + if (OB_ISNULL(tree->children_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument"); + } else { + ret = SMART_CALL(traverse_and_check(tree->children_[i])); + } + } + } + return ret; +} + +int ObUDRAnalyzer::multiple_query_check(const ObString &sql) +{ + int ret = OB_SUCCESS; + ObSEArray queries; + ObParser parser(allocator_, sql_mode_, connection_collation_); + ObMPParseStat parse_stat; + if (OB_FAIL(parser.split_multiple_stmt(sql, queries, parse_stat))) { + LOG_WARN("failed to split multiple stmt", K(ret), K(sql)); + } else if (OB_UNLIKELY(queries.count() <= 0)) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "empty sql"); + LOG_WARN("empty sql not supported", K(sql)); + } else if (OB_UNLIKELY(queries.count() > 1)) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "multi-statement query"); + LOG_WARN("multi-statement query not supported", K(ret), K(queries.count())); + } + return ret; +} + +int ObUDRAnalyzer::parse_and_check(const common::ObString &pattern, + const common::ObString &replacement) +{ + int ret = OB_SUCCESS; + int64_t l_param_cnt = 0; + int64_t r_param_cnt = 0; + ParseResult l_parse_result; + ParseResult r_parse_result; + stmt::StmtType l_stmt_type = stmt::T_NONE; + stmt::StmtType r_stmt_type = stmt::T_NONE; + if (pattern.empty() || replacement.empty()) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "empty sql"); + LOG_WARN("empty sql not supported", K(pattern), K(replacement)); + } else if (OB_FAIL(parse_and_resolve_stmt_type(pattern, l_parse_result, l_stmt_type))) { + LOG_WARN("failed to parser and resolve stmt type", K(ret)); + } else if (OB_FAIL(parse_and_resolve_stmt_type(replacement, r_parse_result, r_stmt_type))) { + LOG_WARN("failed to parser and resolve stmt type", K(ret)); + } else if (!check_is_allow_stmt_type(l_stmt_type) || !check_is_allow_stmt_type(r_stmt_type)) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "SQL types in user defined rule"); + LOG_WARN("SQL type in user defined rule not supported", K(l_stmt_type), K(r_stmt_type), K(ret)); + } else if (OB_FAIL(traverse_and_check(l_parse_result.result_tree_))) { + LOG_WARN("failed to traverse and check", K(ret)); + } else if (FALSE_IT(l_param_cnt = l_parse_result.question_mark_ctx_.count_)) { + } else if (FALSE_IT(r_param_cnt = r_parse_result.question_mark_ctx_.count_)) { + } else if (l_param_cnt != r_param_cnt) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "the number of question marks is not equal"); + LOG_WARN("the number of question marks is not equal", K(ret), K(l_param_cnt), K(r_param_cnt)); + } else if (l_parse_result.question_mark_ctx_.count_ > 0 + || r_parse_result.question_mark_ctx_.count_ > 0) { + const ObQuestionMarkCtx &l_question_mark_ctx = l_parse_result.question_mark_ctx_; + const ObQuestionMarkCtx &r_question_mark_ctx = r_parse_result.question_mark_ctx_; + if ((!l_question_mark_ctx.by_name_ && !l_question_mark_ctx.by_ordinal_) + || (!r_question_mark_ctx.by_name_ && !r_question_mark_ctx.by_ordinal_)) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "binding methods"); + LOG_WARN("binding methods not supported"); + } else if (l_question_mark_ctx.by_ordinal_ != r_question_mark_ctx.by_ordinal_ + || l_question_mark_ctx.by_name_ != r_question_mark_ctx.by_name_) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "different binding methods"); + LOG_WARN("different binding methods not supported"); + } else if (l_question_mark_ctx.count_ != r_question_mark_ctx.count_) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "question marks are not equal"); + LOG_WARN("question marks are not equal not supported"); + } else if (l_question_mark_ctx.count_ > 0 && l_question_mark_ctx.by_name_) { + std::sort(l_question_mark_ctx.name_, l_question_mark_ctx.name_ + l_question_mark_ctx.count_, + [](char const *lhs, char const *rhs) { + return STRCASECMP(lhs, rhs) < 0; + }); + std::sort(r_question_mark_ctx.name_, r_question_mark_ctx.name_ + r_question_mark_ctx.count_, + [](char const *lhs, char const *rhs) { + return STRCASECMP(lhs, rhs) < 0; + }); + for (int64_t i = 0; OB_SUCC(ret) && i < l_question_mark_ctx.count_; ++i) { + if (0 != STRCASECMP(l_question_mark_ctx.name_[i], r_question_mark_ctx.name_[i])) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "question marks are defined with different names"); + LOG_WARN("question marks are defined with different names not supported"); + } + } + } + } + return ret; +} + +int ObUDRAnalyzer::add_dynamic_param_info( + const int64_t raw_param_idx, + const int64_t question_mark_idx, + DynamicParamInfoArray &dynamic_param_infos) +{ + int ret = OB_SUCCESS; + DynamicParamInfo param_info; + param_info.raw_param_idx_ = raw_param_idx; + param_info.question_mark_idx_ = question_mark_idx; + if (OB_FAIL(dynamic_param_infos.push_back(param_info))) { + LOG_WARN("failed to add dynamic param info", K(ret), K(param_info)); + } else { + LOG_DEBUG("succ to add dynamic param info", K(param_info)); + } + return ret; +} + +int ObUDRAnalyzer::add_fixed_param_value( + const int64_t raw_param_idx, + const ParseNode *raw_param, + FixedParamValueArray &fixed_param_infos) +{ + int ret = OB_SUCCESS; + FixedParamValue param_value; + param_value.idx_ = raw_param_idx; + param_value.raw_text_.assign(const_cast(raw_param->raw_text_), raw_param->text_len_); + if (OB_FAIL(fixed_param_infos.push_back(param_value))) { + LOG_WARN("failed to add fixed param info", K(ret), K(param_value)); + } else { + LOG_DEBUG("succ to add fixed param value", K(param_value)); + } + return ret; +} + +int ObUDRAnalyzer::cons_raw_param_infos( + const ObIArray &raw_params, + FixedParamValueArray &fixed_param_infos, + DynamicParamInfoArray &dynamic_param_infos) +{ + int ret = OB_SUCCESS; + ParseNode *raw_param = NULL; + ObPCParam *pc_param = NULL; + common::ObBitSet<> mark_pos; + for (int64_t i = 0; OB_SUCC(ret) && i < raw_params.count(); ++i) { + if (OB_ISNULL(pc_param = raw_params.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(pc_param)); + } else if (NULL == (raw_param = pc_param->node_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(raw_param)); + } else if (T_QUESTIONMARK != raw_param->type_) { + if (OB_FAIL(add_fixed_param_value(i, raw_param, fixed_param_infos))) { + LOG_WARN("failed to add fixed param value", K(ret)); + } + } else if (mark_pos.has_member(raw_param->value_)) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "pattern contains the same question mark"); + LOG_WARN("pattern contains the same question mark not supported", K(ret)); + } else if (OB_FAIL(add_dynamic_param_info(i, raw_param->value_, dynamic_param_infos))) { + LOG_WARN("failed to add dynamic param info", K(ret)); + } else if (OB_FAIL(mark_pos.add_member(raw_param->value_))) { + LOG_WARN("failed to add question_mark_idx", K(ret)); + } + } + if (OB_SUCC(ret)) { + std::sort(dynamic_param_infos.begin(), dynamic_param_infos.end(), + [](DynamicParamInfo &l, DynamicParamInfo &r) + { + return (l.question_mark_idx_ < r.question_mark_idx_); + }); + } + return ret; +} + +int ObUDRAnalyzer::parse_sql_to_gen_match_param_infos( + const common::ObString &pattern, + common::ObString &normalized_pattern, + ObIArray &raw_params) +{ + int ret = OB_SUCCESS; + ObFastParserResult fp_result; + FPContext fp_ctx(connection_collation_); + fp_ctx.sql_mode_ = sql_mode_; + fp_ctx.is_udr_mode_ = true; + if (pattern.empty()) { + LOG_WARN("empty sql", K(pattern)); + } else if (OB_FAIL(ObSqlParameterization::fast_parser(allocator_, + fp_ctx, + pattern, + fp_result))) { + LOG_WARN("failed to fast parser", K(ret), K(sql_mode_), K(pattern)); + } else if (FALSE_IT(normalized_pattern = fp_result.pc_key_.name_)) { + } else if (OB_FAIL(raw_params.assign(fp_result.raw_params_))) { + LOG_WARN("failed to assign raw params", K(ret)); + } + return ret; +} + +int ObUDRAnalyzer::parse_pattern_to_gen_param_infos( + const common::ObString &pattern, + common::ObString &normalized_pattern, + ObIArray &raw_params, + ObQuestionMarkCtx &ctx) +{ + int ret = OB_SUCCESS; + ObFastParserResult fp_result; + FPContext fp_ctx(connection_collation_); + fp_ctx.sql_mode_ = sql_mode_; + fp_ctx.is_udr_mode_ = true; + if (pattern.empty()) { + ret = OB_NOT_SUPPORTED; + LOG_USER_ERROR(OB_NOT_SUPPORTED, "empty pattern"); + LOG_WARN("empty pattern not supported", K(ret)); + } else if (OB_FAIL(ObSqlParameterization::fast_parser(allocator_, + fp_ctx, + pattern, + fp_result))) { + LOG_WARN("failed to fast parser", K(ret), K(sql_mode_), K(pattern)); + } else if (FALSE_IT(normalized_pattern = fp_result.pc_key_.name_)) { + } else if (OB_FAIL(raw_params.assign(fp_result.raw_params_))) { + LOG_WARN("failed to assign raw params", K(ret)); + } else { + ctx = fp_result.question_mark_ctx_; + } + return ret; +} + +template +int ObUDRAnalyzer::serialize_to_hex(const T &infos, common::ObString &infos_str) +{ + int ret = OB_SUCCESS; + char *serialize_buf = nullptr; + char *hex_buf = nullptr; + int64_t serialize_pos = 0; + int64_t hex_pos = 0; + const int64_t serialize_size = infos.get_serialize_size(); + const int64_t hex_size = 2 * serialize_size; + if (OB_ISNULL(serialize_buf = static_cast(allocator_.alloc(serialize_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate", K(ret)); + } else if (OB_FAIL(infos.serialize(serialize_buf, serialize_size, serialize_pos))) { + LOG_WARN("failed to serialize infos", K(ret), K(serialize_size), K(serialize_pos)); + } else if (OB_UNLIKELY(serialize_pos > serialize_size)) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("serialize error", KR(ret), K(serialize_pos), K(serialize_size)); + } else if (OB_ISNULL(hex_buf = static_cast(allocator_.alloc(hex_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", KR(ret), K(hex_size)); + } else if (OB_FAIL(hex_print(serialize_buf, serialize_pos, hex_buf, hex_size, hex_pos))) { + LOG_WARN("fail to print hex", KR(ret), K(serialize_pos), K(hex_size), K(serialize_buf)); + } else if (OB_UNLIKELY(hex_pos > hex_size)) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("encode error", KR(ret), K(hex_pos), K(hex_size)); + } else { + infos_str.assign(hex_buf, hex_pos); + LOG_DEBUG("succ to serialize", K(infos_str)); + } + return ret; +} + +int ObUDRAnalyzer::parse_pattern_to_gen_param_infos_str( + const common::ObString &pattern, + common::ObString &normalized_pattern, + common::ObString &fixed_param_infos_str, + common::ObString &dynamic_param_infos_str, + common::ObString &question_mark_def_name_ctx_str) +{ + int ret = OB_SUCCESS; + FixedParamValueArray fixed_param_infos; + DynamicParamInfoArray dynamic_param_infos; + ObSEArray raw_param_list; + ObQuestionMarkCtx question_mark_ctx; + MEMSET(&question_mark_ctx, 0, sizeof(ObQuestionMarkCtx)); + if (OB_FAIL(parse_pattern_to_gen_param_infos(pattern, normalized_pattern, raw_param_list, question_mark_ctx))) { + LOG_WARN("failed to parse_pattern_to_gen_param_infos", K(ret), K(pattern)); + } else if (OB_FAIL(cons_raw_param_infos(raw_param_list, + fixed_param_infos, + dynamic_param_infos))) { + LOG_WARN("failed to parse and gen param infos", K(ret)); + } else if (!fixed_param_infos.empty() + && OB_FAIL(serialize_to_hex(fixed_param_infos, + fixed_param_infos_str))) { + LOG_WARN("failed to serialize fixed param infos", K(ret)); + } else if (!dynamic_param_infos.empty() + && OB_FAIL(serialize_to_hex(dynamic_param_infos, + dynamic_param_infos_str))) { + LOG_WARN("failed to serialize dynamic param infos", K(ret)); + } else { + QuestionMarkDefNameCtx def_name_ctx(allocator_); + def_name_ctx.name_ = question_mark_ctx.name_; + def_name_ctx.count_ = question_mark_ctx.count_; + + if (question_mark_ctx.by_name_ && question_mark_ctx.name_ != nullptr + && OB_FAIL(serialize_to_hex(def_name_ctx, + question_mark_def_name_ctx_str))) { + LOG_WARN("failed to serialize defined name ctx", K(ret)); + } + } + return ret; +} + +} // namespace sql end +} // namespace oceanbase end diff --git a/src/sql/udr/ob_udr_analyzer.h b/src/sql/udr/ob_udr_analyzer.h new file mode 100644 index 000000000..a1d9efdfe --- /dev/null +++ b/src/sql/udr/ob_udr_analyzer.h @@ -0,0 +1,75 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_ANALYZER_H_ +#define OB_SQL_UDR_OB_UDR_ANALYZER_H_ + +#include "sql/udr/ob_udr_struct.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObUDRAnalyzer +{ +public: + ObUDRAnalyzer(common::ObIAllocator &allocator, + ObSQLMode mode, + common::ObCollationType conn_collation) + : allocator_(allocator), + sql_mode_(mode), + connection_collation_(conn_collation) + {} + static bool check_is_allow_stmt_type(stmt::StmtType stmt_type); + int parse_and_check(const common::ObString &pattern, + const common::ObString &replacement); + int parse_sql_to_gen_match_param_infos(const common::ObString &pattern, + common::ObString &normalized_pattern, + common::ObIArray &raw_params); + int parse_pattern_to_gen_param_infos(const common::ObString &pattern, + common::ObString &normalized_pattern, + common::ObIArray &raw_params, + ObQuestionMarkCtx &question_mark_ctx); + int parse_pattern_to_gen_param_infos_str(const common::ObString &pattern, + common::ObString &normalized_pattern, + common::ObString &fixed_param_infos_str, + common::ObString &dynamic_param_infos_str, + common::ObString &def_name_ctx_str); + +private: + template + int serialize_to_hex(const T &infos, common::ObString &infos_str); + int multiple_query_check(const ObString &sql); + int traverse_and_check(ParseNode *tree); + int check_transform_minus_op(ParseNode *tree); + int find_leftest_const_node(ParseNode &cur_node, ParseNode *&const_node); + int parse_and_resolve_stmt_type(const common::ObString &sql, + ParseResult &parse_result, + stmt::StmtType &stmt_type); + int cons_raw_param_infos(const common::ObIArray &raw_params, + FixedParamValueArray &fixed_param_infos, + DynamicParamInfoArray &dynamic_param_infos); + int add_fixed_param_value(const int64_t raw_param_idx, + const ParseNode *raw_param, + FixedParamValueArray &fixed_param_infos); + int add_dynamic_param_info(const int64_t raw_param_idx, + const int64_t question_mark_idx, + DynamicParamInfoArray &dynamic_param_infos); + +private: + common::ObIAllocator &allocator_; + ObSQLMode sql_mode_; + common::ObCollationType connection_collation_; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRAnalyzer); +}; + +} // namespace sql end +} // namespace oceanbase end + +#endif \ No newline at end of file diff --git a/src/sql/udr/ob_udr_callback.cpp b/src/sql/udr/ob_udr_callback.cpp new file mode 100644 index 000000000..d87adf878 --- /dev/null +++ b/src/sql/udr/ob_udr_callback.cpp @@ -0,0 +1,58 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + +#define USING_LOG_PREFIX SQL_QRR +#include "sql/udr/ob_udr_callback.h" + +namespace oceanbase +{ +namespace sql +{ + +int ObUDRAtomicOp::get_value(ObUDRItemMgr::UDRKeyNodePair *&rule_node) +{ + int ret = OB_SUCCESS; + rule_node = nullptr; + if (OB_ISNULL(rule_node_)) { + ret = OB_NOT_INIT; + LOG_WARN("invalid argument", K(rule_node_)); + } else if (OB_SUCC(lock(*rule_node_))) { + rule_node = rule_node_; + } else { + if (NULL != rule_node_) { + rule_node_->dec_ref_count(); + rule_node_ = NULL; + } + } + return ret; +} + +void ObUDRAtomicOp::operator()(RuleItemKV &entry) +{ + if (NULL != entry.second) { + entry.second->inc_ref_count(); + rule_node_ = entry.second; + } +} + +ObUDRWlockAndRefGuard::~ObUDRWlockAndRefGuard() +{ + if (NULL != rule_node_) { + rule_node_->dec_ref_count(); + rule_node_->unlock(); + } +} + +ObUDRRlockAndRefGuard::~ObUDRRlockAndRefGuard() +{ + if (NULL != rule_node_) { + rule_node_->dec_ref_count(); + rule_node_->unlock(); + } +} + +} // namespace sql end +} // namespace oceanbase end diff --git a/src/sql/udr/ob_udr_callback.h b/src/sql/udr/ob_udr_callback.h new file mode 100644 index 000000000..7bccfc2e3 --- /dev/null +++ b/src/sql/udr/ob_udr_callback.h @@ -0,0 +1,74 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_CALLBACK_H_ +#define OB_SQL_UDR_OB_UDR_CALLBACK_H_ + +#include "sql/udr/ob_udr_item_mgr.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObUDRAtomicOp +{ +protected: + typedef common::hash::HashMapPair RuleItemKV; + +public: + ObUDRAtomicOp() + : rule_node_(NULL) + { + } + virtual ~ObUDRAtomicOp() {} + virtual int get_value(ObUDRItemMgr::UDRKeyNodePair *&rule_node); + // get rule node and increase reference count + void operator()(RuleItemKV &entry); + +protected: + // when get value, need lock + virtual int lock(ObUDRItemMgr::UDRKeyNodePair &rule_node) = 0; +protected: + ObUDRItemMgr::UDRKeyNodePair *rule_node_; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRAtomicOp); +}; + +class ObUDRWlockAndRefGuard : public ObUDRAtomicOp +{ +public: + ObUDRWlockAndRefGuard() : ObUDRAtomicOp() + { + } + virtual ~ObUDRWlockAndRefGuard(); + int lock(ObUDRItemMgr::UDRKeyNodePair &rule_node) + { + return rule_node.lock(false/*wlock*/); + }; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRWlockAndRefGuard); +}; + +class ObUDRRlockAndRefGuard : public ObUDRAtomicOp +{ +public: + ObUDRRlockAndRefGuard() : ObUDRAtomicOp() + { + } + virtual ~ObUDRRlockAndRefGuard(); + int lock(ObUDRItemMgr::UDRKeyNodePair &rule_node) + { + return rule_node.lock(true/*rlock*/); + }; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRRlockAndRefGuard); +}; + +} +} +#endif diff --git a/src/sql/udr/ob_udr_context.h b/src/sql/udr/ob_udr_context.h new file mode 100644 index 000000000..a4e7ade84 --- /dev/null +++ b/src/sql/udr/ob_udr_context.h @@ -0,0 +1,51 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_CONTEXT_H_ +#define OB_SQL_UDR_OB_UDR_CONTEXT_H_ + +#include "lib/container/ob_se_array.h" +#include "lib/string/ob_string.h" +#include "sql/plan_cache/ob_plan_cache_util.h" + +namespace oceanbase +{ +namespace sql +{ + +struct ObUDRContext +{ +public: + ObUDRContext() + : is_ps_mode_(false), + tenant_id_(OB_INVALID_ID), + pattern_digest_(0), + coll_type_(common::CS_TYPE_INVALID), + db_name_(), + normalized_pattern_(), + raw_param_list_() {} + virtual ~ObUDRContext() {} + + TO_STRING_KV(K_(tenant_id), + K_(pattern_digest), + K_(coll_type), + K_(db_name), + K_(normalized_pattern)); + + bool is_ps_mode_; + uint64_t tenant_id_; + uint64_t pattern_digest_; + ObCollationType coll_type_; + common::ObString db_name_; + common::ObString normalized_pattern_; + common::ObSEArray raw_param_list_; +}; + +} // namespace sql end +} // namespace oceanbase end + +#endif \ No newline at end of file diff --git a/src/sql/udr/ob_udr_item.cpp b/src/sql/udr/ob_udr_item.cpp new file mode 100644 index 000000000..3a8d6f2cb --- /dev/null +++ b/src/sql/udr/ob_udr_item.cpp @@ -0,0 +1,164 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#define USING_LOG_PREFIX SQL_QRR +#include "lib/utility/ob_print_utils.h" +#include "sql/udr/ob_udr_item.h" + +namespace oceanbase +{ +namespace sql +{ + +ObUDRItem::~ObUDRItem() +{ + reset(); +} + +void ObUDRItem::reset() +{ +#define SAFE_FREE_STR(name) \ +if (NULL != name.ptr()) { \ + allocator_.free(name.ptr()); \ + name.reset(); \ +} +#define SAFE_FREE_PTR(ptr) \ +if (NULL != ptr) { \ + allocator_.free(ptr); \ + ptr = nullptr; \ +} + tenant_id_ = OB_INVALID_ID; + pattern_digest_ = 0; + rule_id_ = OB_INVALID_ID; + rule_version_ = OB_INVALID_VERSION; + rule_status_ = INVALID_STATUS; + coll_type_ = common::CS_TYPE_INVALID; + SAFE_FREE_STR(db_name_); + SAFE_FREE_STR(rule_name_); + SAFE_FREE_STR(pattern_); + SAFE_FREE_STR(replacement_); + SAFE_FREE_STR(normalized_pattern_); + SAFE_FREE_STR(fixed_param_infos_str_); + SAFE_FREE_STR(dynamic_param_infos_str_); + SAFE_FREE_STR(question_mark_ctx_str_); + fixed_param_info_array_.reset(); + dynamic_param_info_array_.reset(); + question_mark_def_name_ctx_.reset(); + SAFE_FREE_PTR(fixed_param_deserialize_buf_); + SAFE_FREE_PTR(dynamic_param_deserialize_buf_); + SAFE_FREE_PTR(question_mark_deserialize_buf_); +#undef SAFE_FREE_PTR +#undef SAFE_FREE_STR +} + +int64_t ObUDRItem::inc_ref_count() +{ + return ATOMIC_AAF(&ref_count_, 1); +} + +int64_t ObUDRItem::dec_ref_count() +{ + int64_t ref_count = ATOMIC_SAF(&ref_count_, 1); + if (ref_count > 0) { + // do nothing + } else if (0 == ref_count) { + LOG_DEBUG("remove rule item", K(ref_count), K(this)); + this->~ObUDRItem(); + allocator_.free(this);// I'm sure this is the last line, so it's safe here + } else { + LOG_ERROR("invalid ref count", K(ref_count)); + } + return ref_count; +} + +int ObUDRItem::deep_copy(const ObUDRInfo &rule_info) +{ +#define DEEP_COPY_STR(name) \ +if (OB_SUCC(ret)) { \ + if (OB_FAIL(set_##name(rule_info.name##_))) { \ + LOG_WARN("failed to deep copy str", K(rule_info.name##_)); \ + } \ +} + int ret = OB_SUCCESS; + tenant_id_ = rule_info.tenant_id_; + pattern_digest_ = rule_info.pattern_digest_; + rule_id_ = rule_info.rule_id_; + rule_version_ = rule_info.rule_version_; + rule_status_ = rule_info.rule_status_; + coll_type_ = rule_info.coll_type_; + DEEP_COPY_STR(db_name); + DEEP_COPY_STR(rule_name); + DEEP_COPY_STR(pattern); + DEEP_COPY_STR(replacement); + DEEP_COPY_STR(normalized_pattern); + DEEP_COPY_STR(fixed_param_infos_str); + DEEP_COPY_STR(dynamic_param_infos_str); + DEEP_COPY_STR(question_mark_ctx_str); + return ret; +#undef DEEP_COPY_STR +} + +template +int ObUDRItem::deserialize_with_hex_str(const common::ObString &str, T &infos, char *&deserialize_buf) +{ + int ret = OB_SUCCESS; + infos.reset(); + deserialize_buf = nullptr; + const int64_t str_size = str.length(); + const int64_t deserialize_size = str.length() / 2 + 1; + int64_t deserialize_pos = 0; + if (str.empty()) { + // do nothing + LOG_DEBUG("str is empty", K(ret)); + } else if (OB_ISNULL(deserialize_buf = static_cast(allocator_.alloc(deserialize_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret), K(deserialize_size)); + } else if (OB_FAIL(hex_to_cstr(str.ptr(), str_size, deserialize_buf, deserialize_size))) { + LOG_WARN("fail to get cstr from hex", K(ret), K(str_size), K(deserialize_size), K(str)); + } else if (OB_FAIL(infos.deserialize(deserialize_buf, deserialize_size, deserialize_pos))) { + LOG_WARN("fail to deserialize", K(ret), K(deserialize_pos), K(deserialize_size), K(str)); + } else if (OB_UNLIKELY(deserialize_pos > deserialize_size)) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("deserialize error", K(ret), K(deserialize_pos), K(deserialize_size)); + } else { + LOG_DEBUG("succ to deserialize", K(infos)); + } + return ret; +} + +int ObUDRItem::deserialize_fixed_param_info_array(const common::ObString str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(deserialize_with_hex_str(str, fixed_param_info_array_, + fixed_param_deserialize_buf_))) { + LOG_WARN("failed to deserialize", K(ret), K(str)); + } + return ret; +} + +int ObUDRItem::deserialize_dynamic_param_info_array(const common::ObString str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(deserialize_with_hex_str(str, dynamic_param_info_array_, + dynamic_param_deserialize_buf_))) { + LOG_WARN("failed to deserialize", K(ret), K(str)); + } + return ret; +} + +int ObUDRItem::deserialize_question_mark_by_name_ctx(const common::ObString str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(deserialize_with_hex_str(str, question_mark_def_name_ctx_, + question_mark_deserialize_buf_))) { + LOG_WARN("failed to deserialize", K(ret), K(str)); + } + return ret; +} + +} // namespace sql end +} // namespace oceanbase end \ No newline at end of file diff --git a/src/sql/udr/ob_udr_item.h b/src/sql/udr/ob_udr_item.h new file mode 100644 index 000000000..f61b05314 --- /dev/null +++ b/src/sql/udr/ob_udr_item.h @@ -0,0 +1,117 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_ITEM_H_ +#define OB_SQL_UDR_OB_UDR_ITEM_H_ +#include "sql/udr/ob_udr_struct.h" +#include "sql/udr/ob_udr_context.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObUDRItem : protected ObUDRInfo +{ +public: + ObUDRItem(common::ObIAllocator &allocator) + : ObUDRInfo(), + allocator_(allocator), + ref_count_(0), + fixed_param_deserialize_buf_(nullptr), + dynamic_param_deserialize_buf_(nullptr), + question_mark_deserialize_buf_(nullptr), + fixed_param_info_array_(), + dynamic_param_info_array_(), + question_mark_def_name_ctx_(allocator) {} + virtual ~ObUDRItem(); + +#define DEFINE_SIMPLE_SETTER(name, type) \ +OB_INLINE void set_##name(type name) { name##_ = name; } +#define DEFINE_SIMPLE_GETTER(ret_type, name) \ +OB_INLINE ret_type get_##name() const { return name##_; } +#define DEFINE_STR_SETTER(name) \ +OB_INLINE int set_##name(const common::ObString &name) \ +{ \ + return ob_write_string(allocator_, name, name##_); \ +} +#define DEFINE_STR_GETTER(name) \ +OB_INLINE const common::ObString& get_##name() const { return name##_; } + + void reset(); + DEFINE_SIMPLE_SETTER(tenant_id, uint64_t); + DEFINE_SIMPLE_SETTER(pattern_digest, uint64_t); + DEFINE_SIMPLE_SETTER(rule_id, int64_t); + DEFINE_SIMPLE_SETTER(rule_version, int64_t); + DEFINE_SIMPLE_SETTER(coll_type, ObCollationType); + DEFINE_STR_SETTER(db_name); + DEFINE_STR_SETTER(rule_name); + DEFINE_STR_SETTER(pattern); + DEFINE_STR_SETTER(replacement); + DEFINE_STR_SETTER(normalized_pattern); + DEFINE_STR_SETTER(fixed_param_infos_str); + DEFINE_STR_SETTER(dynamic_param_infos_str); + DEFINE_STR_SETTER(question_mark_ctx_str); + + DEFINE_SIMPLE_GETTER(uint64_t, tenant_id); + DEFINE_SIMPLE_GETTER(uint64_t, pattern_digest); + DEFINE_SIMPLE_GETTER(int64_t, rule_id); + DEFINE_SIMPLE_GETTER(int64_t, rule_version); + DEFINE_SIMPLE_GETTER(ObCollationType, coll_type); + DEFINE_STR_GETTER(db_name); + DEFINE_STR_GETTER(rule_name); + DEFINE_STR_GETTER(pattern); + DEFINE_STR_GETTER(replacement); + DEFINE_STR_GETTER(normalized_pattern); + DEFINE_STR_GETTER(fixed_param_infos_str); + DEFINE_STR_GETTER(dynamic_param_infos_str); + DEFINE_STR_GETTER(question_mark_ctx_str); + + inline const FixedParamValueArray &get_fixed_param_value_array() const { return fixed_param_info_array_; } + inline const DynamicParamInfoArray &get_dynamic_param_info_array() const { return dynamic_param_info_array_; } + inline const QuestionMarkDefNameCtx *get_question_mark_def_name_ctx() const { return &question_mark_def_name_ctx_; } + + inline int64_t get_ref_count() const { return ref_count_; } + inline bool is_enable_status() const { return ENABLE_STATUS == rule_status_; } + void set_rule_status(const RuleStatus status) { ATOMIC_STORE(&rule_status_, status); } + RuleStatus get_rule_status() const { return ATOMIC_LOAD(&rule_status_); } + int64_t inc_ref_count(); + int64_t dec_ref_count(); + int deep_copy(const ObUDRInfo &rule_info); + template + int deserialize_with_hex_str(const common::ObString &str, T &infos, char *&deserialize_buf); + int deserialize_fixed_param_info_array(const common::ObString str); + int deserialize_dynamic_param_info_array(const common::ObString str); + int deserialize_question_mark_by_name_ctx(const common::ObString str); + +#undef DEFINE_STR_SETTER +#undef DEFINE_STR_GETTER +#undef DEFINE_SIMPLE_SETTER +#undef DEFINE_SIMPLE_GETTER + + VIRTUAL_TO_STRING_KV(K_(ref_count), + K_(fixed_param_info_array), + K_(dynamic_param_info_array), + K_(question_mark_def_name_ctx)); + +private: + common::ObIAllocator &allocator_; + int64_t ref_count_; + char *fixed_param_deserialize_buf_; + char *dynamic_param_deserialize_buf_; + char *question_mark_deserialize_buf_; + FixedParamValueArray fixed_param_info_array_; + DynamicParamInfoArray dynamic_param_info_array_; + QuestionMarkDefNameCtx question_mark_def_name_ctx_; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRItem); +}; + +} // namespace sql end +} // namespace oceanbase end + +#endif \ No newline at end of file diff --git a/src/sql/udr/ob_udr_item_mgr.cpp b/src/sql/udr/ob_udr_item_mgr.cpp new file mode 100644 index 000000000..ea7bb0b8b --- /dev/null +++ b/src/sql/udr/ob_udr_item_mgr.cpp @@ -0,0 +1,539 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#define USING_LOG_PREFIX SQL_QRR +#include "sql/udr/ob_udr_callback.h" +#include "sql/udr/ob_udr_item_mgr.h" + +namespace oceanbase +{ +namespace sql +{ + +struct DecAllRuleNodeRefFunc +{ +public: + DecAllRuleNodeRefFunc() {} + int operator()(const hash::HashMapPair &kv) + { + int ret = OB_SUCCESS; + if (OB_ISNULL(kv.second)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(ret)); + } else { + kv.second->dec_ref_count(); + } + return ret; + } +}; + +int ObUDRItemMgr::UDRKey::deep_copy(common::ObIAllocator &allocator, const UDRKey &other) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ob_write_string(allocator, other.db_name_, db_name_))) { + LOG_WARN("failed to deep copy db name", K(ret)); + } else if (OB_FAIL(ob_write_string(allocator, other.normalized_pattern_, normalized_pattern_))) { + LOG_WARN("failed to deep copy normalized pattern", K(ret)); + } else { + tenant_id_ = other.tenant_id_; + pattern_digest_ = other.pattern_digest_; + } + return ret; +} + +void ObUDRItemMgr::UDRKey::destory(common::ObIAllocator &allocator) +{ +#define SAFE_FREE_STR(name) \ +if (NULL != name.ptr()) { \ + allocator.free(name.ptr()); \ + name.reset(); \ +} + tenant_id_ = OB_INVALID_ID; + pattern_digest_ = 0; + SAFE_FREE_STR(db_name_); + SAFE_FREE_STR(normalized_pattern_); +#undef SAFE_FREE_STR +} + +int ObUDRItemMgr::UDRKeyNodePair::lock(bool is_rdlock) +{ + int ret = OB_SUCCESS; + if (is_rdlock) { + if (OB_FAIL(ref_lock_.rdlock())) { + LOG_WARN("failed to read lock", K(ret)); + } + } else { + if (OB_FAIL(ref_lock_.wrlock())) { + LOG_WARN("failed to write lock", K(ret)); + } + } + return ret; +} + +void ObUDRItemMgr::UDRKeyNodePair::reset() +{ + ObUDRItem *item = nullptr; + while (!rule_item_list_.empty()) { + rule_item_list_.pop_front(item); + if (OB_ISNULL(item)) { + //do nothing + } else { + item->dec_ref_count(); + item = nullptr; + } + } + rule_key_.destory(allocator_); + rule_item_list_.reset(); +} + +int64_t ObUDRItemMgr::UDRKeyNodePair::inc_ref_count() +{ + return ATOMIC_AAF(&ref_count_, 1); +} + +int64_t ObUDRItemMgr::UDRKeyNodePair::dec_ref_count() +{ + int64_t ref_count = ATOMIC_SAF(&ref_count_, 1); + if (ref_count > 0) { + // do nothing + } else if (0 == ref_count) { + LOG_DEBUG("remove rule node", K(ref_count), K(this)); + this->~UDRKeyNodePair(); + allocator_.free(this);// I'm sure this is the last line, so it's safe here + } else { + LOG_ERROR("invalid ref count", K(ref_count)); + } + return ref_count; +} + +int ObUDRItemMgr::UDRKeyNodePair::deep_copy_rule_key(const UDRKey &other) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(rule_key_.deep_copy(allocator_, other))) { + LOG_WARN("failed to deep copy rule key", K(ret)); + } + return ret; +} + +int ObUDRItemMgr::UDRKeyNodePair::add_rule_item(ObUDRItem *rule_item) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(rule_item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(ret)); + } else if (OB_FAIL(rule_item_list_.push_back(rule_item))) { + LOG_WARN("failed to add rule item", K(ret)); + } else { + rule_item->inc_ref_count(); + } + return ret; +} + +int ObUDRItemMgr::UDRKeyNodePair::remove_rule_item(const int64_t rule_id) +{ + int ret = OB_SUCCESS; + RuleItemList::iterator iter = rule_item_list_.begin(); + for (; OB_SUCC(ret) && iter != rule_item_list_.end(); iter++) { + ObUDRItem *item = *iter; + if (rule_id == item->get_rule_id()) { + if (OB_FAIL(rule_item_list_.erase(iter))) { + LOG_WARN("failed to erase rule item", K(ret)); + } else { + item->dec_ref_count(); + } + break; + } + } + return ret; +} + +int ObUDRItemMgr::UDRKeyNodePair::get_rule_item_by_id(const int64_t rule_id, + ObUDRItem *&rule_item) +{ + int ret = OB_SUCCESS; + rule_item = NULL; + RuleItemList::iterator iter = rule_item_list_.begin(); + for (; OB_SUCC(ret) && iter != rule_item_list_.end(); iter++) { + ObUDRItem *item = *iter; + if (rule_id == item->get_rule_id()) { + rule_item = item; + break; + } + } + return ret; +} + +ObUDRItemMgr::~ObUDRItemMgr() +{ + destroy(); +} + +void ObUDRItemMgr::destroy() +{ + reuse(); + inited_ = false; +} + +int ObUDRItemMgr::reuse() +{ + int ret = OB_SUCCESS; + if (inited_) { + DecAllRuleNodeRefFunc callback; + if (OB_FAIL(rule_key_node_map_.foreach_refactored(callback))) { + LOG_WARN("traversal rule_key_node_map_ failed", K(ret)); + } + } + rule_key_node_map_.reuse(); + return ret; +} + +int ObUDRItemMgr::init(uint64_t tenant_id, common::ObIAllocator &allocator) +{ + int ret = OB_SUCCESS; + int bucket_size = 40960; + if (OB_UNLIKELY(rule_key_node_map_.created())) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", K(ret)); + } else if (OB_FAIL(rule_key_node_map_.create(bucket_size, "RewriteRuleMap", "RewriteRuleNode"))) { + LOG_WARN("failed create rule map", K(ret)); + } else { + inited_ = true; + tenant_id_ = tenant_id; + allocator_ = &allocator; + } + LOG_INFO("init rewrite rule item mapping manager", K(ret)); + return ret; +} + +template +int ObUDRItemMgr::alloc(UDRRefObjGuard &guard, Args&& ...args) +{ + int ret = OB_SUCCESS; + char *ptr = NULL; + if (NULL == (ptr = static_cast(allocator_->alloc(sizeof(T))))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + OB_LOG(WARN, "failed to allocate memory for lib cache node", K(ret)); + } else { + guard.ref_obj_ = new(ptr) T(std::forward(args)...); + guard.ref_obj_->inc_ref_count(); + } + return ret; +} + +template +void ObUDRItemMgr::cons_shallow_copy_rule_key(const Info &rule_info, + UDRKey &rule_key) const +{ + rule_key.tenant_id_ = rule_info.tenant_id_; + rule_key.pattern_digest_ = rule_info.pattern_digest_; + rule_key.db_name_ = rule_info.db_name_; + rule_key.normalized_pattern_ = rule_info.normalized_pattern_; +} + +int ObUDRItemMgr::get_value(const UDRKey &rule_key, + UDRKeyNodePair *&rule_node, + ObUDRAtomicOp &op) +{ + int ret = OB_SUCCESS; + int hash_err = rule_key_node_map_.read_atomic(rule_key, op); + switch (hash_err) { + case OB_SUCCESS: { + if (OB_FAIL(op.get_value(rule_node))) { + LOG_DEBUG("failed to lock rule node", K(ret), K(rule_key)); + } + break; + } + case OB_HASH_NOT_EXIST: { + LOG_DEBUG("entry does not exist.", K(rule_key)); + break; + } + default: { + LOG_WARN("failed to get cache node", K(ret), K(rule_key)); + ret = hash_err; + break; + } + } + return ret; +} + +int ObUDRItemMgr::cons_deep_copy_rule_item(const ObUDRInfo &rule_info, + ObUDRItem &rule_item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(rule_item.deep_copy(rule_info))) { + LOG_WARN("failed to deep copy rule info", K(ret)); + } else if (OB_FAIL(rule_item.deserialize_fixed_param_info_array(rule_info.fixed_param_infos_str_))) { + LOG_WARN("failed to deserialize_fixed_param_info_array", K(ret)); + } else if (OB_FAIL(rule_item.deserialize_dynamic_param_info_array(rule_info.dynamic_param_infos_str_))) { + LOG_WARN("failed to deserialize_dynamic_param_info_array", K(ret)); + } else if (OB_FAIL(rule_item.deserialize_question_mark_by_name_ctx(rule_info.question_mark_ctx_str_))) { + LOG_WARN("failed to deserialize_question_mark_by_name_ctx", K(ret)); + } + return ret; +} + +int ObUDRItemMgr::add_rule_node(const UDRKey &rule_key, UDRKeyNodePair *rule_node) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(rule_key_node_map_.set_refactored(rule_key, rule_node))) { + LOG_WARN("failed to add rule node", K(ret), K(rule_key)); + } else { + rule_node->inc_ref_count(); + } + return ret; +} + +int ObUDRItemMgr::add_item(const UDRKey &rule_key, const ObUDRInfo &rule_info) +{ + int ret = OB_SUCCESS; + UDRKeyNodePair *rule_node = nullptr; + UDRItemRefGuard rule_item_guard; + ObUDRWlockAndRefGuard w_ref_lock_guard; + if (OB_ISNULL(allocator_)) { + ret = OB_NOT_INIT; + LOG_WARN("allocator is null", K(ret)); + } else if (OB_FAIL(alloc(rule_item_guard, *allocator_)) + || OB_ISNULL(rule_item_guard.ref_obj_)) { + LOG_WARN("failed to alloc rule item", K(ret)); + } else if (OB_FAIL(cons_deep_copy_rule_item(rule_info, *rule_item_guard.ref_obj_))) { + LOG_WARN("failed to construct deep copy rule item", K(ret), K(rule_info)); + } else if (OB_FAIL(get_value(rule_key, rule_node, w_ref_lock_guard))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get rule node", K(ret)); + } else if (nullptr == rule_node) { + UDRRefObjGuard rule_node_guard; + if (OB_FAIL(alloc(rule_node_guard, *allocator_)) + || OB_ISNULL(rule_node_guard.ref_obj_)) { + LOG_WARN("failed to alloc rule node", K(ret)); + } else if (OB_FAIL(rule_node_guard.ref_obj_->deep_copy_rule_key(rule_key))) { + LOG_WARN("failed to construct deep copy rule key", K(ret), K(rule_key)); + } else if (OB_FAIL(rule_node_guard.ref_obj_->add_rule_item(rule_item_guard.ref_obj_))) { + LOG_WARN("failed to add rule item", K(ret)); + } else if (OB_FAIL(add_rule_node(rule_node_guard.ref_obj_->rule_key_, + rule_node_guard.ref_obj_))) { + LOG_WARN("failed to add item", K(ret), K(rule_key), K(rule_info)); + } + } else { + if (OB_FAIL(rule_node->add_rule_item(rule_item_guard.ref_obj_))) { + LOG_WARN("failed to add rule item", K(ret)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("succ to add item", K(rule_key), + KPC(rule_item_guard.ref_obj_), K(rule_item_guard.ref_obj_->get_ref_count())); + } + return ret; +} + +int ObUDRItemMgr::remove_rule_node(const UDRKey &rule_key) +{ + int ret = OB_SUCCESS; + UDRKeyNodePair *del_node = nullptr; + if (OB_FAIL(rule_key_node_map_.erase_refactored(rule_key, &del_node))) { + LOG_WARN("failed to erase rule item", K(ret), K(rule_key)); + } else if (OB_ISNULL(del_node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(ret)); + } else { + del_node->dec_ref_count(); + } + return ret; +} + +int ObUDRItemMgr::erase_item(const UDRKey &rule_key, const int64_t rule_id) +{ + int ret = OB_SUCCESS; + ObUDRItem *rule_item = nullptr; + UDRKeyNodePair *rule_node = nullptr; + ObUDRWlockAndRefGuard w_ref_lock_guard; + if (OB_FAIL(get_value(rule_key, rule_node, w_ref_lock_guard))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get rule node", K(ret)); + } else if (nullptr == rule_node) { + ret = OB_SUCCESS; + LOG_INFO("item already not exist", K(rule_key)); + } else { + if (OB_FAIL(rule_node->remove_rule_item(rule_id))) { + LOG_WARN("failed to remove rule item", K(ret)); + } else if (rule_node->is_empty() && OB_FAIL(remove_rule_node(rule_key))) { + LOG_WARN("failed to remove rule node", K(ret), K(rule_key)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("succ to erase item", K(rule_key)); + } + return ret; +} + +int ObUDRItemMgr::match_fixed_param_list(const ObUDRContext &rule_ctx, + const ObUDRItem *rule_item, + bool &is_match) const +{ + int ret = OB_SUCCESS; + is_match = true; + const FixedParamValueArray &fixed_param_list = rule_item->get_fixed_param_value_array(); + for (int64_t i = 0; OB_SUCC(ret) && is_match && i < fixed_param_list.count(); ++i) { + ParseNode *raw_param = NULL; + ObPCParam *pc_param = NULL; + const FixedParamValue &fixed_param = fixed_param_list.at(i); + if (fixed_param.idx_ >= rule_ctx.raw_param_list_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid idx", K(fixed_param.idx_), K(rule_ctx.raw_param_list_.count())); + } else if (OB_ISNULL(pc_param = rule_ctx.raw_param_list_.at(fixed_param.idx_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(pc_param)); + } else if (NULL == (raw_param = pc_param->node_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(raw_param)); + } else if (0 != fixed_param.raw_text_.compare(ObString(raw_param->text_len_, raw_param->raw_text_))) { + is_match = false; + } + } + return ret; +} + +int ObUDRItemMgr::match_rule_item(const ObUDRContext &rule_ctx, + const ObUDRItem *rule_item, + bool &is_match) const +{ + int ret = OB_SUCCESS; + is_match = false; + if (rule_item->get_rule_status() != ObUDRInfo::ENABLE_STATUS) { + LOG_DEBUG("invalid rule item", K(rule_item->get_rule_status())); + } else if (OB_FAIL(match_fixed_param_list(rule_ctx, rule_item, is_match))) { + LOG_WARN("failed to match fixed param list", K(ret)); + } + return ret; +} + +int ObUDRItemMgr::fuzzy_check_by_pattern_digest(const uint64_t pattern_digest, + bool &is_exists) +{ + int ret = OB_SUCCESS; + is_exists = false; + for (RuleKeyNodeMap::iterator iter = rule_key_node_map_.begin(); iter != rule_key_node_map_.end(); ++iter) { + if (pattern_digest == iter->first.pattern_digest_) { + is_exists = true; + break; + } + } + return ret; +} + +int ObUDRItemMgr::get_udr_item(const ObUDRContext &rule_ctx, + UDRItemRefGuard &item_guard, + PatternConstConsList *cst_cons_list) +{ + int ret = OB_SUCCESS; + UDRKey rule_key; + item_guard.ref_obj_ = NULL; + UDRKeyNodePair *rule_node = NULL; + ObUDRRlockAndRefGuard r_ref_lock_guard; + cons_shallow_copy_rule_key(rule_ctx, rule_key); + if (OB_FAIL(get_value(rule_key, rule_node, r_ref_lock_guard))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get rule node", K(ret)); + } else if (nullptr == rule_node) { + ret = OB_SUCCESS; + LOG_DEBUG("item not exist", K(rule_key)); + } else if (NULL != cst_cons_list && OB_FAIL(get_all_cst_cons_by_key(rule_key, *cst_cons_list))) { + LOG_WARN("failed to get all cst cons by key", K(ret)); + } else { + bool is_match = false; + UDRKeyNodePair::RuleItemList::iterator iter = rule_node->rule_item_list_.begin(); + for (; OB_SUCC(ret) && !is_match && iter != rule_node->rule_item_list_.end(); iter++) { + ObUDRItem *item = *iter; + if (OB_FAIL(match_rule_item(rule_ctx, item, is_match))) { + LOG_WARN("failed to match rule item", K(ret)); + } else if (is_match) { + item->inc_ref_count(); + item_guard.ref_obj_ = item; + LOG_TRACE("succ to match rewrite rule item", KPC(item)); + break; + } + } + } + LOG_TRACE("get udr item", K(ret), K(rule_key)); + return ret; +} + +int ObUDRItemMgr::get_all_cst_cons_by_key(const UDRKey &rule_key, + PatternConstConsList &cst_cons_list) +{ + int ret = OB_SUCCESS; + UDRKeyNodePair *rule_node = NULL; + ObUDRRlockAndRefGuard r_ref_lock_guard; + if (OB_FAIL(get_value(rule_key, rule_node, r_ref_lock_guard))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get rule node", K(ret)); + } else if (nullptr == rule_node) { + ret = OB_SUCCESS; + LOG_DEBUG("item not exist", K(rule_key)); + } else { + UDRKeyNodePair::RuleItemList::iterator iter = rule_node->rule_item_list_.begin(); + for (; OB_SUCC(ret) && iter != rule_node->rule_item_list_.end(); iter++) { + ObUDRItem *item = *iter; + if (OB_ISNULL(item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("is null", K(ret)); + } else if (!item->get_fixed_param_value_array().empty() + && OB_FAIL(cst_cons_list.push_back(item->get_fixed_param_value_array()))) { + LOG_WARN("failed to add fixed param value list", K(ret)); + } + } + } + return ret; +} + +int ObUDRItemMgr::get_rule_item_by_id(const UDRKey &rule_key, + const int64_t rule_id, + ObUDRItem *&rule_item) +{ + int ret = OB_SUCCESS; + rule_item = NULL; + UDRKeyNodePair *rule_node = nullptr; + ObUDRRlockAndRefGuard r_ref_lock_guard; + if (OB_FAIL(get_value(rule_key, rule_node, r_ref_lock_guard))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get rule node", K(ret)); + } else if (nullptr == rule_node) { + ret = OB_SUCCESS; + LOG_DEBUG("item not exist", K(rule_key)); + } else if (OB_FAIL(rule_node->get_rule_item_by_id(rule_id, rule_item))) { + LOG_WARN("failed to remove rule item", K(ret)); + } + return ret; +} + +int ObUDRItemMgr::sync_local_cache_rules(const ObIArray& rule_infos) +{ + int ret = OB_SUCCESS; + LOG_INFO("sync local cache rules", K(tenant_id_), K(rule_infos)); + for (int64_t i = 0; i < rule_infos.count() && OB_SUCC(ret); ++i) { + const ObUDRInfo &rule_info = rule_infos.at(i); + UDRKey rule_key; + ObUDRItem *rule_item = NULL; + cons_shallow_copy_rule_key(rule_info, rule_key); + if (rule_info.rule_status_ == ObUDRInfo::DELETE_STATUS) { + if (OB_FAIL(erase_item(rule_key, rule_info.rule_id_))) { + LOG_WARN("failed to erase item", K(ret), K(rule_key)); + } + } else if (OB_FAIL(get_rule_item_by_id(rule_key, rule_info.rule_id_, rule_item))) { + LOG_WARN("failed to get rule item by name", K(ret), K(rule_key), K(rule_info)); + } else if (NULL != rule_item) { + // modify rule status + rule_item->set_rule_status(rule_info.rule_status_); + } else if (OB_FAIL(add_item(rule_key, rule_info))) { + LOG_WARN("failed to add item", K(ret), K(rule_key)); + } + } + return ret; +} + +} // namespace sql end +} // namespace oceanbase end diff --git a/src/sql/udr/ob_udr_item_mgr.h b/src/sql/udr/ob_udr_item_mgr.h new file mode 100644 index 000000000..6d093eb30 --- /dev/null +++ b/src/sql/udr/ob_udr_item_mgr.h @@ -0,0 +1,182 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_ITEM_MGR_H_ +#define OB_SQL_UDR_OB_UDR_ITEM_MGR_H_ + +#include "lib/hash/ob_hashmap.h" +#include "sql/udr/ob_udr_item.h" +#include "sql/udr/ob_udr_context.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObUDRAtomicOp; + +class ObUDRItemMgr +{ +public: + template + class UDRRefObjGuard + { + friend class ObUDRItemMgr; + public: + UDRRefObjGuard() : ref_obj_(NULL) + { + } + ~UDRRefObjGuard() + { + if (NULL != ref_obj_) { + ref_obj_->dec_ref_count(); + ref_obj_ = NULL; + } + } + const T* get_ref_obj() const { return ref_obj_; } + bool is_valid() const { return ref_obj_ != NULL; } + + TO_STRING_KV(K_(ref_obj)); + private: + T* ref_obj_; + }; + + struct UDRKey + { + UDRKey() + : tenant_id_(OB_INVALID_ID), + pattern_digest_(OB_INVALID_ID), + db_name_(), + normalized_pattern_() {} + + UDRKey(uint64_t tenant_id, + uint64_t pattern_digest, + ObCollationType coll_type, + const common::ObString& db_name, + const common::ObString& normalized_pattern) + : tenant_id_(tenant_id), + pattern_digest_(pattern_digest), + db_name_(db_name), + normalized_pattern_(normalized_pattern) {} + + inline uint64_t hash() const + { + uint64_t hash_ret = murmurhash(&tenant_id_, sizeof(uint64_t), 0); + hash_ret = murmurhash(&pattern_digest_, sizeof(uint64_t), hash_ret); + hash_ret = db_name_.hash(hash_ret); + hash_ret = normalized_pattern_.hash(hash_ret); + return hash_ret; + } + inline bool operator==(const UDRKey& key) const + { + return tenant_id_ == key.tenant_id_ + && pattern_digest_ == key.pattern_digest_ + && 0 == db_name_.compare(key.db_name_) + && 0 == normalized_pattern_.compare(key.normalized_pattern_); + } + int deep_copy(common::ObIAllocator &allocator, const UDRKey &other); + void destory(common::ObIAllocator &allocator); + TO_STRING_KV(K_(tenant_id), + K_(pattern_digest), + K_(db_name), + K_(normalized_pattern)); + + uint64_t tenant_id_; + uint64_t pattern_digest_; + common::ObString db_name_; + common::ObString normalized_pattern_; + }; + + struct UDRKeyNodePair + { + typedef common::ObList RuleItemList; + UDRKeyNodePair(common::ObIAllocator &allocator) + : allocator_(allocator), + ref_count_(0), + rule_key_(), + ref_lock_(common::ObLatchIds::REWRITE_RULE_ITEM_LOCK), + rule_item_list_(allocator) {} + ~UDRKeyNodePair() { reset(); } + + void reset(); + int64_t inc_ref_count(); + int64_t dec_ref_count(); + int64_t get_ref_count() const { return ref_count_; } + int lock(bool is_rdlock); + int unlock() { return ref_lock_.unlock(); } + int deep_copy_rule_key(const UDRKey &other); + bool is_empty() const { return rule_item_list_.empty(); } + int add_rule_item(ObUDRItem *rule_item); + int remove_rule_item(const int64_t rule_id); + int get_rule_item_by_id(const int64_t rule_id, ObUDRItem *&rule_item); + const RuleItemList &get_rule_item_list() const { return rule_item_list_; } + + TO_STRING_KV(K_(rule_item_list)); + common::ObIAllocator &allocator_; + int64_t ref_count_; + UDRKey rule_key_; + common::SpinRWLock ref_lock_; + RuleItemList rule_item_list_; + }; + + typedef common::hash::ObHashMap RuleKeyNodeMap; + typedef UDRRefObjGuard UDRItemRefGuard; + + ObUDRItemMgr() + : inited_(false), + allocator_(NULL), + tenant_id_(OB_INVALID_ID) {} + ~ObUDRItemMgr(); + void destroy(); + int reuse(); + int init(uint64_t tenant_id, common::ObIAllocator &allocator); + int sync_local_cache_rules(const ObIArray& rule_infos); + int get_udr_item(const ObUDRContext &rule_ctx, + UDRItemRefGuard &item_guard, + PatternConstConsList *cst_cons_list = NULL); + int fuzzy_check_by_pattern_digest(const uint64_t pattern_digest, + bool &is_exists); + +private: + template + int alloc(UDRRefObjGuard &guard, Args&& ...args); + int get_value(const UDRKey &rule_key, + UDRKeyNodePair *&rule_node, + ObUDRAtomicOp &op); + int erase_item(const UDRKey &rule_key, const int64_t rule_id); + int add_item(const UDRKey &rule_key, const ObUDRInfo &rule_info); + int add_rule_node(const UDRKey &rule_key, UDRKeyNodePair *rule_node); + int remove_rule_node(const UDRKey &rule_key); + int match_rule_item(const ObUDRContext &rule_ctx, + const ObUDRItem *rule_item, + bool &is_match) const; + int match_fixed_param_list(const ObUDRContext &rule_ctx, + const ObUDRItem *rule_item, + bool &is_match) const; + int get_rule_item_by_id(const UDRKey &rule_key, + const int64_t rule_id, + ObUDRItem *&rule_item); + int cons_deep_copy_rule_item(const ObUDRInfo &rule_info, + ObUDRItem &rule_item); + int get_all_cst_cons_by_key(const UDRKey &rule_key, + PatternConstConsList &cst_cons_list); + template + void cons_shallow_copy_rule_key(const Info &rule_info, + UDRKey &rule_key) const; + +private: + bool inited_; + common::ObIAllocator *allocator_; + uint64_t tenant_id_; + RuleKeyNodeMap rule_key_node_map_; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRItemMgr); +}; +} // namespace sql end +} // namespace oceanbase end + +#endif \ No newline at end of file diff --git a/src/sql/udr/ob_udr_mgr.cpp b/src/sql/udr/ob_udr_mgr.cpp new file mode 100644 index 000000000..0d072ea70 --- /dev/null +++ b/src/sql/udr/ob_udr_mgr.cpp @@ -0,0 +1,264 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#define USING_LOG_PREFIX SQL_QRR +#include "lib/oblog/ob_log.h" +#include "lib/oblog/ob_log_module.h" +#include "lib/thread/thread_mgr.h" +#include "lib/rc/ob_rc.h" +#include "lib/allocator/ob_malloc.h" +#include "lib/alloc/alloc_func.h" +#include "observer/ob_server.h" +#include "share/ob_define.h" +#include "lib/time/ob_time_utility.h" +#include "sql/udr/ob_udr_mgr.h" + +namespace oceanbase +{ +using namespace observer; +using namespace common; +namespace sql +{ + +void UDRBackupRecoveryGuard::backup() +{ + is_prepare_protocol_ = sql_ctx_.is_prepare_protocol_; + cur_sql_ = sql_ctx_.cur_sql_; +} + +void UDRBackupRecoveryGuard::recovery() +{ + sql_ctx_.is_prepare_protocol_ = is_prepare_protocol_; + sql_ctx_.cur_sql_ = cur_sql_; + pc_ctx_.is_rewrite_sql_ = false; + pc_ctx_.def_name_ctx_ = nullptr; + if (!is_prepare_protocol_) { + pc_ctx_.is_ps_mode_ = false; + const_cast(pc_ctx_.raw_sql_) = cur_sql_; + pc_ctx_.is_ps_execute_stage_ = false; + pc_ctx_.fp_result_.ps_params_.reuse(); + } +} + +UDRTmpAllocatorGuard::~UDRTmpAllocatorGuard() +{ + if (inited_) { + DESTROY_CONTEXT(mem_context_); + } +} + +int UDRTmpAllocatorGuard::init(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + lib::ContextParam param; + param.set_mem_attr(tenant_id, "QueryRewriteSql", ObCtxIds::DEFAULT_CTX_ID) + .set_properties(lib::USE_TL_PAGE_OPTIONAL) + .set_page_size(OB_MALLOC_NORMAL_BLOCK_SIZE); + if (OB_FAIL(CURRENT_CONTEXT->CREATE_CONTEXT(mem_context_, param))) { + LOG_WARN("failed to create memory entity", K(ret)); + } else if (OB_ISNULL(mem_context_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to create memory entity", K(ret)); + } else { + inited_ = true; + } + return ret; +} + +ObIAllocator* UDRTmpAllocatorGuard::get_allocator() +{ + return inited_ ? &mem_context_->get_arena_allocator() : nullptr; +} + +void ObUDRRefreshTask::runTimerTask() +{ + int ret = OB_SUCCESS; + uint64_t data_version = 0; + if (OB_FAIL(GET_MIN_DATA_VERSION(rule_mgr_->tenant_id_, data_version))) { + LOG_WARN("failed to get min data version", K(ret), K(rule_mgr_->tenant_id_)); + } else if (data_version < DATA_VERSION_4_1_0_0) { + // do nothing + } else if (OB_NOT_NULL(rule_mgr_) && rule_mgr_->inited_) { + LOG_INFO("run rewrite rule refresh task", K(rule_mgr_->tenant_id_)); + if (OB_FAIL(rule_mgr_->sync_rule_from_inner_table())) { + LOG_WARN("failed to sync rule from inner table", K(ret)); + } + } +} + +ObUDRMgr::~ObUDRMgr() +{ + if (inited_) { + destroy(); + } +} + +int ObUDRMgr::init(uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + ObPCMemPctConf default_conf; + if (inited_) { + ret = OB_INIT_TWICE; + LOG_WARN("rewrite rule mgr init twice", K(ret)); + } else if (OB_FAIL(sql_service_.init(&ObServer::get_instance().get_mysql_proxy()))) { + LOG_WARN("failed to init rewrite rule sql service", K(ret)); + } else if (OB_FAIL(TG_CREATE_TENANT(lib::TGDefIDs::ReqMemEvict, tg_id_))) { + LOG_WARN("failed to create tg", K(ret)); + } else if (OB_FAIL(TG_START(tg_id_))) { + LOG_WARN("failed to start tg", K(ret)); + } else if (OB_FAIL(init_mem_context(tenant_id))) { + LOG_WARN("failed to init mem context", K(ret)); + } else if (OB_FAIL(rule_item_mgr_.init(tenant_id, *inner_allocator_))) { + LOG_WARN("failed to init rule item manager", K(ret)); + } else { + refresh_task_.rule_mgr_ = this; + if (OB_FAIL(TG_SCHEDULE(tg_id_, refresh_task_, ObUDRRefreshTask::REFRESH_INTERVAL, true))) { + LOG_WARN("failed to schedule refresh task", K(ret)); + } else { + tenant_id_ = tenant_id; + inited_ = true; + destroyed_ = false; + } + } + + if (OB_FAIL(ret) && !inited_) { + destroy(); + } + return ret; +} + +int ObUDRMgr::init_mem_context(uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + if (OB_LIKELY(NULL == mem_context_)) { + lib::ContextParam param; + param.set_mem_attr(tenant_id, "RewriteRuleMgr", common::ObCtxIds::DEFAULT_CTX_ID); + if (OB_FAIL(ROOT_CONTEXT->CREATE_CONTEXT(mem_context_, param))) { + SQL_ENG_LOG(WARN, "create entity failed", K(ret)); + } else if (OB_ISNULL(mem_context_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("mem entity is null", K(ret)); + } else { + inner_allocator_ = &mem_context_->get_malloc_allocator(); + } + } + return ret; +} + +int ObUDRMgr::mtl_init(ObUDRMgr* &node_list) +{ + int ret = OB_SUCCESS; + uint64_t tenant_id = lib::current_resource_owner_id(); + if (OB_FAIL(node_list->init(tenant_id))) { + LOG_WARN("failed to init event list", K(ret)); + } + return ret; +} + +void ObUDRMgr::destroy() +{ + if (!destroyed_) { + inited_ = false; + destroyed_ = true; + TG_DESTROY(tg_id_); + rule_item_mgr_.destroy(); + if (OB_LIKELY(nullptr != inner_allocator_)) { + inner_allocator_ = nullptr; + } + if (OB_LIKELY(nullptr != mem_context_)) { + DESTROY_CONTEXT(mem_context_); + mem_context_ = nullptr; + } + } +} + +int ObUDRMgr::insert_rule(ObUDRInfo &arg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(sql_service_.insert_rule(arg))) { + LOG_WARN("failed to create rewrite rule", K(ret), K(arg)); + } + return ret; +} + +int ObUDRMgr::alter_rule_status(ObUDRInfo &arg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(sql_service_.alter_rule_status(arg))) { + LOG_WARN("failed to alter rewrite rule status", K(ret), K(arg)); + } + return ret; +} + +int ObUDRMgr::remove_rule(ObUDRInfo &arg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(sql_service_.remove_rule(arg))) { + LOG_WARN("failed to remove rewrite rule", K(ret), K(arg)); + } + return ret; +} + +int ObUDRMgr::get_udr_item(const ObUDRContext &rule_ctx, + ObUDRItemMgr::UDRItemRefGuard &item_guard, + PatternConstConsList *cst_cons_list) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(rule_item_mgr_.get_udr_item(rule_ctx, item_guard, cst_cons_list))) { + LOG_DEBUG("failed to get rewrite rule item", K(ret), K(rule_ctx)); + } + return ret; +} + +int ObUDRMgr::fuzzy_check_by_pattern_digest(const uint64_t pattern_digest, bool &is_exists) +{ + int ret = OB_SUCCESS; + is_exists = false; + if (OB_FAIL(rule_item_mgr_.fuzzy_check_by_pattern_digest(pattern_digest, is_exists))) { + LOG_WARN("failed to fuzzy check by pattern digest", K(ret), K(pattern_digest)); + } + return ret; +} + +int ObUDRMgr::sync_rule_from_inner_table() +{ + int ret = OB_SUCCESS; + lib::ObMutexGuard guard(mutex_); + LOG_DEBUG("sync rule from inner table", K(tenant_id_), K(rule_version_)); + int64_t max_rule_version = OB_INVALID_VERSION; + ObSEArray rule_infos; + UDRTmpAllocatorGuard alloc_guard; + ObIAllocator* allocator = nullptr; + if (OB_FAIL(sql_service_.fetch_max_rule_version(tenant_id_, max_rule_version))) { + LOG_WARN("failed to fetch max rule version", K(ret)); + } else if (rule_version_ >= max_rule_version) { + LOG_TRACE("local version is latest, don't need refresh", K(tenant_id_), K(rule_version_), K(max_rule_version)); + } else if (OB_FAIL(alloc_guard.init(tenant_id_))) { + LOG_WARN("failed to init allocator guard", K(ret)); + } else if (OB_ISNULL(allocator = alloc_guard.get_allocator())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get null allocator", K(ret)); + } else { + LOG_INFO("local version is not latest, need refresh", K(tenant_id_), K(rule_version_), K(max_rule_version)); + if (OB_FAIL(sql_service_.clean_up_items_marked_for_deletion(tenant_id_))) { + LOG_WARN("failed to clean up items marked for deletion", K(ret)); + } else if (OB_FAIL(sql_service_.get_need_sync_rule_infos(*allocator, tenant_id_, rule_version_, rule_infos))) { + LOG_WARN("failed to get need sync rule infos", K(ret), K(rule_version_), K(max_rule_version)); + } else if (rule_infos.empty()) { + // do nothing + LOG_TRACE("rule infos is empty", K(tenant_id_), K(rule_version_), K(max_rule_version)); + } else if (OB_FAIL(rule_item_mgr_.sync_local_cache_rules(rule_infos))) { + LOG_WARN("failed to sync local cache rules", K(ret)); + } else { + set_rule_version(max_rule_version); + } + } + return ret; +} + +} // namespace sql end +} // namespace oceanbase end diff --git a/src/sql/udr/ob_udr_mgr.h b/src/sql/udr/ob_udr_mgr.h new file mode 100644 index 000000000..becdc8e0a --- /dev/null +++ b/src/sql/udr/ob_udr_mgr.h @@ -0,0 +1,121 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_MGR_H_ +#define OB_SQL_UDR_OB_UDR_MGR_H_ +#include "sql/udr/ob_udr_sql_service.h" +#include "sql/udr/ob_udr_item_mgr.h" +#include "lib/task/ob_timer.h" + + +namespace oceanbase +{ +namespace sql +{ +class ObUDRMgr; + +class UDRBackupRecoveryGuard +{ +public: + UDRBackupRecoveryGuard(ObSqlCtx &sql_ctx, ObPlanCacheCtx &pc_ctx) + : sql_ctx_(sql_ctx), + pc_ctx_(pc_ctx), + is_prepare_protocol_(false), + cur_sql_() + { + backup(); + } + + ~UDRBackupRecoveryGuard() + { + recovery(); + } + void backup(); + void recovery(); + +private: + ObSqlCtx &sql_ctx_; + ObPlanCacheCtx &pc_ctx_; + bool is_prepare_protocol_; + common::ObString cur_sql_; +}; + +class UDRTmpAllocatorGuard +{ +public: + UDRTmpAllocatorGuard() + : mem_context_(nullptr), + inited_(false) {} + ~UDRTmpAllocatorGuard(); + int init(const uint64_t tenant_id); + ObIAllocator* get_allocator(); +private: + lib::MemoryContext mem_context_; + bool inited_; +}; + +class ObUDRRefreshTask : public common::ObTimerTask +{ +public: + ObUDRRefreshTask() : rule_mgr_(NULL) {} + void runTimerTask(void); + +public: + const static int64_t REFRESH_INTERVAL = 5L * 1000L * 1000L; // 5s + ObUDRMgr* rule_mgr_; +}; + +class ObUDRMgr +{ + friend class ObUDRRefreshTask; +public: + ObUDRMgr() + : mutex_(), + inner_allocator_(NULL), + inited_(false), + destroyed_(false), + tenant_id_(OB_INVALID_ID), + tg_id_(-1), + rule_version_(OB_INIT_REWRITE_RULE_VERSION) {} + ~ObUDRMgr(); + static int mtl_init(ObUDRMgr* &node_list); + void destroy(); + int insert_rule(ObUDRInfo &arg); + int remove_rule(ObUDRInfo &arg); + int alter_rule_status(ObUDRInfo &arg); + int sync_rule_from_inner_table(); + int get_udr_item(const ObUDRContext &rule_ctx, + ObUDRItemMgr::UDRItemRefGuard &item_guard, + PatternConstConsList *cst_cons_list = NULL); + int fuzzy_check_by_pattern_digest(const uint64_t pattern_digest, bool &is_exists); + void set_rule_version(const int64_t version) { ATOMIC_STORE(&rule_version_, version); } + int64_t get_rule_version() const { return ATOMIC_LOAD(&rule_version_); } + +private: + int init(uint64_t tenant_id); + int init_mem_context(uint64_t tenant_id); + +private: + lib::ObMutex mutex_; + lib::MemoryContext mem_context_; + common::ObIAllocator* inner_allocator_; + ObUDRSqlService sql_service_; + ObUDRRefreshTask refresh_task_; + ObUDRItemMgr rule_item_mgr_; + bool inited_; + bool destroyed_; + uint64_t tenant_id_; + int tg_id_; + int64_t rule_version_; +private: + DISALLOW_COPY_AND_ASSIGN(ObUDRMgr); +}; + + +} // namespace sql end +} // namespace oceanbase end +#endif diff --git a/src/sql/udr/ob_udr_sql_service.cpp b/src/sql/udr/ob_udr_sql_service.cpp new file mode 100644 index 000000000..c4b38ac10 --- /dev/null +++ b/src/sql/udr/ob_udr_sql_service.cpp @@ -0,0 +1,370 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + +#define USING_LOG_PREFIX SQL_QRR +#include "share/inner_table/ob_inner_table_schema_constants.h" +#include "share/schema/ob_schema_utils.h" +#include "lib/oblog/ob_log.h" +#include "lib/oblog/ob_log_module.h" +#include "lib/string/ob_sql_string.h" +#include "lib/mysqlclient/ob_mysql_proxy.h" +#include "lib/mysqlclient/ob_mysql_connection.h" +#include "lib/mysqlclient/ob_mysql_statement.h" +#include "lib/mysqlclient/ob_mysql_connection_pool.h" +#include "lib/utility/ob_print_utils.h" +#include "lib/mysqlclient/ob_mysql_transaction.h" +#include "share/ob_dml_sql_splicer.h" +#include "share/ob_max_id_fetcher.h" +#include "observer/ob_sql_client_decorator.h" +#include "sql/resolver/expr/ob_raw_expr_printer.h" +#include "sql/udr/ob_udr_sql_service.h" + +namespace oceanbase +{ +using namespace share; +using namespace share::schema; +using namespace common::sqlclient; +namespace sql +{ + +#define UDR_START_TRANS(tenant_id) \ +ObMySQLTransaction trans; \ +if (OB_FAIL(ret)) { \ +} else if (OB_ISNULL(sql_proxy_)) { \ + ret = OB_ERR_UNEXPECTED; \ + LOG_WARN("get unexpected null", K(ret), K(sql_proxy_)); \ +} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id))) { \ + LOG_WARN("fail to start transaction", K(ret)); \ +} \ + +#define UDR_END_TRANS \ +if (trans.is_started()) { \ + const bool is_commit = (OB_SUCCESS == ret); \ + int tmp_ret = trans.end(is_commit); \ + if (OB_SUCCESS != tmp_ret) { \ + LOG_WARN("end trans failed", K(tmp_ret), K(is_commit)); \ + ret = (OB_SUCCESS == ret) ? tmp_ret : ret; \ + } \ +} \ + +int ObUDRSqlService::init(ObMySQLProxy *proxy) +{ + int ret = OB_SUCCESS; + if (inited_) { + ret = OB_INIT_TWICE; + LOG_WARN("rewrite rule sql service init twice", K(ret)); + } else if (OB_ISNULL(proxy)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get null mysql proxy", K(ret)); + } else { + sql_proxy_ = proxy; + inited_ = true; + } + return ret; +} + +int ObUDRSqlService::gen_insert_rule_dml(const ObUDRInfo &arg, + const uint64_t tenant_id, + oceanbase::share::ObDMLSqlSplicer &dml) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(dml.add_pk_column("tenant_id", ObSchemaUtils::get_extract_tenant_id(tenant_id, + arg.tenant_id_))) + || OB_FAIL(dml.add_pk_column("rule_name", ObHexEscapeSqlStr(arg.rule_name_))) + || OB_FAIL(dml.add_column("rule_id", arg.rule_id_)) + || OB_FAIL(dml.add_column("pattern", ObHexEscapeSqlStr(arg.pattern_))) + || OB_FAIL(dml.add_column("db_name", ObHexEscapeSqlStr(arg.db_name_))) + || OB_FAIL(dml.add_column("replacement", ObHexEscapeSqlStr(arg.replacement_))) + || OB_FAIL(dml.add_column("normalized_pattern", ObHexEscapeSqlStr(arg.normalized_pattern_))) + || OB_FAIL(dml.add_column("status", static_cast(arg.rule_status_))) + || OB_FAIL(dml.add_column("version", arg.rule_version_)) + || OB_FAIL(dml.add_uint64_column("pattern_digest", arg.pattern_digest_)) + || OB_FAIL(dml.add_column("fixed_param_infos", ObHexEscapeSqlStr(arg.fixed_param_infos_str_.empty() ? + ObString("") : arg.fixed_param_infos_str_))) + || OB_FAIL(dml.add_column("dynamic_param_infos", ObHexEscapeSqlStr(arg.dynamic_param_infos_str_.empty() ? + ObString("") : arg.dynamic_param_infos_str_))) + || OB_FAIL(dml.add_column("def_name_ctx_str", ObHexEscapeSqlStr(arg.question_mark_ctx_str_.empty() ? + ObString("") : arg.question_mark_ctx_str_)))) { + LOG_WARN("add column failed", K(ret), K(arg)); + } + return ret; +} + +int ObUDRSqlService::gen_modify_rule_status_dml(const ObUDRInfo &arg, + const uint64_t tenant_id, + oceanbase::share::ObDMLSqlSplicer &dml) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(dml.add_pk_column("tenant_id", ObSchemaUtils::get_extract_tenant_id(tenant_id, + arg.tenant_id_))) + || OB_FAIL(dml.add_pk_column("rule_name", ObHexEscapeSqlStr(arg.rule_name_))) + || OB_FAIL(dml.add_column("version", arg.rule_version_)) + || OB_FAIL(dml.add_column("status", static_cast(arg.rule_status_)))) { + LOG_WARN("modify column failed", K(ret), K(arg)); + } + return ret; +} + +int ObUDRSqlService::fetch_new_rule_version(const uint64_t tenant_id, + int64_t &new_rule_version) +{ + int ret = OB_SUCCESS; + uint64_t rule_version = OB_INVALID_VERSION; + ObMaxIdFetcher id_fetcher(*sql_proxy_); + if (OB_FAIL(id_fetcher.fetch_new_max_id(tenant_id, + OB_MAX_USED_REWRITE_RULE_VERSION_TYPE, + rule_version, OB_INIT_REWRITE_RULE_VERSION))) { + LOG_WARN("fetch_new_max_id failed", K(ret), "id_type", OB_MAX_USED_REWRITE_RULE_VERSION_TYPE); + } else { + new_rule_version = rule_version; + } + return ret; +} + +int ObUDRSqlService::fetch_max_rule_version(const uint64_t tenant_id, + int64_t &max_rule_version) +{ + int ret = OB_SUCCESS; + uint64_t rule_version = OB_INVALID_VERSION; + ObMaxIdFetcher id_fetcher(*sql_proxy_); + if (OB_FAIL(id_fetcher.fetch_max_id(*sql_proxy_, tenant_id, + OB_MAX_USED_REWRITE_RULE_VERSION_TYPE, rule_version))) { + LOG_WARN("failed to fetch max rule version", K(ret), K(tenant_id)); + } else { + max_rule_version = rule_version; + } + return ret; +} + +int ObUDRSqlService::fetch_new_rule_id(const uint64_t tenant_id, + int64_t &new_rule_id) +{ + int ret = OB_SUCCESS; + uint64_t rule_id = OB_INVALID_ID; + ObMaxIdFetcher id_fetcher(*sql_proxy_); + if (OB_FAIL(id_fetcher.fetch_new_max_id(tenant_id, + OB_MAX_USED_OBJECT_ID_TYPE, + rule_id))) { + LOG_WARN("fetch_new_max_id failed", K(ret), "id_type", OB_MAX_USED_OBJECT_ID_TYPE); + } else { + new_rule_id = rule_id; + } + return ret; +} + +int ObUDRSqlService::insert_rule(ObUDRInfo &arg) +{ + int ret = OB_SUCCESS; + ObDMLSqlSplicer dml; + int64_t affected_rows = 0; + int64_t rule_id = OB_INVALID_ID; + int64_t rule_version = OB_INVALID_VERSION; + UDR_START_TRANS(arg.tenant_id_); + LOG_INFO("insert rule", K(arg)); + ObDMLExecHelper exec(trans, arg.tenant_id_); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(fetch_new_rule_version(arg.tenant_id_, rule_version))) { + LOG_WARN("failed to fetch new rule version", K(ret)); + } else if (FALSE_IT(arg.rule_version_ = rule_version)) { + } else if (OB_FAIL(fetch_new_rule_id(arg.tenant_id_, rule_id))) { + LOG_WARN("failed to fetch new rule id", K(ret)); + } else if (FALSE_IT(arg.rule_id_ = rule_id)) { + } else if (OB_FAIL(gen_insert_rule_dml(arg, arg.tenant_id_, dml))) { + LOG_WARN("failed to gen rewrite rule dml", K(ret)); + } else if (OB_FAIL(exec.exec_insert(OB_ALL_TENANT_REWRITE_RULES_TNAME, dml, affected_rows))) { + LOG_WARN("execute insert failed", K(ret)); + } else if (!is_single_row(affected_rows)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("affected_rows unexpected to be one", K(affected_rows), K(ret)); + } + UDR_END_TRANS; + return ret; +} + +int ObUDRSqlService::alter_rule_status(ObUDRInfo &arg) +{ + int ret = OB_SUCCESS; + UDR_START_TRANS(arg.tenant_id_); + ObDMLSqlSplicer dml; + int64_t affected_rows = 0; + ObDMLExecHelper exec(trans, arg.tenant_id_); + int64_t rule_version = OB_INVALID_VERSION; + LOG_INFO("alter rule status", K(arg)); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(fetch_new_rule_version(arg.tenant_id_, rule_version))) { + LOG_WARN("failed to fetch new rule version", K(ret)); + } else if (FALSE_IT(arg.rule_version_ = rule_version)) { + } else if (OB_FAIL(gen_modify_rule_status_dml(arg, arg.tenant_id_, dml))) { + LOG_WARN("failed to gen rewrite rule dml", K(ret)); + } else if (OB_FAIL(exec.exec_update(OB_ALL_TENANT_REWRITE_RULES_TNAME, dml, affected_rows))) { + LOG_WARN("execute insert failed", K(ret)); + } else if (!is_single_row(affected_rows) && !is_zero_row(affected_rows)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("affected_rows unexpected", K(affected_rows), K(ret)); + } + UDR_END_TRANS; + return ret; +} + +int ObUDRSqlService::gen_recyclebin_rule_name(const int64_t rule_version, + const int64_t buf_len, + char *buf, + ObString &recyclebin_rule_name) +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "__recyclebin_"))) { + LOG_WARN("append name to buf error", K(ret)); + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%ld", rule_version))) { + LOG_WARN("append name to buf error", K(ret)); + } else { + recyclebin_rule_name.assign_ptr(buf, pos); + } + return ret; +} + +int ObUDRSqlService::remove_rule(ObUDRInfo &arg) +{ + int ret = OB_SUCCESS; + UDR_START_TRANS(arg.tenant_id_); + ObSqlString sql; + ObSqlString rule_name_sql_str; + int64_t affected_rows = 0; + int64_t rule_version = OB_INVALID_VERSION; + ObString recyclebin_rule_name; + char buf[OB_MAX_ORIGINAL_NANE_LENGTH] = {0}; + const int64_t buf_len = OB_MAX_ORIGINAL_NANE_LENGTH; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(fetch_new_rule_version(arg.tenant_id_, rule_version))) { + LOG_WARN("failed to fetch new rule version", K(ret)); + } else if (OB_FAIL(gen_recyclebin_rule_name(rule_version, buf_len, buf, recyclebin_rule_name))) { + LOG_WARN("failed to gen recyclebin rule name", K(ret)); + } else if (OB_FAIL(sql_append_hex_escape_str(arg.rule_name_, rule_name_sql_str))) { + LOG_WARN("failed to process rule name", K(ret)); + } else if (sql.assign_fmt("update %s set gmt_modified = now(), rule_name = '%.*s' ,\ + version = %ld, status = %ld \ + where tenant_id = %lu and rule_name = %.*s", + OB_ALL_TENANT_REWRITE_RULES_TNAME, + LEN_AND_PTR(recyclebin_rule_name), + rule_version, + static_cast(ObUDRInfo::DELETE_STATUS), + ObSchemaUtils::get_extract_tenant_id(arg.tenant_id_, arg.tenant_id_), + LEN_AND_PTR(rule_name_sql_str.string()))) { + LOG_WARN("update from __all_tenant_rewrite_rules table failed.", K(ret)); + } else if (OB_FAIL(trans.write(arg.tenant_id_, sql.ptr(), affected_rows))) { + LOG_WARN("fail to exec sql", K(ret), K(sql)); + } else if (!is_single_row(affected_rows) && !is_zero_row(affected_rows)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("affected_rows unexpected", K(affected_rows), K(ret), K(sql)); + } + UDR_END_TRANS; + return ret; +} + +int ObUDRSqlService::clean_up_items_marked_for_deletion(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + UDR_START_TRANS(tenant_id); + ObSqlString sql; + int64_t affected_rows = 0; + if (OB_FAIL(ret)) { + } else if (sql.assign_fmt("delete FROM %s WHERE status = %ld \ + AND DATEDIFF(now(), gmt_modified) >= %ld", + OB_ALL_TENANT_REWRITE_RULES_TNAME, + static_cast(ObUDRInfo::DELETE_STATUS), + DELETE_DATE_INTERVAL_THRESHOLD)) { + LOG_WARN("delete from __all_tenant_rewrite_rules table failed.", K(ret)); + } else if (OB_FAIL(trans.write(tenant_id, sql.ptr(), affected_rows))) { + LOG_WARN("fail to exec sql", K(ret)); + } else { + LOG_INFO("affected rows", K(affected_rows)); + } + UDR_END_TRANS; + return ret; +} + +int ObUDRSqlService::get_need_sync_rule_infos(ObIAllocator& allocator, + const uint64_t tenant_id, + const int64_t local_rule_version, + ObIArray& rule_infos) +{ + int ret = OB_SUCCESS; + ObSQLClientRetryWeak sql_client_retry_weak(sql_proxy_); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + sqlclient::ObMySQLResult *result = NULL; + ObSqlString sql; + const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id); + int64_t affected_rows = 0; + const char* sql_fmt = "select rule_id, rule_name, pattern, db_name, replacement, normalized_pattern, " + "status, version, pattern_digest, fixed_param_infos, " + "dynamic_param_infos, def_name_ctx_str " + "from %s where tenant_id = %lu and version > %ld;"; + if (OB_ISNULL(sql_proxy_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(sql.assign_fmt(sql_fmt, OB_ALL_TENANT_REWRITE_RULES_TNAME, + ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id), + local_rule_version))) { + LOG_WARN("failed to assign format sql", K(ret)); + } else if (OB_FAIL(sql_client_retry_weak.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("execute sql failed", "sql", sql.ptr(), K(ret)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to execute ", "sql", sql.ptr(), K(ret)); + } else { + while (OB_SUCC(result->next())) { + ObUDRInfo rule_info; + rule_info.tenant_id_ = tenant_id; + EXTRACT_INT_FIELD_MYSQL(*result, "status", rule_info.rule_status_, ObUDRInfo::RuleStatus); + EXTRACT_INT_FIELD_MYSQL(*result, "rule_id", rule_info.rule_id_, int64_t); + EXTRACT_INT_FIELD_MYSQL(*result, "version", rule_info.rule_version_, int64_t); + EXTRACT_UINT_FIELD_MYSQL(*result, "pattern_digest", rule_info.pattern_digest_, uint64_t); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "rule_name", rule_info.rule_name_); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "pattern", rule_info.pattern_); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "db_name", rule_info.db_name_); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "replacement", rule_info.replacement_); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "normalized_pattern", rule_info.normalized_pattern_); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "fixed_param_infos", rule_info.fixed_param_infos_str_); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "dynamic_param_infos", rule_info.dynamic_param_infos_str_); + EXTRACT_VARCHAR_FIELD_MYSQL(*result, "def_name_ctx_str", rule_info.question_mark_ctx_str_); + + if (OB_SUCC(ret)) { + if (OB_FAIL(ob_write_string(allocator, rule_info.rule_name_, rule_info.rule_name_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(ob_write_string(allocator, rule_info.pattern_, rule_info.pattern_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(ob_write_string(allocator, rule_info.db_name_, rule_info.db_name_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(ob_write_string(allocator, rule_info.replacement_, rule_info.replacement_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(ob_write_string(allocator, rule_info.normalized_pattern_, rule_info.normalized_pattern_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(ob_write_string(allocator, rule_info.fixed_param_infos_str_, rule_info.fixed_param_infos_str_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(ob_write_string(allocator, rule_info.dynamic_param_infos_str_, rule_info.dynamic_param_infos_str_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(ob_write_string(allocator, rule_info.question_mark_ctx_str_, rule_info.question_mark_ctx_str_))) { + LOG_WARN("failed to write string"); + } else if (OB_FAIL(rule_infos.push_back(rule_info))) { + LOG_WARN("failed to push back rule info", K(ret)); + } + } + } + } + if (OB_ITER_END != ret) { + LOG_WARN("get next row failed", K(ret)); + } else { + ret = OB_SUCCESS; + } + } + return ret; +} + +#undef UDR_START_TRANS +#undef UDR_END_TRANS + +} // namespace oceanbase end +} // namespace sql end diff --git a/src/sql/udr/ob_udr_sql_service.h b/src/sql/udr/ob_udr_sql_service.h new file mode 100644 index 000000000..da72554dd --- /dev/null +++ b/src/sql/udr/ob_udr_sql_service.h @@ -0,0 +1,72 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + +#ifndef OB_SQL_UDR_OB_UDR_SQL_SERVICE_H_ +#define OB_SQL_UDR_OB_UDR_SQL_SERVICE_H_ +#include "lib/mysqlclient/ob_mysql_result.h" +#include "sql/udr/ob_udr_struct.h" + +namespace oceanbase +{ +namespace common +{ +class ObMySQLProxy; +} +namespace share +{ +class ObDMLSqlSplicer; +} + +namespace sql +{ + +class ObUDRSqlService +{ +public: + ObUDRSqlService() + : inited_(false), + sql_proxy_(nullptr) + {} + ~ObUDRSqlService() {} + int init(ObMySQLProxy *proxy); + int insert_rule(ObUDRInfo &arg); + int remove_rule(ObUDRInfo &arg); + int alter_rule_status(ObUDRInfo &arg); + int fetch_max_rule_version(const uint64_t tenant_id, + int64_t &max_rule_version); + int get_need_sync_rule_infos(ObIAllocator& allocator, + const uint64_t tenant_id, + const int64_t local_rule_version, + ObIArray& rule_infos); + int clean_up_items_marked_for_deletion(const uint64_t tenant_id); + +private: + int fetch_new_rule_id(const uint64_t tenant_id, + int64_t &new_rule_id); + int fetch_new_rule_version(const uint64_t tenant_id, + int64_t &new_rule_version); + int gen_insert_rule_dml(const ObUDRInfo &arg, + const uint64_t tenant_id, + oceanbase::share::ObDMLSqlSplicer &dml); + int gen_modify_rule_status_dml(const ObUDRInfo &arg, + const uint64_t tenant_id, + oceanbase::share::ObDMLSqlSplicer &dml); + int gen_recyclebin_rule_name(const int64_t rule_version, + const int64_t buf_len, + char *buf, + ObString &recyclebin_rule_name); + +private: + // Trigger delete DATE interval threshold + static const int64_t DELETE_DATE_INTERVAL_THRESHOLD = 2; + bool inited_; + ObMySQLProxy* sql_proxy_; +}; + +} // namespace sql end +} // namespace oceanbase end + +#endif diff --git a/src/sql/udr/ob_udr_struct.cpp b/src/sql/udr/ob_udr_struct.cpp new file mode 100644 index 000000000..f387749cc --- /dev/null +++ b/src/sql/udr/ob_udr_struct.cpp @@ -0,0 +1,124 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#define USING_LOG_PREFIX SQL_QRR +#include "sql/udr/ob_udr_struct.h" + +namespace oceanbase +{ +namespace sql +{ + +OB_SERIALIZE_MEMBER(FixedParamValue, idx_, raw_text_); +OB_SERIALIZE_MEMBER(DynamicParamInfo, raw_param_idx_, question_mark_idx_); + +DEFINE_SERIALIZE(QuestionMarkDefNameCtx) +{ + int ret = OB_SUCCESS; + const int64_t serialize_size = get_serialize_size(); + if (OB_ISNULL(buf) || OB_UNLIKELY(serialize_size > buf_len - pos)) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("size overflow", K(ret), KP(buf), K(serialize_size), "remain", buf_len - pos); + } else if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, count_))) { + LOG_WARN("failed to decode count", K(ret)); + } else { + for (int64_t i = 0; i < count_ && OB_SUCC(ret); ++i) { + char *ptr = name_[i]; + const int64_t len = strlen(ptr); + if (OB_FAIL(serialization::encode_vstr(buf, buf_len, pos, ptr, len))) { + LOG_WARN("string serialize failed", K(ret)); + } + } + } + return ret; +} + +DEFINE_DESERIALIZE(QuestionMarkDefNameCtx) +{ + int ret = OB_SUCCESS; + const int64_t min_needed_size = 1; + if (OB_ISNULL(buf) || OB_UNLIKELY((data_len - pos) < min_needed_size)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), KP(buf), "remain", data_len - pos); + } else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &count_))) { + LOG_WARN("failed to encode count", K(data_len), K(pos), K(count_), K(ret)); + } else if (OB_ISNULL(name_ = (char **)allocator_.alloc(sizeof(char*) * count_))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("alloc memory failed", K(ret)); + } else { + for (int64_t i = 0; i < count_ && OB_SUCC(ret); ++i) { + char *ptr = nullptr; + int64_t len = 0; + ptr = const_cast(common::serialization::decode_vstr(buf, data_len, pos, &len)); + if (OB_ISNULL(ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("decode NULL string", K(ret)); + } else { + name_[i] = ptr; + } + } + } + return ret; +} + +DEFINE_GET_SERIALIZE_SIZE(QuestionMarkDefNameCtx) +{ + int64_t size = 0; + size += serialization::encoded_length_vi64(count_); + for (int64_t i = 0; i < count_; ++i) { + const int64_t len = strlen(name_[i]); + size += serialization::encoded_length_vstr(len); + } + return size; +} + +int QuestionMarkDefNameCtx::assign(const char **name, const int64_t count) +{ + int ret = OB_SUCCESS; + count_ = count; + if (OB_ISNULL(name_ = (char **)allocator_.alloc(sizeof(char*) * count))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("alloc memory failed", K(ret)); + } else { + for (int64_t i = 0; i < count && OB_SUCC(ret); ++i) { + char *ptr = nullptr; + const int32_t len = sizeof(strlen(name[i])); + if (OB_ISNULL(ptr = (char *)allocator_.alloc(len))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("alloc memory failed", K(ret)); + } else { + MEMCPY(ptr, name[i], len); + ptr[len] = '\0'; + name_[i] = ptr; + } + } + } + return ret; +} + +int ObUDRInfo::assign(const ObUDRInfo &other) +{ + int ret = OB_SUCCESS; + tenant_id_ = other.tenant_id_; + pattern_digest_ = other.pattern_digest_; + rule_id_ = other.rule_id_; + rule_version_ = other.rule_version_; + rule_status_ = other.rule_status_; + coll_type_ = other.coll_type_; + db_name_ = other.db_name_; + rule_name_ = other.rule_name_; + pattern_ = other.pattern_; + replacement_ = other.replacement_; + normalized_pattern_ = other.normalized_pattern_; + fixed_param_infos_str_ = other.fixed_param_infos_str_; + dynamic_param_infos_str_ = other.dynamic_param_infos_str_; + question_mark_ctx_str_ = other.question_mark_ctx_str_; + return ret; +} + +} // namespace sql end +} // namespace oceanbase end diff --git a/src/sql/udr/ob_udr_struct.h b/src/sql/udr/ob_udr_struct.h new file mode 100644 index 000000000..99e567c13 --- /dev/null +++ b/src/sql/udr/ob_udr_struct.h @@ -0,0 +1,155 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_STRUCT_H_ +#define OB_SQL_UDR_OB_UDR_STRUCT_H_ + +#include "lib/container/ob_se_array.h" +#include "lib/string/ob_string.h" + +namespace oceanbase +{ +namespace sql +{ + +struct FixedParamValue +{ + OB_UNIS_VERSION(1); +public: + int64_t idx_; + common::ObString raw_text_; + + FixedParamValue() + :idx_(common::OB_INVALID_ID) + {} + + void reset() + { + idx_ = common::OB_INVALID_ID; + raw_text_.reset(); + } + + TO_STRING_KV(K_(idx), K_(raw_text)); +}; + +// Information about the question mark, e.g: +// select 1, ? from dual; -> {{rw_param_idx: 1, question_mark_idx: 1}} +struct DynamicParamInfo +{ + OB_UNIS_VERSION(1); +public: + int64_t raw_param_idx_; + int64_t question_mark_idx_; + + DynamicParamInfo() + :raw_param_idx_(common::OB_INVALID_ID), + question_mark_idx_(common::OB_INVALID_ID) + {} + + void reset() + { + raw_param_idx_ = common::OB_INVALID_ID; + question_mark_idx_ = common::OB_INVALID_ID; + } + + TO_STRING_KV(K_(raw_param_idx), K_(question_mark_idx)); +}; + +struct QuestionMarkDefNameCtx +{ + common::ObIAllocator &allocator_; + char **name_; + int64_t count_; + + QuestionMarkDefNameCtx(common::ObIAllocator &allocator) + : allocator_(allocator), + name_(nullptr), + count_(0) + {} + int assign(const char **name, const int64_t count); + void reset() + { + if (nullptr != name_) { + allocator_.free(name_); + name_ = nullptr; + } + count_ = 0; + } + + TO_STRING_KV(K_(name), K_(count)); + NEED_SERIALIZE_AND_DESERIALIZE; +}; + +typedef common::ObSEArray FixedParamValueArray; +typedef common::ObSEArray DynamicParamInfoArray; +typedef common::ObSEArray PatternConstConsList; + +struct ObUDRInfo +{ +public: + enum RuleStatus + { + INVALID_STATUS = 0, + ENABLE_STATUS, + DISABLE_STATUS, + DELETE_STATUS, + MAX_STATUS, + }; + + ObUDRInfo() + : tenant_id_(OB_INVALID_ID), + pattern_digest_(0), + rule_id_(OB_INVALID_ID), + rule_version_(OB_INVALID_VERSION), + rule_status_(INVALID_STATUS), + coll_type_(common::CS_TYPE_INVALID), + db_name_(), + rule_name_(), + pattern_(), + replacement_(), + normalized_pattern_(), + fixed_param_infos_str_(), + dynamic_param_infos_str_(), + question_mark_ctx_str_() {} + virtual ~ObUDRInfo() {} + int assign(const ObUDRInfo &other); + + VIRTUAL_TO_STRING_KV(K_(tenant_id), + K_(pattern_digest), + K_(rule_id), + K_(rule_version), + K_(rule_status), + K_(coll_type), + K_(db_name), + K_(rule_name), + K_(pattern), + K_(replacement), + K_(normalized_pattern), + K_(fixed_param_infos_str), + K_(dynamic_param_infos_str), + K_(question_mark_ctx_str)); + + uint64_t tenant_id_; + uint64_t pattern_digest_; + int64_t rule_id_; + int64_t rule_version_; + RuleStatus rule_status_; + ObCollationType coll_type_; + common::ObString db_name_; + common::ObString rule_name_; + common::ObString pattern_; + common::ObString replacement_; + common::ObString normalized_pattern_; + common::ObString fixed_param_infos_str_; + common::ObString dynamic_param_infos_str_; + common::ObString question_mark_ctx_str_; +}; + +} // namespace sql end +} // namespace oceanbase end + +#endif \ No newline at end of file diff --git a/src/sql/udr/ob_udr_utils.cpp b/src/sql/udr/ob_udr_utils.cpp new file mode 100644 index 000000000..ee363a3af --- /dev/null +++ b/src/sql/udr/ob_udr_utils.cpp @@ -0,0 +1,273 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#define USING_LOG_PREFIX SQL_QRR +#include "sql/ob_sql.h" +#include "share/config/ob_server_config.h" +#include "sql/resolver/ob_resolver_utils.h" +#include "sql/udr/ob_udr_analyzer.h" +#include "sql/udr/ob_udr_mgr.h" +#include "sql/udr/ob_udr_utils.h" + +namespace oceanbase +{ +namespace sql +{ + +int ObUDRUtils::match_udr_item(const ObString &pattern, + const ObSQLSessionInfo &session_info, + ObIAllocator &allocator, + ObUDRItemMgr::UDRItemRefGuard &guard, + PatternConstConsList *cst_cons_list) +{ + int ret = OB_SUCCESS; + ObUDRContext rule_ctx; + sql::ObUDRMgr *rule_mgr = MTL(sql::ObUDRMgr*); + ObUDRAnalyzer analyzer(allocator, + session_info.get_sql_mode(), + session_info.get_local_collation_connection()); + if (OB_FAIL(analyzer.parse_sql_to_gen_match_param_infos(pattern, + rule_ctx.normalized_pattern_, + rule_ctx.raw_param_list_))) { + LOG_WARN("failed to parse to gen param infos", K(ret), K(pattern)); + } else { + rule_ctx.pattern_digest_ = rule_ctx.normalized_pattern_.hash(); + rule_ctx.tenant_id_ = session_info.get_effective_tenant_id(); + rule_ctx.db_name_ = session_info.get_database_name(); + rule_ctx.coll_type_ = session_info.get_local_collation_connection(); + if (OB_FAIL(rule_mgr->get_udr_item(rule_ctx, guard, cst_cons_list))) { + if (OB_HASH_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to match rewrite rule item", K(ret)); + } + } else if (guard.is_valid()) { + LOG_TRACE("succ to match rewrite rule item", KPC(guard.get_ref_obj())); + } + } + return ret; +} + +int ObUDRUtils::cons_udr_const_cons_list(const PatternConstConsList &cst_const_list, + ObPlanCacheCtx &pc_ctx) +{ + int ret = OB_SUCCESS; + OX (pc_ctx.tpl_sql_const_cons_.set_capacity(cst_const_list.count())); + for (int64_t i = 0; OB_SUCC(ret) && i < cst_const_list.count(); ++i) { + const FixedParamValueArray &fixed_param_list = cst_const_list.at(i); + NotParamInfoList not_param_info_list; + for (int64_t j = 0; OB_SUCC(ret) && j < fixed_param_list.count(); ++j) { + const FixedParamValue &fixed_param = fixed_param_list.at(j); + NotParamInfo not_param_info; + not_param_info.idx_ = fixed_param.idx_; + not_param_info.raw_text_ = fixed_param.raw_text_; + OZ (not_param_info_list.push_back(not_param_info)); + } + OZ (pc_ctx.tpl_sql_const_cons_.push_back(not_param_info_list)); + } + return ret; +} + +int ObUDRUtils::refill_udr_exec_ctx(const ObUDRItemMgr::UDRItemRefGuard &item_guard, + ObSqlCtx &context, + ObResultSet &result, + ObPlanCacheCtx &pc_ctx) +{ + int ret = OB_SUCCESS; + if (item_guard.is_valid()) { + ObIAllocator &allocator = result.get_mem_pool(); + const ObUDRItem *rule_item = item_guard.get_ref_obj(); + const ObString &replacement = rule_item->get_replacement(); + ParamStore param_store( (ObWrapperAllocator(allocator)) ); + const DynamicParamInfoArray& dynamic_param_list = rule_item->get_dynamic_param_info_array(); + OX (context.cur_sql_ = replacement); + OX (pc_ctx.is_rewrite_sql_ = true); + OX (pc_ctx.rule_name_ = rule_item->get_rule_name()); + OX (pc_ctx.def_name_ctx_ = const_cast(rule_item->get_question_mark_def_name_ctx())); + OZ (cons_udr_param_store(dynamic_param_list, pc_ctx, param_store)); + OZ (ObSql::construct_ps_param(param_store, pc_ctx)); + OX (pc_ctx.normal_parse_const_cnt_ = param_store.count()); + if (!pc_ctx.is_ps_mode_) { + OX (context.is_prepare_protocol_ = true); + OX (const_cast(pc_ctx.raw_sql_) = replacement); + OX (pc_ctx.is_ps_mode_ = true); + OX (pc_ctx.set_is_ps_execute_stage()); + } + } + return ret; +} + +int ObUDRUtils::add_param_to_param_store(const ObObjParam ¶m, + ParamStore ¶m_store) +{ + int ret = OB_SUCCESS; + if (lib::is_oracle_mode() + && ( (param.is_varchar() && 0 == param.get_varchar().length()) + || (param.is_char() && 0 == param.get_char().length()) + || (param.is_nstring() && 0 == param.get_string_len()) )) { + const_cast(param).set_null(); + const_cast(param).set_param_meta(); + } + if (OB_FAIL(param_store.push_back(param))) { + LOG_WARN("pushback param failed", K(ret)); + } + return ret; +} + +int ObUDRUtils::cons_udr_param_store(const DynamicParamInfoArray& dynamic_param_list, + ObPlanCacheCtx &pc_ctx, + ParamStore ¶m_store) +{ + int ret = OB_SUCCESS; + ObIAllocator &allocator = pc_ctx.allocator_; + ObPhysicalPlanCtx *phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx(); + if (dynamic_param_list.empty()) { + // do nothing + } else if (OB_ISNULL(phy_ctx)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), KP(phy_ctx)); + } else { + if (OB_ISNULL(pc_ctx.sql_ctx_.session_info_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(pc_ctx.sql_ctx_.session_info_)); + } else if (pc_ctx.is_ps_mode_) { + ObSQLMode sql_mode = pc_ctx.sql_ctx_.session_info_->get_sql_mode(); + ObCollationType conn_coll = pc_ctx.sql_ctx_.session_info_->get_local_collation_connection(); + FPContext fp_ctx(conn_coll); + fp_ctx.enable_batched_multi_stmt_ = pc_ctx.sql_ctx_.handle_batched_multi_stmt(); + fp_ctx.sql_mode_ = sql_mode; + if (OB_FAIL(ObSqlParameterization::fast_parser(allocator, + fp_ctx, + pc_ctx.raw_sql_, + pc_ctx.fp_result_))) { + LOG_WARN("failed to fast parser", K(ret), K(sql_mode), K(pc_ctx.raw_sql_)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(clac_dynamic_param_store(dynamic_param_list, + pc_ctx, + param_store))) { + LOG_WARN("failed to calc dynamic param store", K(ret)); + } + } + return ret; +} + +int ObUDRUtils::clac_dynamic_param_store(const DynamicParamInfoArray& dynamic_param_list, + ObPlanCacheCtx &pc_ctx, + ParamStore ¶m_store) +{ + int ret = OB_SUCCESS; + ObString literal_prefix; + ObObjParam value; + const bool is_paramlize = false; + int64_t server_collation = CS_TYPE_INVALID; + + ObIAllocator &allocator = pc_ctx.allocator_; + ObSQLSessionInfo *session = pc_ctx.exec_ctx_.get_my_session(); + ObIArray &raw_params = pc_ctx.fp_result_.raw_params_; + ObPhysicalPlanCtx *phy_ctx = pc_ctx.exec_ctx_.get_physical_plan_ctx(); + if (OB_ISNULL(session) || OB_ISNULL(phy_ctx)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), KP(session), KP(phy_ctx)); + } else if (OB_FAIL(param_store.reserve(dynamic_param_list.count()))) { + LOG_WARN("failed to reserve array", K(ret), K(dynamic_param_list.count())); + } else if (lib::is_oracle_mode() && OB_FAIL( + session->get_sys_variable(share::SYS_VAR_COLLATION_SERVER, server_collation))) { + LOG_WARN("get sys variable failed", K(ret)); + } else { + ObCollationType coll_conn = static_cast(session->get_local_collation_connection()); + for (int i = 0; OB_SUCC(ret) && i < dynamic_param_list.count(); ++i) { + value.reset(); + const DynamicParamInfo &dynamic_param_info = dynamic_param_list.at(i); + ParseNode *raw_param = NULL; + ObPCParam *pc_param = NULL; + if (dynamic_param_info.raw_param_idx_ >= raw_params.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid idx", K(dynamic_param_info.raw_param_idx_), K(raw_params.count())); + } else if (OB_ISNULL(pc_param = raw_params.at(dynamic_param_info.raw_param_idx_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("raw param is null", K(ret)); + } else if (OB_ISNULL(raw_param = pc_param->node_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("node is null", K(ret)); + } else if (T_QUESTIONMARK == raw_param->type_) { + if (!pc_ctx.is_ps_mode_ || raw_param->value_ >= pc_ctx.fp_result_.ps_params_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", K(ret), K(raw_param->value_), K(dynamic_param_info.raw_param_idx_), + K(pc_ctx.is_ps_mode_), K(pc_ctx.fp_result_.ps_params_.count())); + } else if (OB_FAIL(param_store.push_back(*pc_ctx.fp_result_.ps_params_.at(raw_param->value_)))) { + LOG_WARN("pushback param failed", K(ret)); + } + } else if (OB_FAIL(ObResolverUtils::resolve_const(raw_param, + stmt::T_NONE, + allocator, + coll_conn, + session->get_nls_collation_nation(), + session->get_timezone_info(), + value, + is_paramlize, + literal_prefix, + session->get_actual_nls_length_semantics(), + static_cast(server_collation), + NULL, session->get_sql_mode()))) { + LOG_WARN("fail to resolve const", K(ret)); + } else if (OB_FAIL(add_param_to_param_store(value, param_store))) { + LOG_WARN("failed to add param to param store", K(ret), K(value), K(param_store)); + } + } + if (OB_SUCC(ret)) { + if (phy_ctx->get_param_store_for_update().assign(param_store)) { + LOG_WARN("failed to assign param store", K(ret)); + } else { + pc_ctx.fp_result_.cache_params_ = &(phy_ctx->get_param_store_for_update()); + } + } + } + return ret; +} + +int ObUDRUtils::match_udr_and_refill_ctx(const ObString &pattern, + ObSqlCtx &sql_ctx, + ObResultSet &result, + ObPlanCacheCtx &pc_ctx, + bool &is_match_udr, + ObUDRItemMgr::UDRItemRefGuard &item_guard) +{ + int ret = OB_SUCCESS; + is_match_udr = false; + ObSQLSessionInfo &session = result.get_session(); + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(session.get_effective_tenant_id())); + if (!tenant_config.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant config is invalid", K(ret)); + } else if (tenant_config->enable_user_defined_rewrite_rules && !pc_ctx.is_inner_sql()) { + ObIAllocator &allocator = result.get_mem_pool(); + PatternConstConsList cst_cons_list; + if (OB_FAIL(match_udr_item(pattern, session, allocator, item_guard, &cst_cons_list))) { + LOG_WARN("failed to match user defined rewrite rule", K(ret)); + } else if (!cst_cons_list.empty() + && OB_FAIL(cons_udr_const_cons_list(cst_cons_list, pc_ctx))) { + LOG_WARN("failed to cons tpl sql const cons list", K(ret)); + } else if (!item_guard.is_valid()) { + is_match_udr = false; + LOG_TRACE("no matching user-defined rules", K(ret)); + } else if (OB_FAIL(refill_udr_exec_ctx(item_guard, + sql_ctx, + result, + pc_ctx))) { + LOG_WARN("failed to refill rewrite sql exec ctx", K(ret)); + } else { + is_match_udr = true; + LOG_TRACE("succ to match user-defined rule", K(ret)); + } + } + return ret; +} + +} // namespace sql end +} // namespace oceanbase end diff --git a/src/sql/udr/ob_udr_utils.h b/src/sql/udr/ob_udr_utils.h new file mode 100644 index 000000000..e6c28fe54 --- /dev/null +++ b/src/sql/udr/ob_udr_utils.h @@ -0,0 +1,56 @@ +// Copyright 2015-2016 Alibaba Inc. All Rights Reserved. +// Author: +// LuoFan luofan.zp@alibaba-inc.com +// Normalizer: +// LuoFan luofan.zp@alibaba-inc.com + + +#ifndef OB_SQL_UDR_OB_UDR_UTILS_H_ +#define OB_SQL_UDR_OB_UDR_UTILS_H_ + +#include "sql/udr/ob_udr_struct.h" + +namespace oceanbase +{ +namespace sql +{ +class ObSqlCtx; +class ObResultSet; +class ObPlanCacheCtx; + +class ObUDRUtils +{ +public: + static int match_udr_and_refill_ctx(const ObString &pattern, + ObSqlCtx &sql_ctx, + ObResultSet &result, + ObPlanCacheCtx &pc_ctx, + bool &is_match_udr, + ObUDRItemMgr::UDRItemRefGuard &item_guard); + static int match_udr_item(const ObString &pattern, + const ObSQLSessionInfo &session_info, + ObIAllocator &allocator, + ObUDRItemMgr::UDRItemRefGuard &guard, + PatternConstConsList *cst_cons_list = nullptr); + static int cons_udr_param_store(const DynamicParamInfoArray &dynamic_param_list, + ObPlanCacheCtx &pc_ctx, + ParamStore ¶m_store); + +private: + static int cons_udr_const_cons_list(const PatternConstConsList &cst_const_list, + ObPlanCacheCtx &pc_ctx); + static int refill_udr_exec_ctx(const ObUDRItemMgr::UDRItemRefGuard &item_guard, + ObSqlCtx &context, + ObResultSet &result, + ObPlanCacheCtx &pc_ctx); + static int clac_dynamic_param_store(const DynamicParamInfoArray& dynamic_param_list, + ObPlanCacheCtx &pc_ctx, + ParamStore ¶m_store); + static int add_param_to_param_store(const ObObjParam ¶m, + ParamStore ¶m_store); +}; + +} // namespace sql end +} // namespace oceanbase end + +#endif \ No newline at end of file diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result index 24b02619c..3501e4ced 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result @@ -86,6 +86,7 @@ enable_sys_table_ddl enable_sys_unit_standalone enable_tcp_keepalive enable_upgrade_mode +enable_user_defined_rewrite_rules external_kms_info freeze_trigger_percentage fuse_row_cache_priority diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result index 7a7b37555..9fb0fe7ba 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result @@ -216,6 +216,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 412 __all_service_epoch 0 201001 1 413 __all_spatial_reference_systems 0 201001 1 416 __all_column_checksum_error_info 0 201001 1 +443 __all_tenant_rewrite_rules 0 201001 1 445 __all_cluster_event_history 0 201001 1 10001 __tenant_virtual_all_table 2 201001 1 10002 __tenant_virtual_table_column 2 201001 1 @@ -818,6 +819,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 21337 CDB_OB_USERS 1 201001 1 21338 DBA_OB_DATABASE_PRIVILEGE 1 201001 1 21339 CDB_OB_DATABASE_PRIVILEGE 1 201001 1 +21340 DBA_OB_USER_DEFINED_RULES 1 201001 1 21345 DBA_OB_CLUSTER_EVENT_HISTORY 1 201001 1 check sys table count and table_id range success check count and table_id range for virtual table success diff --git a/unittest/sql/parser/test_fast_parser.cpp b/unittest/sql/parser/test_fast_parser.cpp index 74d4eec12..763811dfa 100644 --- a/unittest/sql/parser/test_fast_parser.cpp +++ b/unittest/sql/parser/test_fast_parser.cpp @@ -203,8 +203,11 @@ int TestFastParser::parse(const ObString &sql) char *no_param_sql_ptr = NULL; int64_t no_param_sql_len = 0; ParamList *p_list = NULL; - bool enable_batched_multi_stmt = false; - int ret2 = ObFastParser::parse(sql, enable_batched_multi_stmt, no_param_sql_ptr, no_param_sql_len, p_list, param_num, connection_collation, allocator_); + FPContext fp_ctx(connection_collation); + fp_ctx.enable_batched_multi_stmt_ = false; + fp_ctx.is_udr_mode_ = false; + int ret2 = ObFastParser::parse(sql, fp_ctx, allocator_, + no_param_sql_ptr, no_param_sql_len, p_list, param_num); if ((OB_SUCCESS == ret1) != (OB_SUCCESS == ret2)) { SQL_PC_LOG(ERROR, "parser results are not equal", K(ret1), K(ret2), K(sql)); return OB_ERROR; diff --git a/unittest/sql/plan_cache/test_sql_parameterization.cpp b/unittest/sql/plan_cache/test_sql_parameterization.cpp index 41ed352d3..2339c3897 100644 --- a/unittest/sql/plan_cache/test_sql_parameterization.cpp +++ b/unittest/sql/plan_cache/test_sql_parameterization.cpp @@ -61,11 +61,11 @@ TEST_F(TestSqlParameterization, parameterize_sql) ObString stmt = ObString::make_string("select * from t1 where c1 = 3 group by 2 order by 1"); ObSEArray not_params; ObFastParserResult fp_result; + FPContext fp_ctx(ObCharset::get_system_collation()); + fp_ctx.sql_mode_ = SMO_DEFAULT; if (OB_FAIL(ObSqlParameterization::fast_parser(allocator, - SMO_DEFAULT, - ObCharset::get_system_collation(), + fp_ctx, stmt, - false, fp_result))) { SQL_PC_LOG(WARN, "fail to fast_parser", K(stmt)); }