[FEAT MERGE] patch 423 sql compatible features to 432

Co-authored-by: akaError <lzg020616@163.com>
Co-authored-by: JinmaoLi <ljm.csmaster@gmail.com>
Co-authored-by: qingzhu521 <q15000557748@gmail.com>
This commit is contained in:
yinyj17
2024-05-27 11:18:00 +00:00
committed by ob-robot
parent a718b67350
commit 5ce9ef5136
208 changed files with 11028 additions and 495 deletions

View File

@ -1955,7 +1955,8 @@ int ObChangeTenantExecutor::execute(ObExecContext &ctx, ObChangeTenantStmt &stmt
LOG_WARN("ptr is null", KR(ret));
} else if (FALSE_IT(pre_effective_tenant_id = session_info->get_effective_tenant_id())) {
} else if (FALSE_IT(login_tenant_id = session_info->get_login_tenant_id())) {
} else if (FALSE_IT(session_info->get_session_priv_info(session_priv))) {
} else if (OB_FAIL(session_info->get_session_priv_info(session_priv))) {
LOG_WARN("fail to get session priv info", K(ret));
} else if (effective_tenant_id == pre_effective_tenant_id) {
// do nothing
} else if (OB_SYS_TENANT_ID != login_tenant_id) { //case 1

View File

@ -37,7 +37,6 @@ int ObKillExecutor::execute(ObExecContext &ctx, ObKillStmt &stmt)
ObKillSessionArg arg;
ObAddr addr;
ObSQLSessionMgr &session_mgr = OBSERVER.get_sql_session_mgr();
if (OB_FAIL(arg.init(ctx, stmt))) {
LOG_WARN("fail to init kill_session arg", K(ret), K(arg), K(ctx), K(stmt));
} else {

View File

@ -0,0 +1,33 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/cmd/ob_mock_executor.h"
#include "sql/resolver/cmd/ob_mock_stmt.h"
namespace oceanbase
{
using namespace common;
namespace sql
{
int ObMockExecutor::execute(ObExecContext &exec_ctx, ObMockStmt &stmt)
{
int ret = OB_SUCCESS;
if (stmt::T_FLUSH_PRIVILEGES == stmt.get_stmt_type()) {
LOG_USER_WARN(OB_NOT_SUPPORTED, "After executing the GRANT and REVOKE statements, "
"the permissions will be automatically applied and take effect. "
"There is no need to execute the FLUSH PRIVILEGES command. FLUSH PRIVILEGES");
}
return ret;
}
}
}

View File

@ -0,0 +1,37 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_ENGINE_MOCK_EXECUTOR_H_
#define OCEANBASE_SQL_ENGINE_MOCK_EXECUTOR_H_
#include "share/ob_define.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/resolver/cmd/ob_mock_stmt.h"
namespace oceanbase
{
namespace sql
{
class ObMockExecutor
{
public:
ObMockExecutor() = default;
virtual ~ObMockExecutor() = default;
int execute(ObExecContext &exec_ctx, ObMockStmt &stmt);
private:
DISALLOW_COPY_AND_ASSIGN(ObMockExecutor);
};
}
}
#endif

View File

@ -147,7 +147,7 @@ int ObDropRoleExecutor::execute(ObExecContext &ctx, ObDropRoleStmt &stmt)
OZ (arg.users_.push_back(stmt.get_user_names().at(i)));
OZ (arg.hosts_.push_back(stmt.get_host_names().at(i)));
}
OZ (ObDropUserExecutor::drop_user(common_rpc_proxy, arg));
OZ (ObDropUserExecutor::drop_user(common_rpc_proxy, arg, false));
}
return ret;

View File

