[FEAT MERGE]implement user-defined rewrite rules

This commit is contained in:
obdev 2022-12-30 08:10:42 +00:00 committed by ob-robot
parent 21c0bac716
commit 9dcc0a529e
100 changed files with 5540 additions and 315 deletions

View File

@ -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;

View File

@ -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)

View File

@ -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")

View File

@ -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)

View File

@ -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);

View File

@ -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<char *>(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();

View File

@ -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<char *>(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();

View File

@ -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

View File

@ -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

View File

@ -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_);
}

View File

@ -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_);
}

View File

@ -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);

View File

@ -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,

View File

@ -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));

View File

@ -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;

View File

@ -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))

View File

@ -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)

View File

@ -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 &params, 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 &params, 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 &params_;
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 &params, 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_ */

View File

@ -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;
}

View File

@ -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

View File

@ -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;

View File

@ -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);

View File

@ -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

View File

@ -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<ObAddr, 8> 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

View File

@ -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

View File

@ -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;
}

View File

@ -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);

View File

@ -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

View File

@ -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));
}
}

File diff suppressed because one or more lines are too long

View File

@ -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;

View File

@ -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));
}
}

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -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) {

View File

@ -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];

View File

@ -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";

View File

@ -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

View File

@ -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;
//

View File

@ -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;
//

View File

@ -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;

View File

@ -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;

View File

@ -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)
{

View File

@ -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;
}

View File

@ -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,

View File

@ -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)

View File

@ -8092,6 +8092,24 @@ private:
common::ObSArray<common::ObAddr> 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);

View File

@ -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

View File

@ -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<uint64_t>(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

View File

@ -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 ============= */

View File

@ -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));

View File

@ -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* \
)

View File

@ -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

View File

@ -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();

View File

@ -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

View File

@ -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<ObPCParam*> &raw_params,
const ObIArray<int64_t> &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<ObDelUpdStmt*>(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 &params,
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.

View File

@ -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 &param,
ParamStore &param_store);
int construct_param_store(const ParamStore &params,
ParamStore &param_store);
int construct_ps_param_store(const ParamStore &params,
@ -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和typeObPsStmtInfops cache
*/
int do_add_ps_cache(const ObString &sql,
const ObString &no_param_sql,
const ObIArray<ObPCParam*> &raw_params,
const common::ObIArray<int64_t> &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 &param,
ParamStore &param_store);
typedef hash::ObHashMap<uint64_t, ObPlanCache*> PlanCacheMap;
friend class ::test::TestOptimizerUtils;

View File

@ -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 *&param_list,
int64_t &param_num,
ObCollationType connection_collation,
common::ObIAllocator &allocator,
ObSQLMode sql_mode /* default = 0*/)
int64_t &param_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 *&param_list,
int64_t &param_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<char *>(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<char *>(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<char *>(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<char *>(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<char *>(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}

View File

@ -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 *&param_list,
int64_t &param_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 *&param_list,
int64_t &param_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 *&param_list,
int64_t &param_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:

View File

@ -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;

View File

@ -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

View File

@ -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);

View File

@ -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

View File

@ -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;
}

View File

@ -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);

View File

@ -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<ObPlanCacheCtx&>(ctx);
if (cache_obj != NULL && ObLibCacheNameSpace::NS_CRSR == cache_obj->get_ns()) {
plan = static_cast<ObPhysicalPlan *>(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

View File

@ -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<NotParamInfo, 4> NotParamInfoList;
// Template SQL Constant Constraints
typedef common::ObSEArray<NotParamInfoList, 4> 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<ObPCParam *, common::ObIAllocator> raw_params_;
common::ObFixedArray<const common::ObObjParam *, common::ObIAllocator> 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<FixedParamValue, common::ObIAllocator> fixed_param_info_list_;
common::ObFixedArray<DynamicParamInfo, common::ObIAllocator> dynamic_param_info_list_;
common::ObFixedArray<NotParamInfoList, common::ObIAllocator> tpl_sql_const_cons_;
// ********** for rewrite end **********
};
struct ObPlanCacheStat

View File

@ -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_),

View File

@ -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 &not_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 &not_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 &not_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 &param_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));
}
}

View File

@ -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<PCVSchemaObj *, common::ObIAllocator> 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:
* 1First match tpl_sql_const_cons_constraint list
* 2If it hits, use the hit rule to compare it with not_param_info_, if it is the same
* the match succeeds, otherwise it fails
* 3If 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);
};

View File