@ -104,23 +104,67 @@ int ObCreateTableExecutor::ObInsSQLPrinter::inner_print(char *buf, int64_t buf_l
const char sep_char = lib::is_oracle_mode()? '"': '`';
const ObSelectStmt *select_stmt = NULL;
int64_t pos1 = 0;
uint64_t insert_mode = 0;
if (OB_ISNULL(stmt_) || OB_ISNULL(select_stmt= stmt_->get_sub_select())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("null stmt", K(ret));
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos1,
do_osg_
? "insert /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c"
: "insert /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) NO_GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c",
stmt_->get_parallelism(),
sep_char,
stmt_->get_database_name().length(),
stmt_->get_database_name().ptr(),
sep_char,
sep_char,
stmt_->get_table_name().length(),
stmt_->get_table_name().ptr(),
sep_char))) {
LOG_WARN("fail to print insert into string", K(ret));
} else {
insert_mode = stmt_->get_insert_mode();
if (insert_mode != 0 &&
insert_mode != 1 &&
insert_mode != 2 ) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected insert_mode", K(insert_mode), K(ret));
} else if (insert_mode == 1 /*ignore*/) {
if (OB_FAIL(databuff_printf(buf, buf_len, pos1,
do_osg_
? "insert ignore /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c"
: "insert ignore /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) NO_GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c",
stmt_->get_parallelism(),
sep_char,
stmt_->get_database_name().length(),
stmt_->get_database_name().ptr(),
sep_char,
sep_char,
stmt_->get_table_name().length(),
stmt_->get_table_name().ptr(),
sep_char))) {
LOG_WARN("fail to print insert into string", K(ret));
}
} else if (insert_mode == 2 /*replace*/) {
if (OB_FAIL(databuff_printf(buf, buf_len, pos1,
do_osg_
? "replace /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c"
: "replace /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) NO_GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c",
stmt_->get_parallelism(),
sep_char,
stmt_->get_database_name().length(),
stmt_->get_database_name().ptr(),
sep_char,
sep_char,
stmt_->get_table_name().length(),
stmt_->get_table_name().ptr(),
sep_char))) {
LOG_WARN("fail to print insert into string", K(ret));
}
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos1,
do_osg_
? "insert /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c"
: "insert /*+ ENABLE_PARALLEL_DML PARALLEL(%lu) NO_GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c",
stmt_->get_parallelism(),
sep_char,
stmt_->get_database_name().length(),
stmt_->get_database_name().ptr(),
sep_char,
sep_char,
stmt_->get_table_name().length(),
stmt_->get_table_name().ptr(),
sep_char))) {
LOG_WARN("fail to print insert into string", K(ret));
}
}
if (OB_FAIL(ret)) {
//do nothing
} else if (lib::is_oracle_mode()) {
const ObTableSchema &table_schema = stmt_->get_create_table_arg().schema_;
int64_t used_column_count = 0;

View File

@ -23,6 +23,7 @@
#include "sql/resolver/dcl/ob_lock_user_stmt.h"
#include "sql/resolver/dcl/ob_rename_user_stmt.h"
#include "sql/resolver/dcl/ob_alter_user_profile_stmt.h"
#include "sql/resolver/dcl/ob_alter_user_proxy_stmt.h"
#include "sql/resolver/dcl/ob_alter_user_primary_zone_stmt.h"
#include "sql/engine/ob_exec_context.h"
@ -268,6 +269,16 @@ int ObCreateUserExecutor::create_user(obrpc::ObCommonRpcProxy *rpc_proxy,
return ret;
}
int ObDropUserExecutor::build_fail_msg_for_one(const ObString &user, const ObString &host,
common::ObSqlString &msg) {
int ret = OB_SUCCESS;
if (OB_FAIL(msg.append_fmt("'%.*s'@'%.*s'",
user.length(), user.ptr(),
host.length(), host.ptr()))) {
LOG_WARN("Build msg fail", K(user), K(host), K(ret));
}
return ret;
}
int ObDropUserExecutor::build_fail_msg(const common::ObIArray<common::ObString> &users,
const common::ObIArray<common::ObString> &hosts, common::ObSqlString &msg)
@ -284,10 +295,8 @@ int ObDropUserExecutor::build_fail_msg(const common::ObIArray<common::ObString>
if (OB_SUCC(ret)) {
const ObString &user = users.at(i);
const ObString &host = hosts.at(i);
if (OB_FAIL(msg.append_fmt("'%.*s'@'%.*s'",
user.length(), user.ptr(),
host.length(), host.ptr()))) {
LOG_WARN("Build msg fail", K(user), K(host), K(ret));
if (OB_FAIL(build_fail_msg_for_one(user, host, msg))) {
LOG_WARN("Build fail msg fail", K(user), K(host), K(ret));
}
}
}
@ -373,7 +382,7 @@ int ObDropUserExecutor::execute(ObExecContext &ctx, ObDropUserStmt &stmt)
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(drop_user(common_rpc_proxy, arg))) {
if (OB_FAIL(drop_user(common_rpc_proxy, arg, stmt.get_if_exists()))) {
LOG_WARN("Drop user completely failed", K(ret));
} else {
//do nothing
@ -384,7 +393,8 @@ int ObDropUserExecutor::execute(ObExecContext &ctx, ObDropUserStmt &stmt)
}
int ObDropUserExecutor::drop_user(obrpc::ObCommonRpcProxy *rpc_proxy,
const obrpc::ObDropUserArg &arg)
const obrpc::ObDropUserArg &arg,
bool if_exist_stmt)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!arg.is_valid())) {
@ -409,12 +419,28 @@ int ObDropUserExecutor::drop_user(obrpc::ObCommonRpcProxy *rpc_proxy,
failed_index, failed_users,
failed_hosts))) {
LOG_WARN("Failed to extract user name", K(arg), K(ret));
} else if (OB_FAIL(ObDropUserExecutor::build_fail_msg(failed_users, failed_hosts, fail_msg))) {
LOG_WARN("Build fail msg error", K(arg), K(ret));
} else {
const char *ERR_CMD = (arg.is_role_ && lib::is_mysql_mode()) ? "DROP ROLE" : "DROP USER";
ret = OB_CANNOT_USER;
LOG_USER_ERROR(OB_CANNOT_USER, (int)strlen(ERR_CMD), ERR_CMD, (int)fail_msg.length(), fail_msg.ptr());
} else if (if_exist_stmt) {
if (OB_UNLIKELY(failed_users.count() < 1) || OB_UNLIKELY(failed_users.count() != failed_users.count())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(failed_users.count()), K(failed_users.count()), K(ret));
} else {
for (int i = 0; OB_SUCC(ret) && i < failed_users.count(); ++i) {
ObSqlString fail_msg_one;
if (OB_FAIL(ObDropUserExecutor::build_fail_msg_for_one(failed_users.at(i), failed_hosts.at(i), fail_msg_one))) {
LOG_WARN("Build fail msg error", K(arg), K(ret));
} else {
LOG_USER_WARN(OB_CANNOT_USER_IF_EXISTS, (int)fail_msg_one.length(), fail_msg_one.ptr());
}
}
}
} else if (!if_exist_stmt) {
if (OB_FAIL(ObDropUserExecutor::build_fail_msg(failed_users, failed_hosts, fail_msg))) {
LOG_WARN("Build fail msg error", K(arg), K(ret));
} else {
const char *ERR_CMD = (arg.is_role_ && lib::is_mysql_mode()) ? "DROP ROLE" : "DROP USER";
ret = OB_CANNOT_USER;
LOG_USER_ERROR(OB_CANNOT_USER, (int)strlen(ERR_CMD), ERR_CMD, (int)fail_msg.length(), fail_msg.ptr());
}
}
}
}
@ -608,6 +634,28 @@ int ObAlterUserProfileExecutor::execute(ObExecContext &ctx, ObAlterUserProfileSt
return ret;
}
int ObAlterUserProxyExecutor::execute(ObExecContext &ctx, ObAlterUserProxyStmt &stmt)
{
int ret = OB_SUCCESS;
ObTaskExecutorCtx *task_exec_ctx = NULL;
obrpc::ObCommonRpcProxy *common_rpc_proxy = NULL;
obrpc::ObAlterUserProxyRes res;
if (OB_ISNULL(stmt.get_query_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error", K(ret));
} else if (OB_FALSE_IT(stmt.get_ddl_arg().ddl_stmt_str_ = stmt.get_query_ctx()->get_sql_stmt())) {
} else if (OB_ISNULL(task_exec_ctx = GET_TASK_EXECUTOR_CTX(ctx))) {
ret = OB_NOT_INIT;
LOG_WARN("get task executor context failed", K(ret));
} else if (OB_ISNULL(common_rpc_proxy = task_exec_ctx->get_common_rpc())) {
ret = OB_NOT_INIT;
LOG_WARN("get common rpc proxy failed", K(ret));
} else if (OB_FAIL(common_rpc_proxy->alter_user_proxy(stmt.get_ddl_arg(), res))) {
LOG_WARN("alter user proxy failed", K(stmt.get_ddl_arg()), K(ret));
}
return ret;
}
int ObRenameUserExecutor::execute(ObExecContext &ctx, ObRenameUserStmt &stmt)
{
int ret = OB_SUCCESS;

View File

@ -75,8 +75,13 @@ public:
const common::ObIArray<int64_t> &index,
common::ObIArray<common::ObString> &dst_users,
common::ObIArray<common::ObString> &dst_hosts);
static int build_fail_msg_for_one(const ObString &user,
const ObString &host,
common::ObSqlString &msg);
static int drop_user(obrpc::ObCommonRpcProxy *rpc_proxy,
const obrpc::ObDropUserArg &arg);
const obrpc::ObDropUserArg &arg,
bool if_exists);
int execute(ObExecContext &ctx, ObDropUserStmt &stmt);
private:
@ -110,6 +115,18 @@ public:
DISALLOW_COPY_AND_ASSIGN(ObAlterUserProfileExecutor);
};
class ObAlterUserProxyStmt;
class ObAlterUserProxyExecutor
{
private:
public:
ObAlterUserProxyExecutor() {}
virtual ~ObAlterUserProxyExecutor() {}
int execute(ObExecContext &ctx, ObAlterUserProxyStmt &stmt);
DISALLOW_COPY_AND_ASSIGN(ObAlterUserProxyExecutor);
};
class ObRenameUserStmt;
class ObRenameUserExecutor
{

View File

@ -375,6 +375,9 @@
#include "ob_expr_st_symdifference.h"
#include "ob_expr_priv_st_asmvtgeom.h"
#include "ob_expr_priv_st_makevalid.h"
#include "ob_expr_inner_table_option_printer.h"
#include "ob_expr_password.h"
namespace oceanbase
{
@ -1141,14 +1144,14 @@ static ObExpr::EvalFunc g_expr_eval_functions[] = {
NULL, //ObExprXmlConcat::eval_xml_concat, /* 662 */
NULL, //ObExprXmlForest::eval_xml_forest, /* 663 */
NULL, //ObExprExistsNodeXml::eval_existsnode_xml, /* 664 */
NULL, //ObExprPassword::eval_password, /* 665 */
ObExprPassword::eval_password, /* 665 */
ObExprDocID::generate_doc_id, /* 666 */
ObExprWordSegment::generate_fulltext_column, /* 667 */
ObExprWordCount::generate_word_count, /* 668 */
ObExprBM25::eval_bm25_relevance_expr, /* 669 */
ObExprTransactionId::eval_transaction_id, /* 670 */
NULL, //ObExprInnerTableOptionPrinter::eval_inner_table_option_printer, /* 671 */
NULL, //ObExprInnerTableSequenceGetter::eval_inner_table_sequence_getter, /* 672 */
ObExprInnerTableOptionPrinter::eval_inner_table_option_printer, /* 671 */
ObExprInnerTableSequenceGetter::eval_inner_table_sequence_getter, /* 672 */
NULL, //ObExprDecodeTraceId::calc_decode_trace_id_expr, /* 673 */
ObExprInnerRowCmpVal::eval_inner_row_cmp_val, /* 674 */
ObExprIs::json_is_true, /* 675 */

View File

@ -0,0 +1,212 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/expr/ob_expr_inner_table_option_printer.h"
#include <string.h>
#include "share/object/ob_obj_cast.h"
#include "objit/common/ob_item_type.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/session/ob_sql_session_info.h"
#include "share/schema/ob_schema_struct.h"
#include "share/schema/ob_schema_printer.h"
//#include "sql/engine/expr/ob_expr_promotion_util.h"
namespace oceanbase
{
using namespace common;
namespace sql
{
ObExprInnerTableOptionPrinter::ObExprInnerTableOptionPrinter(ObIAllocator &alloc)
: ObStringExprOperator(alloc, T_FUN_SYS_INNER_TABLE_OPTION_PRINTER, N_INNER_TABLE_OPTION_PRINTER, 3,
VALID_FOR_GENERATED_COL, INTERNAL_IN_MYSQL_MODE, INTERNAL_IN_ORACLE_MODE)
{
need_charset_convert_ = false;
}
ObExprInnerTableOptionPrinter::~ObExprInnerTableOptionPrinter()
{
}
inline int ObExprInnerTableOptionPrinter::calc_result_type3(ObExprResType &type,
ObExprResType &type1,
ObExprResType &type2,
ObExprResType &type3,
common::ObExprTypeCtx &type_ctx) const
{
int ret = OB_SUCCESS;
UNUSED(type_ctx);
type1.set_calc_type(ObIntType);
type2.set_calc_type(ObIntType);
type3.set_calc_type(ObIntType);
type.set_type(ObVarcharType);
type.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
type.set_collation_level(common::CS_LEVEL_IMPLICIT);
return ret;
}
int ObExprInnerTableOptionPrinter::cg_expr(ObExprCGCtx &, const ObRawExpr &, ObExpr &rt_expr) const
{
int ret = OB_SUCCESS;
CK(3 == rt_expr.arg_cnt_);
rt_expr.eval_func_ = &ObExprInnerTableOptionPrinter::eval_inner_table_option_printer;
return ret;
}
int ObExprInnerTableOptionPrinter::eval_inner_table_option_printer(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &expr_datum)
{
int ret = OB_SUCCESS;
share::schema::ObSchemaGetterGuard schema_guard;
const ObSQLSessionInfo *session = ctx.exec_ctx_.get_my_session();
ObDatum *tenant_id = nullptr;
ObDatum *database_id = nullptr;
ObDatum *table_id = nullptr;
const ObTableSchema *table_schema = nullptr;
if (OB_ISNULL(session)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get my session", K(ret));
} else if (OB_ISNULL(GCTX.schema_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get schema_service", K(ret));
} else if (OB_FAIL(expr.eval_param_value(ctx, tenant_id, database_id, table_id))) {
LOG_WARN("failed to eval table id", K(ret));
} else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(session->get_effective_tenant_id(), schema_guard))) {
LOG_WARN("failed to get schema guard", K(ret));
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id->get_int(), table_id->get_int(), table_schema))) {
LOG_WARN("failed to get table schema", K(ret), K(table_id->get_int()));
} else if (nullptr == table_schema) {
expr_datum.set_null();
} else {
ObSchemaPrinter schema_printer(schema_guard);
int64_t pos = 0;
char *buf = expr.get_str_res_mem(ctx, MAX_TABLE_STATUS_CREATE_OPTION_LENGTH);
if (OB_ISNULL(buf)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to alloc buf", K(ret));
} else if (FALSE_IT(MEMSET(buf, 0, MAX_TABLE_STATUS_CREATE_OPTION_LENGTH))) {
} else if (OB_FAIL(schema_printer.print_table_definition_table_options(*table_schema,
buf,
MAX_TABLE_STATUS_CREATE_OPTION_LENGTH,
pos,
true))) {
LOG_WARN("print table definition table options failed", K(ret));
} else {
int64_t len = strlen(buf);
if (len > 0) {
expr_datum.set_string(ObString(len, buf));
} else {
expr_datum.set_null();
}
}
}
return ret;
}
DEF_SET_LOCAL_SESSION_VARS(ObExprInnerTableOptionPrinter, raw_expr) {
int ret = OB_SUCCESS;
if (lib::is_mysql_mode()) {
SET_LOCAL_SYSVAR_CAPACITY(1);
EXPR_ADD_LOCAL_SYSVAR(SYS_VAR_COLLATION_CONNECTION);
}
return ret;
}
ObExprInnerTableSequenceGetter::ObExprInnerTableSequenceGetter(ObIAllocator &alloc)
: ObExprOperator(alloc, T_FUN_SYS_INNER_TABLE_SEQUENCE_GETTER, N_INNER_TABLE_SEQUENCE_GETTER, 3,
VALID_FOR_GENERATED_COL, INTERNAL_IN_MYSQL_MODE, INTERNAL_IN_ORACLE_MODE)
{
}
ObExprInnerTableSequenceGetter::~ObExprInnerTableSequenceGetter()
{
}
inline int ObExprInnerTableSequenceGetter::calc_result_type3(ObExprResType &type,
ObExprResType &type1,
ObExprResType &type2,
ObExprResType &type3,
common::ObExprTypeCtx &type_ctx) const
{
int ret = OB_SUCCESS;
UNUSED(type_ctx);
type1.set_calc_type(ObIntType);
type2.set_calc_type(ObIntType);
type3.set_calc_type(ObIntType);
type.set_type(ObUInt64Type);
return ret;
}
int ObExprInnerTableSequenceGetter::cg_expr(ObExprCGCtx &, const ObRawExpr &, ObExpr &rt_expr) const
{
int ret = OB_SUCCESS;
CK(3 == rt_expr.arg_cnt_);
rt_expr.eval_func_ = &ObExprInnerTableSequenceGetter::eval_inner_table_sequence_getter;
return ret;
}
int ObExprInnerTableSequenceGetter::eval_inner_table_sequence_getter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &expr_datum)
{
int ret = OB_SUCCESS;
share::schema::ObSchemaGetterGuard schema_guard;
const ObSQLSessionInfo *session = ctx.exec_ctx_.get_my_session();
ObDatum *tenant_id = nullptr;
ObDatum *table_id = nullptr;
ObDatum *auto_inc_col_id = nullptr;
const ObTableSchema *table_schema = nullptr;
if (OB_ISNULL(session)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get my session", K(ret));
} else if (OB_ISNULL(GCTX.schema_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get schema_service", K(ret));
} else if (OB_FAIL(expr.eval_param_value(ctx, tenant_id, table_id, auto_inc_col_id))) {
LOG_WARN("failed to eval table id", K(ret));
} else if (auto_inc_col_id->is_null() || 0 == auto_inc_col_id->get_int()) {
expr_datum.set_null();
} else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(session->get_effective_tenant_id(), schema_guard))) {
LOG_WARN("failed to get schema guard", K(ret));
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id->get_int(), table_id->get_int(), table_schema))) {
LOG_WARN("failed to get table schema", K(ret), K(table_id->get_int()));
} else if (nullptr == table_schema) {
expr_datum.set_null();
} else {
uint64_t auto_increment = 0;
if (OB_FAIL(share::ObAutoincrementService::get_instance().get_sequence_value(
table_schema->get_tenant_id(), table_schema->get_table_id(),
table_schema->get_autoinc_column_id(), table_schema->is_order_auto_increment_mode(),
table_schema->get_truncate_version(), auto_increment))) {
SHARE_SCHEMA_LOG(WARN, "fail to get auto_increment value", K(ret));
} else if (auto_increment > 0) {
if (table_schema->get_auto_increment() > auto_increment) {
auto_increment = table_schema->get_auto_increment();
}
expr_datum.set_uint(auto_increment);
} else {
expr_datum.set_null();
}
}
return ret;
}
DEF_SET_LOCAL_SESSION_VARS(ObExprInnerTableSequenceGetter, raw_expr) {
int ret = OB_SUCCESS;
if (lib::is_mysql_mode()) {
SET_LOCAL_SYSVAR_CAPACITY(1);
EXPR_ADD_LOCAL_SYSVAR(share::SYS_VAR_COLLATION_CONNECTION);
}
return ret;
}
}
}

View File

@ -0,0 +1,65 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef _OB_SQL_EXPR_INNER_TABLE_OPTION_PRINTER_H_
#define _OB_SQL_EXPR_INNER_TABLE_OPTION_PRINTER_H_
#include "sql/engine/expr/ob_expr_operator.h"
namespace oceanbase
{
namespace sql
{
class ObExprInnerTableOptionPrinter : public ObStringExprOperator
{
public:
explicit ObExprInnerTableOptionPrinter(common::ObIAllocator &alloc);
virtual ~ObExprInnerTableOptionPrinter();
virtual int calc_result_type3(ObExprResType &type,
ObExprResType &type1,
ObExprResType &type2,
ObExprResType &type3,
common::ObExprTypeCtx &type_ctx) const;
virtual int cg_expr(ObExprCGCtx &op_cg_ctx,
const ObRawExpr &raw_expr,
ObExpr &rt_expr) const override;
static int eval_inner_table_option_printer(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &expr_datum);
DECLARE_SET_LOCAL_SESSION_VARS;
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObExprInnerTableOptionPrinter);
};
class ObExprInnerTableSequenceGetter : public ObExprOperator
{
public:
explicit ObExprInnerTableSequenceGetter(common::ObIAllocator &alloc);
virtual ~ObExprInnerTableSequenceGetter();
virtual int calc_result_type3(ObExprResType &type,
ObExprResType &type1,
ObExprResType &type2,
ObExprResType &type3,
common::ObExprTypeCtx &type_ctx) const;
virtual int cg_expr(ObExprCGCtx &op_cg_ctx,
const ObRawExpr &raw_expr,
ObExpr &rt_expr) const override;
static int eval_inner_table_sequence_getter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &expr_datum);
DECLARE_SET_LOCAL_SESSION_VARS;
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObExprInnerTableSequenceGetter);
};
}
}
#endif /* _OB_SQL_EXPR_INNER_TABLE_OPTION_PRINTER_H_ */