@ -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))) {

View File

@ -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 &param);
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<int64_t> &param_idxs,
const common::ObIArray<ObPCParam *> &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<ObPCParam*> *raw_params_;
common::ObIArray<int64_t> *fixed_param_idx_;
stmt::StmtType stmt_type_;
};
} //end of namespace sql
} //end of namespace oceanbase

View File

@ -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<ObPCParam*> &raw_params,
const common::ObIArray<int64_t> &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));

View File

@ -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<ObPCParam*> &raw_params,
const common::ObIArray<int64_t> &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();

View File

@ -63,6 +63,7 @@ struct TransformTreeCtx
SQL_EXECUTION_MODE mode_;
bool is_project_list_scope_;
int64_t assign_father_level_;
const ObIArray<FixedParamValue> *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<FixedParamValue> *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_);

View File

@ -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<FixedParamValue> *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);

View File

@ -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<ObString, 1> 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<char *>(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<ObPCParam*> &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<ObPCParam*> &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<ObPCParam*> &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<typename T>
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<char *>(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<char*>(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<ObPCParam*, 16> 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<FixedParamValueArray>(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<DynamicParamInfoArray>(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<QuestionMarkDefNameCtx>(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

View File

@ -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<ObPCParam*> &raw_params);
int parse_pattern_to_gen_param_infos(const common::ObString &pattern,
common::ObString &normalized_pattern,
common::ObIArray<ObPCParam*> &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<typename T>
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<ObPCParam*> &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

View File

@ -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

View File

@ -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<ObUDRItemMgr::UDRKey, ObUDRItemMgr::UDRKeyNodePair*> 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

View File

@ -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<ObPCParam*, 16> raw_param_list_;
};
} // namespace sql end
} // namespace oceanbase end
#endif

164
src/sql/udr/ob_udr_item.cpp Normal file
View File

@ -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<typename T>
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<char*>(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<FixedParamValueArray>(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<DynamicParamInfoArray>(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<QuestionMarkDefNameCtx>(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

117
src/sql/udr/ob_udr_item.h Normal file
View File

@ -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<typename T>
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

View File

@ -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<ObUDRItemMgr::UDRKey, ObUDRItemMgr::UDRKeyNodePair*> &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<typename T,
typename ...Args>
int ObUDRItemMgr::alloc(UDRRefObjGuard<T> &guard, Args&& ...args)
{
int ret = OB_SUCCESS;
char *ptr = NULL;
if (NULL == (ptr = static_cast<char *>(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>(args)...);
guard.ref_obj_->inc_ref_count();
}
return ret;
}
template<typename Info>
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<ObUDRItem>(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<UDRKeyNodePair> rule_node_guard;
if (OB_FAIL(alloc<UDRKeyNodePair>(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<ObUDRContext>(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<ObUDRInfo>& 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<ObUDRInfo>(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

View File

@ -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<typename T>
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<ObUDRItem*, ObIAllocator> 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<UDRKey, UDRKeyNodePair*> RuleKeyNodeMap;
typedef UDRRefObjGuard<ObUDRItem> 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<ObUDRInfo>& 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<typename T, typename ...Args>
int alloc(UDRRefObjGuard<T> &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<typename Info>
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

264
src/sql/udr/ob_udr_mgr.cpp Normal file
View File

@ -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<ObString &>(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<ObUDRInfo, 32> 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

121
src/sql/udr/ob_udr_mgr.h Normal file
View File

@ -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

View File

@ -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<int64_t>(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<int64_t>(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<int64_t>(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<int64_t>(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<ObUDRInfo>& 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

View File

@ -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<ObUDRInfo>& 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

View File

@ -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<char *>(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

155
src/sql/udr/ob_udr_struct.h Normal file
View File

@ -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<FixedParamValue, 8> FixedParamValueArray;
typedef common::ObSEArray<DynamicParamInfo, 8> DynamicParamInfoArray;
typedef common::ObSEArray<FixedParamValueArray, 8> 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

View File

@ -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<QuestionMarkDefNameCtx *>(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<ObString &>(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 &param,
ParamStore &param_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<ObObjParam &>(param).set_null();
const_cast<ObObjParam &>(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 &param_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 &param_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<ObPCParam *> &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<ObCollationType>(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<ObCollationType>(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

View File

@ -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 &param_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 &param_store);
static int add_param_to_param_store(const ObObjParam &param,
ParamStore &param_store);
};
} // namespace sql end
} // namespace oceanbase end
#endif

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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<const ObPCParam *, OB_PC_NOT_PARAM_COUNT> 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));
}