View File

@ -64,6 +64,7 @@
#include "sql/engine/expr/ob_expr_agg_param_list.h"
#include "sql/engine/expr/ob_expr_is_serving_tenant.h"
#include "sql/engine/expr/ob_expr_hex.h"
#include "sql/engine/expr/ob_expr_password.h"
#include "sql/engine/expr/ob_expr_in.h"
#include "sql/engine/expr/ob_expr_not_in.h"
#include "sql/engine/expr/ob_expr_int2ip.h"
@ -438,6 +439,7 @@
#include "sql/engine/expr/ob_expr_st_symdifference.h"
#include "sql/engine/expr/ob_expr_priv_st_asmvtgeom.h"
#include "sql/engine/expr/ob_expr_priv_st_makevalid.h"
#include "sql/engine/expr/ob_expr_inner_table_option_printer.h"
#include "sql/engine/expr/ob_expr_lock_func.h"
@ -675,6 +677,7 @@ void ObExprOperatorFactory::register_expr_operators()
REG_OP(ObExprGreaterThan);
REG_OP(ObExprGreatest);
REG_OP(ObExprHex);
REG_OP(ObExprPassword);
REG_OP(ObExprIn);
REG_OP(ObExprNotIn);
REG_OP(ObExprInt2ip);
@ -1077,6 +1080,8 @@ void ObExprOperatorFactory::register_expr_operators()
REG_OP(ObExprPrivSTAsMVTGeom);
REG_OP(ObExprPrivSTMakeValid);
REG_OP(ObExprCurrentRole);
REG_OP(ObExprInnerTableOptionPrinter);
REG_OP(ObExprInnerTableSequenceGetter);
}();
// 注册oracle系统函数
REG_OP_ORCL(ObExprSysConnectByPath);
@ -1398,6 +1403,8 @@ void ObExprOperatorFactory::register_expr_operators()
REG_OP_ORCL(ObExprTransactionId);
REG_OP_ORCL(ObExprInnerRowCmpVal);
REG_OP_ORCL(ObExprLastRefreshScn);
REG_OP_ORCL(ObExprInnerTableOptionPrinter);
REG_OP_ORCL(ObExprInnerTableSequenceGetter);
// REG_OP_ORCL(ObExprTopNFilter);
// REG_OP_ORCL(ObExprSdoRelate);
}

View File

@ -0,0 +1,118 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX SQL_ENG
#include "sql/engine/expr/ob_expr_password.h"
#include "sql/engine/expr/ob_expr_util.h"
#include "lib/encrypt/ob_encrypted_helper.h"
using namespace oceanbase::common;
namespace oceanbase
{
namespace sql
{
ObExprPassword::ObExprPassword(ObIAllocator &alloc)
: ObStringExprOperator(alloc, T_FUN_SYS_PASSWORD, N_PASSWORD, 1, VALID_FOR_GENERATED_COL)
{
}
ObExprPassword::~ObExprPassword()
{
}
# define SHA_PASSWORD_CHAR_LENGTH (SHA_DIGEST_LENGTH * 2 + 2)
int ObExprPassword::calc_result_type1(ObExprResType &type, ObExprResType &text,
common::ObExprTypeCtx &type_ctx) const
{
int ret = OB_SUCCESS;
type.set_type(ObVarcharType);
type.set_length(SHA_PASSWORD_CHAR_LENGTH);
type.set_collation_type(type_ctx.get_coll_type());
type.set_collation_level(CS_LEVEL_COERCIBLE);
text.set_calc_type(ObVarcharType);
ObExprResType tmp_type;
OZ(ObExprOperator::aggregate_charsets_for_string_result(tmp_type, &text, 1,
type_ctx.get_coll_type()));
OX(text.set_calc_collation_type(tmp_type.get_collation_type()));
OX(text.set_calc_collation_level(tmp_type.get_collation_level()));
return ret;
}
int ObExprPassword::cg_expr(ObExprCGCtx &, const ObRawExpr &, ObExpr &rt_expr) const
{
int ret = OB_SUCCESS;
rt_expr.eval_func_ = eval_password;
return ret;
}
int ObExprPassword::eval_password(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &expr_datum)
{
int ret = OB_SUCCESS;
ObDatum *arg = NULL;
if (OB_FAIL(expr.eval_param_value(ctx, arg))) {
LOG_ERROR("evaluate parameter value failed", K(ret), K(arg));
} else if (arg->is_null()) {
expr_datum.set_string("", 0);
} else if (arg->get_string().empty()) {
expr_datum.set_string("", 0);
} else {
ObEvalCtx::TempAllocGuard alloc_guard(ctx);
char *enc_buf = static_cast<char *>(alloc_guard.get_allocator().alloc(SHA_PASSWORD_CHAR_LENGTH));
if (enc_buf == NULL) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc memory failed", K(ret));
} else {
ObString tmp_str;
ObString upp_str;
ObString res_str;
ObString str_in_ctx;
tmp_str.assign_ptr(enc_buf, SHA_PASSWORD_CHAR_LENGTH);
if (OB_FAIL(ObEncryptedHelper::encrypt_passwd_to_stage2(arg->get_string(), tmp_str))) {
LOG_WARN("encrypt password failed", K(ret));
} else if (OB_FAIL(ObCharset::toupper(ObCollationType::CS_TYPE_UTF8MB4_GENERAL_CI,
tmp_str,
upp_str,
alloc_guard.get_allocator()))) {
LOG_WARN("convert string to upper failed", K(ret), K(tmp_str));
} else if (OB_FAIL(ObExprUtil::convert_string_collation(upp_str,
ObCollationType::CS_TYPE_UTF8MB4_GENERAL_CI,
res_str,
expr.datum_meta_.cs_type_,
alloc_guard.get_allocator()))) {
LOG_WARN("convert string collation failed", K(ret), K(upp_str));
} else if (OB_FAIL(ObExprUtil::deep_copy_str(res_str, str_in_ctx, ctx.get_expr_res_alloc()))) {
LOG_WARN("failed to cpoy str to context", K(ret));
} else {
expr_datum.set_string(str_in_ctx);
LOG_USER_WARN(OB_ERR_DEPRECATED_SYNTAX_NO_REP, "\'PASSWORD\'");
}
}
}
return ret;
}
DEF_SET_LOCAL_SESSION_VARS(ObExprPassword, raw_expr) {
int ret = OB_SUCCESS;
if (lib::is_mysql_mode()) {
SET_LOCAL_SYSVAR_CAPACITY(1);
EXPR_ADD_LOCAL_SYSVAR(share::SYS_VAR_COLLATION_SERVER);
}
return ret;
}
}
}

View File

@ -0,0 +1,44 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_SQL_ENGINE_EXPR_OB_EXPR_PASSWORD_
#define OCEANBASE_SQL_ENGINE_EXPR_OB_EXPR_PASSWORD_
#include "sql/engine/expr/ob_expr_operator.h"
namespace oceanbase
{
namespace sql
{
class ObExprPassword : public ObStringExprOperator
{
public:
explicit ObExprPassword(common::ObIAllocator &alloc);
virtual ~ObExprPassword();
virtual int calc_result_type1(ObExprResType &type,
ObExprResType &text,
common::ObExprTypeCtx &type_ctx) const;
virtual int cg_expr(ObExprCGCtx &expr_cg_ctx,
const ObRawExpr &raw_expr,
ObExpr &rt_expr) const override;
static int eval_password(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &datum);
DECLARE_SET_LOCAL_SESSION_VARS;
private:
DISALLOW_COPY_AND_ASSIGN(ObExprPassword);
};
}
}
#endif

View File

@ -19,9 +19,11 @@
#include "lib/oblog/ob_log.h"
#include "share/object/ob_obj_cast.h"
#include "share/ob_compatibility_control.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/expr/ob_expr_result_type_util.h"
#include "sql/engine/expr/ob_expr_lob_utils.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase
{
@ -234,7 +236,30 @@ int ObExprReplace::eval_replace(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &exp
} else if (text->is_null()
|| (is_mysql && from->is_null())
|| (is_mysql && NULL != to && to->is_null())) {
expr_datum.set_null();
if (is_mysql && !from->is_null() && 0 == from->len_) {
ObSolidifiedVarsGetter helper(expr, ctx, ctx.exec_ctx_.get_my_session());
const ObSQLSessionInfo *session = ctx.exec_ctx_.get_my_session();
uint64_t compat_version = 0;
ObCompatType compat_type = COMPAT_MYSQL57;
bool is_enable = false;
if (OB_ISNULL(session)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session info is null", K(ret));
} else if (OB_FAIL(helper.get_compat_version(compat_version))) {
LOG_WARN("failed to get compat version", K(ret));
} else if (OB_FAIL(ObCompatControl::check_feature_enable(compat_version,
ObCompatFeatureType::FUNC_REPLACE_NULL, is_enable))) {
LOG_WARN("failed to check feature enable", K(ret));
} else if (OB_FAIL(session->get_compatibility_control(compat_type))) {
LOG_WARN("failed to get compat type", K(ret));
} else if (is_enable && COMPAT_MYSQL57 == compat_type) {
expr_datum.set_datum(*text);
} else {
expr_datum.set_null();
}
} else {
expr_datum.set_null();
}
} else if (is_clob && (0 == text->len_)) {
expr_datum.set_datum(*text);
} else if (!is_lob_res) { // non text tc inputs
@ -288,8 +313,9 @@ int ObExprReplace::eval_replace(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &exp
DEF_SET_LOCAL_SESSION_VARS(ObExprReplace, raw_expr) {
int ret = OB_SUCCESS;
SET_LOCAL_SYSVAR_CAPACITY(1);
SET_LOCAL_SYSVAR_CAPACITY(2);
EXPR_ADD_LOCAL_SYSVAR(share::SYS_VAR_COLLATION_CONNECTION);
EXPR_ADD_LOCAL_SYSVAR(share::SYS_VAR_OB_COMPATIBILITY_VERSION);
return ret;
}

View File

@ -101,6 +101,9 @@ ObExprSysContext::UserEnvParameter ObExprSysContext::userenv_parameters_[] =
{"CLIENT_INFO", eval_client_info},
{"MODULE", eval_module},
{"CLIENT_IDENTIFIER", eval_client_identifier},
{"PROXY_USER", eval_proxy_user},
{"PROXY_USERID", eval_proxy_user_id},
{"AUTHENTICATED_IDENTITY", eval_auth_identity},
};
ObExprSysContext::NLS_Lang ObExprSysContext::lang_map_[] =
{
@ -876,5 +879,80 @@ int ObExprSysContext::get_schema_guard(share::schema::ObSchemaGetterGuard &schem
return ret;
}
int ObExprSysContext::eval_proxy_user(const ObExpr &expr, ObDatum &res, const ObDatum &arg1,
const ObDatum &arg2, ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
UNUSED(arg1);
UNUSED(arg2);
const ObSQLSessionInfo *session = ctx.exec_ctx_.get_my_session();
CK(OB_NOT_NULL(session));
if (OB_SUCC(ret)) {
const ObString &user_name = session->get_proxy_user_name();
ObString out_user_name;
ObExprStrResAlloc res_alloc(expr, ctx);
ObEvalCtx::TempAllocGuard alloc_guard(ctx);
ObIAllocator &calc_alloc = alloc_guard.get_allocator();
OZ(ObExprUtil::convert_string_collation(user_name, ObCharset::get_system_collation(),
out_user_name, expr.datum_meta_.cs_type_,
calc_alloc));
OZ(deep_copy_ob_string(res_alloc, out_user_name, out_user_name));
OX(res.set_string(out_user_name));
}
return ret;
}
int ObExprSysContext::eval_proxy_user_id(const ObExpr &expr, ObDatum &res,
const ObDatum &arg1, const ObDatum &arg2,
ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
UNUSED(arg1);
UNUSED(arg2);
const ObSQLSessionInfo *session = ctx.exec_ctx_.get_my_session();
CK(OB_NOT_NULL(session));
if (OB_SUCC(ret)) {
uint64_t id = session->get_proxy_user_id();
char out_id[256];
sprintf(out_id, "%lu", id);
ObString out_id_str(strlen(out_id), out_id);
ObExprStrResAlloc res_alloc(expr, ctx);
ObEvalCtx::TempAllocGuard alloc_guard(ctx);
ObIAllocator &calc_alloc = alloc_guard.get_allocator();
OZ(ObExprUtil::convert_string_collation(out_id_str, ObCharset::get_system_collation(),
out_id_str, expr.datum_meta_.cs_type_,
calc_alloc));
OZ(deep_copy_ob_string(res_alloc, out_id_str, out_id_str));
OX(res.set_string(out_id_str));
}
return ret;
}
int ObExprSysContext::eval_auth_identity(const ObExpr &expr, ObDatum &res, const ObDatum &arg1,
const ObDatum &arg2, ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
UNUSED(arg1);
UNUSED(arg2);
const ObSQLSessionInfo *session = ctx.exec_ctx_.get_my_session();
CK(OB_NOT_NULL(session));
if (OB_SUCC(ret)) {
const ObString user_name = session->get_user_name();
ObString out_user_name;
ObExprStrResAlloc res_alloc(expr, ctx);
ObEvalCtx::TempAllocGuard alloc_guard(ctx);
ObIAllocator &calc_alloc = alloc_guard.get_allocator();
OZ(ObExprUtil::convert_string_collation(user_name, ObCharset::get_system_collation(),
out_user_name, expr.datum_meta_.cs_type_,
calc_alloc));
OZ(deep_copy_ob_string(res_alloc, out_user_name, out_user_name));
OX(res.set_string(out_user_name));
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -99,6 +99,15 @@ public:
static int eval_application_context(const ObExpr &expr, common::ObDatum &res,
const common::ObString &arg1, const common::ObString &arg2,
ObEvalCtx &ctx);
static int eval_proxy_user(const ObExpr &expr, ObDatum &res, const ObDatum &arg1,
const ObDatum &arg2, ObEvalCtx &ctx);
static int eval_proxy_user_id(const ObExpr &expr, ObDatum &res,
const ObDatum &arg1, const ObDatum &arg2,
ObEvalCtx &ctx);
static int eval_auth_identity(const ObExpr &expr, ObDatum &res, const ObDatum &arg1,
const ObDatum &arg2, ObEvalCtx &ctx);
typedef int (*eval_fun)(const ObExpr &expr, common::ObDatum &res,
const common::ObDatum &arg1,
const common::ObDatum &arg2, ObEvalCtx &ctx);

View File

@ -74,13 +74,14 @@ int ObExprSysPrivilegeCheck::check_show_priv(bool &allow_show,
if (OB_UNLIKELY(NULL == schema_guard)) {
ret = OB_SCHEMA_ERROR;
}
exec_ctx.get_my_session()->get_session_priv_info(session_priv);
allow_show = true;
if (OB_SUCC(ret)) {
//tenant_id in table is static casted to int64_t,
//and use statis_cast<uint64_t> for retrieving(same with schema_service)
// schema拆分后,普通租户schema表的tenant_id为0,此时鉴权取session_priv.tenant_id_
if (session_priv.tenant_id_ != static_cast<uint64_t>(tenant_id)
if (OB_FAIL(exec_ctx.get_my_session()->get_session_priv_info(session_priv))) {
LOG_WARN("fail to get session priv info", K(ret));
} else if (session_priv.tenant_id_ != static_cast<uint64_t>(tenant_id)
&& OB_INVALID_TENANT_ID != tenant_id) {
//not current tenant's row
} else if (0 == level_str.case_compare("db_acc")) {

View File

@ -884,7 +884,7 @@ DEF_SET_LOCAL_SESSION_VARS(ObExprTrim, raw_expr) {
int ret = OB_SUCCESS;
if (lib::is_mysql_mode()) {
SET_LOCAL_SYSVAR_CAPACITY(1);
EXPR_ADD_LOCAL_SYSVAR(SYS_VAR_COLLATION_CONNECTION);
EXPR_ADD_LOCAL_SYSVAR(share::SYS_VAR_COLLATION_CONNECTION);
}
return ret;
}

View File

@ -949,3 +949,19 @@ int ObSolidifiedVarsGetter::get_max_allowed_packet(int64_t &max_size)
}
return ret;
}
int ObSolidifiedVarsGetter::get_compat_version(uint64_t &compat_version)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(session_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else if (OB_FAIL(session_->get_compatibility_version(compat_version))) {
LOG_WARN("failed to get compat version", K(ret));
} else if (NULL != local_session_var_
&& OB_FAIL(ObSQLUtils::merge_solidified_var_into_compat_version(local_session_var_->get_local_vars(),
compat_version))) {
LOG_WARN("try get local compat version failed", K(ret));
}
return ret;
}

View File

@ -277,6 +277,7 @@ public:
int get_local_nls_timestamp_tz_format(ObString &format);
int get_local_nls_format_by_type(const ObObjType type, ObString &format_str);
int get_max_allowed_packet(int64_t &max_size);
int get_compat_version(uint64_t &compat_version);
//get the specified solidified var
int get_local_var(share::ObSysVarClassType var_type, share::schema::ObSessionSysVar *&sys_var);
private: