/** * 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 RS #include "rootserver/ob_ddl_service.h" #include "lib/oblog/ob_log.h" #include "lib/time/ob_time_utility.h" #include "lib/string/ob_strings.h" #include "lib/string/ob_sql_string.h" #include "lib/hash/ob_placement_hashset.h" #include "lib/hash/ob_placement_hashmap.h" #include "lib/hash/ob_hashmap.h" #include "lib/worker.h" #include "lib/container/ob_array_iterator.h" #include "lib/mysqlclient/ob_mysql_transaction.h" #include "share/ob_srv_rpc_proxy.h" #include "common/ob_timeout_ctx.h" #include "common/rowkey/ob_rowkey.h" #include "share/ob_define.h" #include "share/inner_table/ob_inner_table_schema.h" #include "share/schema/ob_table_schema.h" #include "share/schema/ob_multi_version_schema_service.h" #include "share/schema/ob_part_mgr_util.h" #include "share/schema/ob_schema_printer.h" #include "share/schema/ob_schema_utils.h" #include "share/schema/ob_ddl_sql_service.h" #include "share/schema/ob_security_audit_sql_service.h" #include "share/schema/ob_user_sql_service.h" #include "share/schema/ob_schema_service_sql_impl.h" #include "share/ob_autoincrement_service.h" #include "share/config/ob_server_config.h" #include "share/ob_primary_zone_util.h" #include "share/ob_replica_info.h" #include "share/ob_index_builder_util.h" #include "share/sequence/ob_sequence_ddl_proxy.h" #include "share/ob_schema_status_proxy.h" #include "share/ob_tenant_mgr.h" #include "lib/worker.h" #include "share/ob_schema_status_proxy.h" #include "share/ob_global_stat_proxy.h" #include "share/ob_freeze_info_proxy.h" #include "share/ob_service_epoch_proxy.h" #include "sql/resolver/ob_stmt_type.h" #include "sql/resolver/ddl/ob_ddl_resolver.h" #include "sql/resolver/expr/ob_raw_expr_modify_column_name.h" #include "sql/resolver/expr/ob_raw_expr_printer.h" #include "ob_server_manager.h" #include "ob_zone_manager.h" #include "rootserver/ob_schema2ddl_sql.h" #include "rootserver/ob_unit_manager.h" #include "rootserver/ob_index_builder.h" #include "rootserver/ob_zone_unit_provider.h" #include "rootserver/ob_rs_job_table_operator.h" #include "rootserver/ob_ddl_sql_generator.h" #include "rootserver/ddl_task/ob_ddl_task.h" #include "rootserver/ob_ddl_help.h" #include "rootserver/ob_locality_util.h" #include "rootserver/ob_root_service.h" #include "rootserver/ob_vertical_partition_builder.h" #include "rootserver/ddl_task/ob_constraint_task.h" #include "rootserver/ddl_task/ob_ddl_retry_task.h" #include "rootserver/freeze/ob_freeze_info_manager.h" #include "rootserver/freeze/ob_major_freeze_helper.h" #include "rootserver/ob_primary_ls_service.h"//ObTenantLSInfo #include "lib/utility/ob_tracepoint.h" #include "observer/ob_server_struct.h" #include "storage/tx/ob_ts_mgr.h" #include "storage/tx/ob_i_ts_source.h" #include "sql/engine/px/ob_px_util.h" #include "share/ob_unit_replica_counter.h" #include "sql/ob_sql_utils.h" #include "observer/omt/ob_tenant_timezone_mgr.h" #include "share/schema/ob_schema_mgr.h" #include "rootserver/ob_tablet_creator.h" #include "rootserver/ob_table_creator.h" #include "rootserver/ob_balance_group_ls_stat_operator.h" #include "share/ob_share_util.h" #include "share/ob_leader_election_waiter.h" #include "rootserver/ob_tablet_drop.h" #include "share/schema/ob_context_mgr.h" #include "share/schema/ob_context_ddl_proxy.h" #include "share/ob_global_context_operator.h" #include "share/ls/ob_ls_creator.h" #include "share/ls/ob_ls_operator.h" #include "ob_lob_meta_builder.h" #include "ob_lob_piece_builder.h" #include "share/ls/ob_ls_life_manager.h"//ObLSLifeAgentManager #include "share/restore/ob_physical_restore_table_operator.h"//ObPhysicalRestoreTableOperator #include "storage/tablelock/ob_table_lock_rpc_client.h" #include "rootserver/restore/ob_restore_util.h"//insert_user_tenant_restore_job #include "logservice/palf/palf_base_info.h"//PalfBaseInfo namespace oceanbase { using namespace common; using namespace share; using namespace obrpc; using namespace storage; namespace rootserver { #define MODIFY_LOCALITY_NOT_ALLOWED() \ do { \ ret = OB_OP_NOT_ALLOW; \ LOG_WARN("modify locality is not allowed", K(ret)); \ } while (0) // The input of value must be a string #define SET_TENANT_VARIABLE(sysvar_id, value) \ if (OB_SUCC(ret)) {\ int64_t store_idx = OB_INVALID_INDEX; \ if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(sysvar_id, store_idx))) { \ LOG_WARN("failed to calc sys var store idx", KR(ret), K(sysvar_id)); \ } else if (OB_UNLIKELY(store_idx < 0 \ || store_idx >= ObSysVarFactory::ALL_SYS_VARS_COUNT)) { \ ret = OB_ERR_UNEXPECTED; \ LOG_WARN("got store_idx is invalid", K(ret), K(store_idx)); \ } else if (OB_FAIL(sys_params[store_idx].init( \ sys_variable_schema.get_tenant_id(),\ ObSysVariables::get_name(store_idx),\ ObSysVariables::get_type(store_idx),\ value,\ ObSysVariables::get_min(store_idx),\ ObSysVariables::get_max(store_idx),\ ObSysVariables::get_info(store_idx),\ ObSysVariables::get_flags(store_idx)))) {\ LOG_WARN("failed to set tenant variable", \ KR(ret), K(value), K(sysvar_id), K(store_idx));\ }\ } // Convert macro integer to string for setting into system variable #define VAR_INT_TO_STRING(buf, value) \ if (OB_SUCC(ret)) {\ if (OB_FAIL(databuff_printf(buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%d", static_cast(value)))) {\ LOG_WARN("failed to print value in buf", K(value), K(ret));\ }\ } #define GRANT_SYS_ROLE_NUM 2 /* len of role array is 2 */ #define GRANT_ROLE_MIN_ROLE_NUM 3 /* min len of role array is 3 */ ObDDLService::ObDDLService() : inited_(false), stopped_(false), rpc_proxy_(NULL), common_rpc_(NULL), sql_proxy_(NULL), schema_service_(NULL), lst_operator_(NULL), server_mgr_(NULL), zone_mgr_(NULL), unit_mgr_(NULL), snapshot_mgr_(NULL), pz_entity_cnt_lock_() { } int ObDDLService::init(obrpc::ObSrvRpcProxy &rpc_proxy, obrpc::ObCommonRpcProxy &common_rpc, common::ObMySQLProxy &sql_proxy, share::schema::ObMultiVersionSchemaService &schema_service, share::ObLSTableOperator &lst_operator, ObServerManager &server_mgr, ObZoneManager &zone_mgr, ObUnitManager &unit_mgr, ObSnapshotInfoManager &snapshot_mgr) { int ret = OB_SUCCESS; if (inited_) { ret = OB_INIT_TWICE; LOG_WARN("init twice"); } else { rpc_proxy_ = &rpc_proxy; common_rpc_ = &common_rpc; sql_proxy_ = &sql_proxy; schema_service_ = &schema_service; lst_operator_ = &lst_operator; server_mgr_ = &server_mgr; zone_mgr_ = &zone_mgr; unit_mgr_ = &unit_mgr; snapshot_mgr_ = &snapshot_mgr; stopped_ = false; inited_ = true; } return ret; } int ObDDLService::get_zones_in_region( const common::ObRegion ®ion, const common::ObIArray &zone_list, common::ObIArray &zones) { int ret = OB_SUCCESS; common::ObArray zone_infos; zones.reset(); if (OB_UNLIKELY(NULL == zone_mgr_)) { ret = OB_NOT_INIT; LOG_WARN("zone mgr in null", K(ret)); } else if (OB_FAIL(zone_mgr_->get_zone(zone_infos))) { LOG_WARN("fail to get zone", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < zone_infos.count(); ++i) { share::ObZoneInfo &zone_info = zone_infos.at(i); if (region == ObRegion(zone_info.region_.info_.ptr()) && has_exist_in_array(zone_list, ObString(zone_info.zone_.ptr()))) { if (OB_FAIL(zones.push_back(zone_info.zone_))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } else {} // not belongings, do nothing } } return ret; } int ObDDLService::get_tenant_schema_guard_with_version_in_inner_table(const uint64_t tenant_id, ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; bool is_standby = false; bool is_restore = false; bool use_local = false; int64_t version_in_inner_table = OB_INVALID_VERSION; ObRefreshSchemaStatus schema_status; if (OB_INVALID_TENANT_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant_id", K(ret), K(tenant_id)); } else if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service is null", K(ret)); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("failed to get is standby cluster", K(ret)); } else if (OB_FAIL(schema_service_->check_tenant_is_restore(NULL, tenant_id, is_restore))) { LOG_WARN("fail to check tenant is restore", KR(ret), K(tenant_id)); } else if ((is_standby || is_restore) && OB_SYS_TENANT_ID != tenant_id) { ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_; if (OB_ISNULL(schema_status_proxy)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_status_proxy is null", KR(ret)); } else if (OB_FAIL(schema_status_proxy->get_refresh_schema_status(tenant_id, schema_status))) { LOG_WARN("failed to get tenant refresh schema status", KR(ret), K(tenant_id)); } else if (OB_INVALID_VERSION == schema_status.readable_schema_version_) { // 1. The standalone cluster: the schema status has been reset, it can use the internal table to refresh, // in this time the standalone cluster already has a leader // 2. The second of physical recovery, after reset schema status, modify_schema can be modified use_local = false; } else if (is_standby) { use_local = true; } else if (is_restore) { ret = OB_NOT_SUPPORTED; LOG_WARN("tenant is still restoring, ddl not supported", KR(ret), K(tenant_id), K(schema_status)); } } if (OB_FAIL(ret)) { } else if (use_local) { // Only for the failover/switchover stage of the standalone cluster if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else { LOG_INFO("use local tenant schema guard", K(ret), K(tenant_id)); } } else { // 1. the normal tenants do DDL in primary cluster. // 2. restore tenant is in modify_schema stage in primary cluster. schema_status.tenant_id_ = tenant_id; // Ensure that the user tenant schema is updated to the latest if (OB_FAIL(schema_service_->get_schema_version_in_inner_table(*sql_proxy_, schema_status, version_in_inner_table))) { LOG_WARN("fail to get latest schema version in inner table", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard, version_in_inner_table))) { if (OB_SCHEMA_EAGAIN == ret) { int t_ret = OB_SUCCESS; ObArray tenant_ids; if (OB_SUCCESS != (t_ret = tenant_ids.push_back(tenant_id))) { LOG_WARN("fail to push back tenant_id", K(t_ret), K(tenant_id)); } else if (OB_SUCCESS != (t_ret = schema_service_->refresh_and_add_schema(tenant_ids))) { LOG_WARN("fail to refresh and add schema", K(t_ret), K(tenant_id)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard, version_in_inner_table))) { LOG_WARN("fail to retry get schema guard", K(ret), K(tenant_id), K(version_in_inner_table)); } } else { LOG_WARN("get schema manager failed!", K(ret)); } } } return ret; } int ObDDLService::check_tenant_in_alter_locality( const uint64_t tenant_id, bool &in_alter_locality) { int ret = OB_SUCCESS; share::schema::ObSchemaGetterGuard schema_guard; const share::schema::ObTenantSchema *tenant_schema = NULL; in_alter_locality = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("not init", K(ret)); } else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("tenant not exists", K(ret), K(tenant_id)); } else if (OB_UNLIKELY(NULL == tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", K(ret), K(tenant_id)); } else if (!tenant_schema->get_previous_locality_str().empty()) { // previous locality is not null, alter locality is processing in_alter_locality = true; } else { // do nothing } return ret; } int ObDDLService::create_user_tables( const bool if_not_exist, const ObString &ddl_stmt_str, const ObErrorInfo &error_info, ObIArray &table_schemas, const int64_t frozen_version, ObSchemaGetterGuard &schema_guard, const obrpc::ObSequenceDDLArg &sequence_ddl_arg, const uint64_t last_replay_log_id, const ObIArray *dep_infos, ObIArray &mock_fk_parent_table_schema_array) { int ret = OB_SUCCESS; RS_TRACE(create_user_tables_begin); uint64_t tenant_id = OB_INVALID_TENANT_ID; if (OB_FAIL(check_inner_stat())) { LOG_WARN("not init", K(ret)); } else if (table_schemas.count() < 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schemas have no element", K(ret)); } else { tenant_id = table_schemas.at(0).get_tenant_id(); // for checking unique index name duplicate when create user table in oracle mode bool is_oracle_mode = false; if (OB_FAIL(table_schemas.at(0).check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check if tenant mode is oracle mode", K(ret)); } else if (is_oracle_mode) { // oracle mode uint64_t i = 1; bool idx_name_is_exist = false; for (i = 1; OB_SUCC(ret) && !idx_name_is_exist && i < table_schemas.count(); ++i) { if (table_schemas.at(i).is_index_table()) { if (OB_FAIL(table_schemas.at(i).generate_origin_index_name())) { LOG_WARN("generate origin index name failed", K(ret), K(table_schemas.at(i).get_table_name_str())); } else if (OB_FAIL(check_index_table_exist(table_schemas.at(i).get_tenant_id(), table_schemas.at(i).get_database_id(), table_schemas.at(0).get_table_id(), table_schemas.at(i).get_origin_index_name_str(), schema_guard, idx_name_is_exist))) { LOG_WARN("failed to check index table", K(ret)); } } } if (OB_SUCC(ret)) { if(idx_name_is_exist && ((--i) < table_schemas.count())) { ret = OB_ERR_KEY_NAME_DUPLICATE; LOG_USER_ERROR(OB_ERR_KEY_NAME_DUPLICATE, table_schemas.at(i).get_origin_index_name_str().length(), table_schemas.at(i).get_origin_index_name_str().ptr()); LOG_WARN("duplicate index name", K(ret), K(table_schemas.at(i).get_table_name_str())); } } } else { // mysql mode // do nothing, only oracle mode need this } } if (OB_FAIL(ret)) { //do nothing } else if (OB_FAIL(create_tables_in_trans(if_not_exist, ddl_stmt_str, error_info, table_schemas, frozen_version, sequence_ddl_arg, last_replay_log_id, dep_infos, mock_fk_parent_table_schema_array))) { LOG_WARN("create_tables_in_trans failed", K(ret)); } RS_TRACE(public_schema_begin); if (OB_SUCC(ret)) { DEBUG_SYNC(CREATE_TABLE_BEFORE_PUBLISH_SCHEMA); if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", KR(ret), K(tenant_id)); } else { RS_TRACE(public_schema_end); } } RS_TRACE_EXT(create_user_tables_end, OB_ID(ret), ret); return ret; } int ObDDLService::create_inner_expr_index(ObMySQLTransaction &trans, const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema, ObIArray &new_columns, ObTableSchema &index_schema, const int64_t frozen_version, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; uint64_t tenant_id = new_table_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(check_table_exist(index_schema))) { if (OB_ERR_TABLE_EXIST != ret) { LOG_WARN("check_table_exist failed", K(index_schema), K(ret)); } } else { ObSchemaService *schema_service = schema_service_->get_schema_service(); uint64_t new_table_id = index_schema.get_table_id(); if (OB_UNLIKELY(NULL == schema_service)) { ret = OB_ERR_SYS; LOG_WARN("schema service must be not null", K(ret)); } else if (OB_FAIL(schema_service->fetch_new_table_id( index_schema.get_tenant_id(), new_table_id))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else { index_schema.set_table_id(new_table_id); } } if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); for (int64_t i = 0; OB_SUCC(ret) && i < new_columns.count(); ++i) { ObColumnSchemaV2 *new_column_schema = new_columns.at(i); if (OB_ISNULL(new_column_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("new column schema is null"); } else if (OB_FAIL(ddl_operator.insert_single_column( trans, new_table_schema, *new_column_schema))) { LOG_WARN("failed to create table schema, ", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.alter_table_options(schema_guard, new_table_schema, orig_table_schema, false, trans))) { LOG_WARN("alter table options failed", K(ret), K(new_table_schema)); } else if (OB_FAIL(ddl_operator.create_table( index_schema, trans, ddl_stmt_str, true, false))) { LOG_WARN("failed to create index schema", K(ret)); } } const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version(); if (OB_SUCC(ret)) { if (OB_INVALID_VERSION == last_schema_version) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid schema version", K(ret), K(last_schema_version)); } else if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans, tenant_id, index_schema.get_table_id(), last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", K(ret)); } } if (OB_SUCC(ret) && index_schema.has_tablet() && OB_FAIL(create_index_tablet(index_schema, trans, schema_guard))) { LOG_WARN("fail to create_index_tablet", KR(ret), K(index_schema)); } } return ret; } int ObDDLService::create_global_index( ObMySQLTransaction &trans, const obrpc::ObCreateIndexArg &arg, const share::schema::ObTableSchema &table_schema, share::schema::ObTableSchema &index_schema, const int64_t frozen_version) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(table_schema.check_create_index_on_hidden_primary_key(index_schema))) { LOG_WARN("fail to check create global index on table", K(ret), K(index_schema)); } else if (OB_FAIL(create_index_table(arg, index_schema, frozen_version, trans))) { LOG_WARN("fail to create global index", K(ret)); } return ret; } int ObDDLService::create_global_inner_expr_index( ObMySQLTransaction &trans, const obrpc::ObCreateIndexArg &arg, const share::schema::ObTableSchema &orig_table_schema, share::schema::ObTableSchema &new_table_schema, common::ObIArray &new_columns, share::schema::ObTableSchema &index_schema, const int64_t frozen_version) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(create_inner_expr_index(trans, orig_table_schema, new_table_schema, new_columns, index_schema, frozen_version, &arg.ddl_stmt_str_))) { LOG_WARN("fail to create inner expr index", K(ret)); } return ret; } // create_index_table is used by create index int ObDDLService::create_index_table( const obrpc::ObCreateIndexArg &arg, ObTableSchema &table_schema, const int64_t frozen_version, ObMySQLTransaction &sql_trans) { int ret = OB_SUCCESS; const ObString *ddl_stmt_str = &arg.ddl_stmt_str_; uint64_t new_table_id = table_schema.get_table_id(); // You can specify the data table id to build an index uint64_t tenant_id = table_schema.get_tenant_id(); ObSchemaService *schema_service = NULL; const ObDatabaseSchema *database_schema = NULL; ObSchemaGetterGuard schema_guard; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(check_table_exist(table_schema))) { if (OB_ERR_TABLE_EXIST != ret) { LOG_WARN("check_table_exist failed", K(table_schema), K(ret)); } } else { schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must not null"); } else if (OB_FAIL(set_tablegroup_id(table_schema))) { LOG_WARN("set_tablegroup_id failed", K(table_schema.get_tablegroup_name()), K(ret)); } else if (OB_FAIL(schema_service->fetch_new_table_id( table_schema.get_tenant_id(), new_table_id))) { LOG_WARN("fail to fetch new table id", K(ret)); } else { table_schema.set_table_id(new_table_id); // zone_list, resource_pool_list, primary_zone not set, copy from all_database_schema if (OB_FAIL(schema_guard.get_database_schema( tenant_id, table_schema.get_database_id(), database_schema))) { LOG_WARN("get_database_schema failed", K(tenant_id), K(table_schema.get_database_id()), K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database schema should not be null", K(ret)); } else if (!arg.is_inner_ && database_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not create index in recyclebin", K(ret), K(*database_schema)); } else {} // no more to do if (OB_SUCC(ret)) { if (OB_FAIL(try_check_and_set_table_schema_in_tablegroup(schema_guard, table_schema))) { LOG_WARN("check table schema in tablegroup failed", K(ret)); } } } if (OB_SUCC(ret)) { if (OB_SUCC(ret)) { if (OB_FAIL(create_table_in_trans(table_schema, frozen_version, ddl_stmt_str, &sql_trans, schema_guard))) { LOG_WARN("create_table_in_trans failed", KR(ret), K(frozen_version), K(ddl_stmt_str), K(table_schema)); } } } } return ret; } int ObDDLService::check_create_with_db_id(ObDatabaseSchema &schema) { int ret = OB_SUCCESS; const uint64_t db_id = schema.get_database_id(); if (OB_INVALID_ID != db_id) { const bool enable_sys_table_ddl = common::ObServerConfig::get_instance().enable_sys_table_ddl; char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE]; if (!enable_sys_table_ddl) { //Only when the configuration item switch is turned on can the internal table be created ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s", "create database with database_id"); } else if (!is_inner_db(db_id)) { ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s %lu", "create inner db with invalid database_id: ", db_id); } else { schema.set_database_id(db_id); } if (OB_FAIL(ret)) { LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg); } } return ret; } int ObDDLService::replace_table_schema_type(ObTableSchema &schema) { int ret = OB_SUCCESS; const uint64_t table_id = schema.get_table_id(); if (OB_INVALID_ID != table_id) { if (!schema.is_user_table() && !schema.is_view_table()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("should not reach here"); } else { const bool enable_sys_table_ddl = common::ObServerConfig::get_instance().enable_sys_table_ddl; char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE]; if (!enable_sys_table_ddl) { // Only when the configuration item switch is turned on can the internal table is created ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s", "create table with table_id"); } else if (!is_inner_table(table_id)) { // Cannot specify table_id to create non-internal table ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s %lu", "create inner table with invalid table_id: ", table_id); } else if (schema.is_user_table() && is_sys_view(table_id)) { // table_id of table cannot fall into the system view id range ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s", "create table with table_id which fall into sys_view interval"); } else if (schema.is_user_view() && !is_sys_view(table_id)) { // table_id of view cannot fall into the non-system view range ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s", "create view with table_id which fall into non-sys_view interval"); } else if (is_reserved_table_id(table_id)) { ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s", "table_id which fall into reserved interval"); } else { ObTableType type = get_inner_table_type_by_id(table_id); if (MAX_TABLE_TYPE != type) { schema.set_table_type(type); schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); schema.set_table_id(table_id); } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("type is error", K(type), K(ret)); } } if (OB_FAIL(ret)) { if (OB_OP_NOT_ALLOW == ret) { LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg); } else { LOG_USER_ERROR(OB_ERR_UNEXPECTED, err_msg); } } } } return ret; } // Actually, object_id can be allocated within the same ddl trans. // To avoid refactor more codes, generate_object_id_for_partition_schema() should be called with generate_tablet_id(). // // [@input]gen_subpart_only: // - True : for add/drop subpartition situations, part_id is valid and should not be generated again. // - False : other situations. int ObDDLService::generate_object_id_for_partition_schema( ObPartitionSchema &partition_schema, const bool gen_subpart_only /* = false*/) { int ret = OB_SUCCESS; const uint64_t tenant_id = partition_schema.get_tenant_id(); // table_id/tablegroup_id const int64_t schema_id = partition_schema.get_table_id(); ObPartitionLevel part_level = partition_schema.get_part_level(); ObSchemaService *schema_service = schema_service_->get_schema_service(); int64_t all_partition_num = OB_INVALID_ID; if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must not null", K(ret)); } else if (part_level >= PARTITION_LEVEL_MAX) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part level is unexpected", K(partition_schema), K(ret)); } else if (is_inner_table(schema_id) || is_sys_tablegroup_id(schema_id) || PARTITION_LEVEL_ZERO == part_level) { // 1. For non-partitioned schemas'(include view、virtual table): // object_id is equal to its schema_id. // 2. For partitioned virtual table(list columns only): // object_id for its partition is hard code by schema. // For the above reasons, we won't allocate object_id for table/tablegroup } else if (0 >= (all_partition_num = partition_schema.get_all_part_num())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid partition num", KR(ret), K(all_partition_num), K(partition_schema)); } else { // For table/tablegroup's partitions ObObjectID max_object_id = OB_INVALID_ID; ObObjectID min_object_id = OB_INVALID_ID; ObObjectID object_id = OB_INVALID_ID; ObPartition** part_array = partition_schema.get_part_array(); int64_t partition_num = partition_schema.get_partition_num(); int64_t first_part_num = partition_schema.get_first_part_num(); int64_t subpartition_num = PARTITION_LEVEL_TWO == part_level ? all_partition_num : 0; int64_t object_num = subpartition_num; if (!gen_subpart_only) { object_num += first_part_num; } if (first_part_num <= 0 || all_partition_num <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid first_part_num/all_partition_num", KR(ret), K(first_part_num), K(subpartition_num), K(partition_schema)); } else if (OB_ISNULL(part_array) || first_part_num != partition_num) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_array is empty or first_part_num is not equal to partition_num", KR(ret), K(partition_schema)); } else if (OB_FAIL(schema_service->fetch_new_partition_ids(tenant_id, object_num, max_object_id))) { LOG_WARN("fail to get max object id", KR(ret), K(tenant_id), K(object_num)); } else if (0 >= (min_object_id = max_object_id - object_num + 1)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("min_object_id should be greator than 0", KR(ret), K(min_object_id), K(max_object_id), K(object_num)); } else { object_id = min_object_id; } // 1. generate object_id for partitions if (OB_SUCC(ret) && !gen_subpart_only) { for (int64_t i = 0; OB_SUCC(ret) && i < partition_num; i++) { ObPartition* part = part_array[i]; if (OB_ISNULL(part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", KR(ret), K(i)); } else if (object_id > max_object_id) { ret = OB_ERR_UNEXPECTED; LOG_WARN("object_id is invalid", KR(ret), K(object_id), K(max_object_id), K(partition_schema)); } else { part->set_part_id(object_id++); } } // end for } // 2. generate object_id for subpartitions if (OB_SUCC(ret) && PARTITION_LEVEL_TWO == part_level) { for (int64_t i = 0; OB_SUCC(ret) && i < partition_num; i++) { ObPartition* part = part_array[i]; if (OB_ISNULL(part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", KR(ret), K(i)); } else if (part->get_part_id() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_id is invalid", KR(ret), KPC(part)); } else if (OB_ISNULL(part->get_subpart_array()) || part->get_subpartition_num() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sub_part_array is null or invalid subpartition num", KR(ret), KPC(part)); } else { for (int64_t j = 0; OB_SUCC(ret) && j < part->get_subpartition_num(); j++) { ObSubPartition *subpart = part->get_subpart_array()[j]; if (OB_ISNULL(subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("subpart is null", KR(ret), KPC(part), K(j)); } else if (object_id > max_object_id) { ret = OB_ERR_UNEXPECTED; LOG_WARN("object_id is invalid", KR(ret), K(object_id), K(max_object_id), K(partition_schema)); } else { subpart->set_part_id(part->get_part_id()); subpart->set_sub_part_id(object_id++); } } // end for subpart } } // end for part } } return ret; } int ObDDLService::generate_tables_tablet_id(ObIArray &table_schemas) { int ret = OB_SUCCESS; for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); i++) { if (OB_FAIL(generate_tablet_id(table_schemas.at(i)))) { LOG_WARN("fail to generate_tablet_id", KR(ret), K(table_schemas.at(i))); } } return ret; } int ObDDLService::generate_tablet_id(ObTableSchema &table_schema) { int ret = OB_SUCCESS; uint64_t new_tablet_id = OB_INVALID_ID; ObPartitionLevel part_level = table_schema.get_part_level(); ObSchemaService *schema_service = schema_service_->get_schema_service(); uint64_t tablet_num = OB_INVALID_ID; if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must not null", K(ret)); } else if (part_level >= PARTITION_LEVEL_MAX) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part level is unexpected", K(table_schema), K(ret)); } else if (is_sys_table(table_schema.get_table_id())) { table_schema.set_tablet_id(table_schema.get_table_id()); } else if (table_schema.is_vir_table() || table_schema.is_view_table()) { } else if ((tablet_num = table_schema.get_all_part_num()) <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to get tablet num", K(table_schema), K(ret), K(tablet_num)); } else if (OB_FAIL(schema_service->fetch_new_tablet_ids( table_schema, new_tablet_id, tablet_num))) { LOG_WARN("fail to fetch new table id", K(table_schema), K(ret)); } else if (PARTITION_LEVEL_ZERO == part_level) { table_schema.set_tablet_id(new_tablet_id); } else { uint64_t max_id = new_tablet_id + tablet_num - 1; ObPartition **part_array = table_schema.get_part_array(); int64_t part_num = table_schema.get_partition_num(); if (OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part array is null", K(table_schema), K(ret)); } else { for (int64_t i = 0; i < part_num && OB_SUCC(ret); ++i) { if (OB_ISNULL(part_array[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(i), K(table_schema), K(ret)); } else if (PARTITION_LEVEL_ONE == part_level) { if (max_id < new_tablet_id) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fetch max id error", K(table_schema), K(max_id), K(ret)); } else { part_array[i]->set_tablet_id(new_tablet_id++); } } else if (PARTITION_LEVEL_TWO == part_level) { ObSubPartition **sub_part_array = part_array[i]->get_subpart_array(); int64_t sub_part_num = part_array[i]->get_subpartition_num(); if (OB_ISNULL(sub_part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part array is null", K(table_schema), K(ret)); } else { for (int64_t j = 0; j < sub_part_num && OB_SUCC(ret); j++) { if (OB_ISNULL(sub_part_array[j])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(j), K(table_schema), K(ret)); } else if (max_id < new_tablet_id) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fetch max id error", K(table_schema), K(max_id), K(ret)); } else { sub_part_array[j]->set_tablet_id(new_tablet_id++); } } } } else { ret = OB_NOT_SUPPORTED; LOG_WARN("4.0 not support part type", K(table_schema), K(ret)); } } } } return ret; } // For create table/tablegroup // 1. Add missing partition/subpartition schema. // 2. Reorganize part_idx/subpart_idx. int ObDDLService::try_format_partition_schema(ObPartitionSchema &partition_schema) { int ret = OB_SUCCESS; // 1. generate missing partition/subpartition. bool generated = false; if (OB_FAIL(partition_schema.try_generate_hash_part())) { LOG_WARN("fail to generate hash part", KR(ret), K(partition_schema)); } else if (OB_FAIL(partition_schema.try_generate_hash_subpart(generated))) { LOG_WARN("fail to generate hash part", KR(ret), K(partition_schema)); } else if (generated) { // skip } else if (OB_FAIL(partition_schema.try_generate_subpart_by_template(generated))) { LOG_WARN("fail to generate_subpart_by_template", KR(ret), K(partition_schema)); } if (OB_SUCC(ret) && generated) { partition_schema.set_sub_part_template_def_valid(); LOG_INFO("convert schema to nontemplate", K(partition_schema)); } // 2. generate part_idx/subpart_idx. if (FAILEDx(partition_schema.try_init_partition_idx())) { LOG_WARN("fail to init partition idx", KR(ret)); } return ret; } /* generate_schema is called when creating a data table, * IndexBuilder::generate_schema is called when create a index table. */ int ObDDLService::generate_schema( const ObCreateTableArg &arg, ObTableSchema &schema, const int64_t frozen_version) { int ret = OB_SUCCESS; const ObIArray &constraints = arg.constraint_list_; const uint64_t tenant_id = schema.get_tenant_id(); uint64_t new_table_id = schema.get_table_id(); ObSchemaService *schema_service = NULL; const ObDatabaseSchema *database_schema = NULL; const ObTenantSchema *tenant_schema = NULL; const ObTablespaceSchema *tablespace_schema = NULL; bool is_oracle_mode = false; ObSchemaGetterGuard guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, guard))) { LOG_WARN("get schema guard failed", K(ret)); } else { schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must not null", K(ret)); } } // support to create inner table if enable_sys_table_ddl is opened // -- system view ----> table_type will be TABLE_TYPE_VIEW // -- other(virtual table or core table) ----> table_type will be TABLE_TYPE_SYS if (OB_SUCC(ret) && OB_FAIL(replace_table_schema_type(schema))) { LOG_WARN("not supported operator", K(ret)); } // set basic schema info, will be checked by check_table_exist if (OB_SUCC(ret)) { LOG_DEBUG("hualong schema is ", K(schema)); if (OB_FAIL(set_tablegroup_id(schema))) { LOG_WARN("set_tablegroup_id failed", "tablegroup name", schema.get_tablegroup_name(), K(ret)); } else if (OB_FAIL(guard.get_database_schema( tenant_id, schema.get_database_id(), database_schema))) { LOG_WARN("get_database_schema failed", K(tenant_id), "database_id", schema.get_database_id(), K(ret)); } else if (NULL == database_schema) { ret = OB_ERR_NO_DB_SELECTED; LOG_WARN("not find this database schema", K(schema.get_database_id()), K(ret)); } else if (!arg.is_inner_ && database_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not create table in recyclebin", K(ret), K(schema)); } else if (OB_FAIL(guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id)); } else if (OB_UNLIKELY(NULL == tenant_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant schema is null", K(ret), KP(tenant_schema)); } else if (OB_INVALID_ID != new_table_id && is_inner_table(new_table_id)) { // Specify table_id to create a system table, it may create a tenant-level system table, skip } else if (OB_FAIL(schema_service->fetch_new_table_id( schema.get_tenant_id(), new_table_id))) { LOG_WARN("fail to fetch new table id", K(ret)); } else if (FALSE_IT(schema.set_table_id(new_table_id))) { } else if (OB_FAIL(try_format_partition_schema(schema))) { LOG_WARN("fail to try_format_partition_schema", K(schema), KR(ret)); } else if (OB_FAIL(generate_object_id_for_partition_schema(schema))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(schema)); } else if (OB_FAIL(generate_tablet_id(schema))) { LOG_WARN("fail to fetch new table id", K(schema), K(ret)); } if (OB_SUCC(ret) && OB_INVALID_ID != schema.get_tablespace_id()) { if (OB_FAIL(guard.get_tablespace_schema(tenant_id, schema.get_tablespace_id(), tablespace_schema))) { LOG_WARN("fail to get tablespace schema", K(schema), K(ret)); } else if (OB_UNLIKELY(NULL == tablespace_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tablespace schema is null", K(ret), K(schema)); } else if (OB_FAIL(schema.set_encrypt_key(tablespace_schema->get_encrypt_key()))) { LOG_WARN("fail to set encrypt key", K(ret), K(schema)); } else { schema.set_master_key_id(tablespace_schema->get_master_key_id()); } } } if (OB_SUCC(ret)) { if (schema.has_partition()) { // include table of standalone and binding common::ObArray pool_names; if (OB_FAIL(try_check_and_set_table_schema_in_tablegroup(guard, schema))) { LOG_WARN("check table schema in tablegroup failed", K(ret)); } } else {} // has no partition } // constraints if (OB_FAIL(ret)) { } else if (OB_FAIL(schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < constraints.count(); ++i) { uint64_t new_cst_id = OB_INVALID_ID; ObConstraint &cst = const_cast(constraints.at(i)); cst.set_tenant_id(tenant_id); cst.set_table_id(schema.get_table_id()); if (OB_FAIL(schema_service->fetch_new_constraint_id(tenant_id, new_cst_id))) { LOG_WARN("failed to fetch new constraint id", K(ret)); } else if (FALSE_IT(cst.set_constraint_id(new_cst_id))) { } else if (!cst.get_constraint_name_str().empty()) { // Check whether the name of the constraint is repeated bool is_constraint_name_exist = false; if (OB_FAIL(check_constraint_name_is_exist(guard, schema, cst.get_constraint_name_str(), false, is_constraint_name_exist))) { LOG_WARN("fail to check constraint name is exist or not", K(ret), K(cst.get_constraint_name_str())); } else if (is_constraint_name_exist) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; if (!is_oracle_mode) { LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst.get_constraint_name_str().length(), cst.get_constraint_name_str().ptr()); } LOG_WARN("cst name is duplicate", K(ret), K(cst.get_constraint_name_str())); } } // The process that check whether the constraint name is repeated is end if (OB_SUCC(ret)) { if (OB_FAIL(schema.add_constraint(cst))) { LOG_WARN("add constraint failed", K(ret), K(i)); } } } // fill table schema for interval part if (OB_SUCC(ret) && schema.has_partition() && schema.is_interval_part()) { int64_t part_num = schema.get_part_option().get_part_num(); ObPartition **part_array = schema.get_part_array(); const ObRowkey *transition_point = NULL; if (PARTITION_LEVEL_TWO == schema.get_part_level() && !schema.has_sub_part_template_def()) { ret = OB_NOT_SUPPORTED; LOG_WARN("interval part of composited-partitioned table not support", K(ret), K(schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval part of composited-partitioned table without template"); } else if (1 != schema.get_partition_key_column_num()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("more than one partition key not support", K(ret), K(schema)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "more than one partition key"); } else if (OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("range part array is null", K(ret), K(schema)); } else if (0 == part_num) { ret = OB_ERR_UNEXPECTED; LOG_INFO("range part num is 0", K(ret), K(schema)); } else if (FALSE_IT(transition_point = &part_array[part_num - 1]->get_high_bound_val())) { } else if (OB_FAIL(ObPartitionUtils::check_interval_partition_table(*transition_point, schema.get_interval_range()))) { LOG_WARN("fail to check_interval_partition_table", K(ret), K(schema)); } else if (OB_FAIL(schema.set_transition_point(*transition_point))) { LOG_WARN("fail to set transition point", K(ret), K(schema)); } } return ret; } int ObDDLService::get_uk_cst_id_for_self_ref(const ObIArray &table_schemas, const ObCreateForeignKeyArg &foreign_key_arg, ObForeignKeyInfo &foreign_key_info) { int ret = OB_SUCCESS; bool is_match = false; for (int64_t i = 1; OB_SUCC(ret) && !is_match && i < table_schemas.count(); ++i) { const ObTableSchema &index_table_schema = table_schemas.at(i); if (index_table_schema.is_unique_index()) { const ObColumnSchemaV2 *index_col = NULL; const ObIndexInfo &index_info = index_table_schema.get_index_info(); ObSEArray uk_columns; for (int64_t j = 0; OB_SUCC(ret) && j < index_info.get_size(); ++j) { if (OB_ISNULL(index_col = index_table_schema.get_column_schema(index_info.get_column(j)->column_id_))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get index column schema failed", K(ret)); } else if (index_col->is_hidden() || index_col->is_shadow_column()) { // do nothing } else if (OB_FAIL(uk_columns.push_back(index_col->get_column_name()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("push back index column failed", K(ret)); } else {} // do nothing } if (OB_SUCC(ret)) { const ObIArray &parent_columns = foreign_key_arg.parent_columns_; if (OB_FAIL(sql::ObResolverUtils::check_match_columns(parent_columns, uk_columns, is_match))) { LOG_WARN("Failed to check_match_columns", K(ret)); } else if (is_match) { foreign_key_info.ref_cst_type_ = foreign_key_arg.ref_cst_type_; foreign_key_info.ref_cst_id_ = index_table_schema.get_table_id(); } } } } return ret; } // In oracle mode, check constraints and foreign key constraints are in the same namespace. // So we need to check if the new constraint name dup with all kinds of constraints in oracle mode. // In mysql mode, check constraints and foreign key constraints are in the separate namespaces. // So we need to check if the new check constraint name dup with check constraints and fk constraint name dup with fk constraints in mysql mode. int ObDDLService::check_constraint_name_is_exist(share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObTableSchema &table_schema, const common::ObString &constraint_name, const bool is_foreign_key, // this param is only effective in mysql mode bool &is_constraint_name_exist) { int ret = OB_SUCCESS; uint64_t constraint_id = OB_INVALID_ID; bool is_oracle_mode = false; is_constraint_name_exist = false; if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("check if oracle compat mode failed", K(ret)); } else { // check if fk name is exist if (is_oracle_mode || is_foreign_key) { if (OB_FAIL(schema_guard.get_foreign_key_id(table_schema.get_tenant_id(), table_schema.get_database_id(), constraint_name, constraint_id))) { LOG_WARN("get foreign key id failed", K(ret), K(table_schema.get_tenant_id()), K(table_schema.get_database_id()), K(constraint_name)); } else if (OB_INVALID_ID != constraint_id) { is_constraint_name_exist = true; } } // check if cst name is exist if (OB_SUCC(ret) && !is_constraint_name_exist && (is_oracle_mode || !is_foreign_key)) { if (table_schema.is_mysql_tmp_table()) { // tmp table in mysql mode, do nothing } else if (OB_FAIL(schema_guard.get_constraint_id(table_schema.get_tenant_id(), table_schema.get_database_id(), constraint_name, constraint_id))) { LOG_WARN("get constraint id failed", K(ret), K(table_schema.get_tenant_id()), K(table_schema.get_database_id()), K(constraint_name)); } else if (OB_INVALID_ID != constraint_id) { is_constraint_name_exist = true; } } } return ret; } int ObDDLService::deal_with_cst_for_alter_table( share::schema::ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, const ObTableSchema *orig_table_schema, obrpc::ObAlterTableArg &alter_table_arg, ObMockFKParentTableSchema &mock_fk_parent_table_schema) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const ObTableSchema *table_schema = NULL; bool is_oracle_mode = false; if (OB_ISNULL(orig_table_schema)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("table schema is null", KR(ret), K(alter_table_arg)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, alter_table_schema.get_origin_database_name(), alter_table_schema.get_origin_table_name(), false, table_schema))) { LOG_WARN("get table schema failed", K(ret), K(alter_table_schema.get_origin_database_name()), K(alter_table_schema.get_origin_table_name())); } else if (OB_FAIL(table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } // check check constraint info if (OB_FAIL(ret)) { } else if (alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::ADD_CONSTRAINT) { ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin(); for (; OB_SUCC(ret) && iter != alter_table_schema.constraint_end(); iter++) { if (OB_ISNULL(iter) || OB_ISNULL(*iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("constraint is null", K(ret)); } else if ((*iter)->get_constraint_name_str().empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("check constraint name is null", K(ret)); } else { bool is_check_constraint_name_exist = true; if (OB_FAIL(check_constraint_name_is_exist( schema_guard, *table_schema, (*iter)->get_constraint_name_str(), false, is_check_constraint_name_exist))) { LOG_WARN("fail to check check constraint name is exist or not", K(ret), K((*iter)->get_constraint_name_str())); } else if (is_check_constraint_name_exist) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; LOG_WARN("check constraint name is duplicate", K(ret), K((*iter)->get_constraint_name_str())); if (!is_oracle_mode) { LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, (*iter)->get_constraint_name_str().length(), (*iter)->get_constraint_name_str().ptr()); } } } } } // check foreign key info for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.foreign_key_arg_list_.count(); i++) { const ObCreateForeignKeyArg &foreign_key_arg = alter_table_arg.foreign_key_arg_list_.at(i); ObForeignKeyInfo foreign_key_info; // Check for duplicate foreign key constraint names if (!foreign_key_arg.foreign_key_name_.empty()) { bool is_foreign_key_name_exist = true; if (OB_FAIL(check_constraint_name_is_exist(schema_guard, *table_schema, foreign_key_arg.foreign_key_name_, true, is_foreign_key_name_exist))) { LOG_WARN("fail to check foreign key name is exist or not", K(ret), K(foreign_key_arg.foreign_key_name_)); } else if (is_foreign_key_name_exist) { if (foreign_key_arg.is_modify_fk_state_) { // Check whether the constraint is a foreign key constraint uint64_t fk_constraint_id = OB_INVALID_ID; if (OB_FAIL(schema_guard.get_foreign_key_id(table_schema->get_tenant_id(), table_schema->get_database_id(), foreign_key_arg.foreign_key_name_, fk_constraint_id))) { LOG_WARN("get foreign key id failed", K(ret), K(table_schema->get_tenant_id()), K(table_schema->get_database_id()), K(foreign_key_arg.foreign_key_name_)); } else if (OB_INVALID_ID != fk_constraint_id) { // There is a corresponding foreign key, do nothing } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("no such fk constraint", K(ret), K(foreign_key_arg.foreign_key_name_)); } } else if (is_oracle_mode) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; LOG_WARN("fk name is duplicate", K(ret), K(foreign_key_arg.foreign_key_name_)); } else { // mysql mode ret = OB_ERR_DUP_KEY; LOG_USER_ERROR(OB_ERR_DUP_KEY, table_schema->get_table_name_str().length(), table_schema->get_table_name_str().ptr()); } } } // Check whether the foreign key constraint name is repeated end if (OB_SUCC(ret) && foreign_key_arg.is_modify_fk_state_) { bool is_found = false; const ObIArray &foreign_key_infos = orig_table_schema->get_foreign_key_infos(); for (int64_t i = 0; !is_found && i < foreign_key_infos.count(); ++i) { if (0 == foreign_key_arg.foreign_key_name_.case_compare(foreign_key_infos.at(i).foreign_key_name_)) { is_found = true; foreign_key_info = foreign_key_infos.at(i); foreign_key_info.is_modify_fk_state_ = true; if (foreign_key_arg.is_modify_enable_flag_) { foreign_key_info.is_modify_enable_flag_ = true; foreign_key_info.enable_flag_ = foreign_key_arg.enable_flag_; } if (foreign_key_arg.is_modify_validate_flag_) { foreign_key_info.is_modify_validate_flag_ = true; foreign_key_info.validate_flag_ = foreign_key_arg.validate_flag_; } if (foreign_key_arg.is_modify_rely_flag_) { foreign_key_info.is_modify_rely_flag_ = true; foreign_key_info.rely_flag_ = foreign_key_arg.rely_flag_; } } } if (!is_found) { ret = OB_ERR_UNEXPECTED; LOG_WARN("no such fk constraint", K(ret), K(foreign_key_arg.foreign_key_name_)); } if (OB_SUCC(ret)) { if (foreign_key_info.child_table_id_ != foreign_key_info.parent_table_id_) { // If the reference table is itself, there is no need to update sync_versin_for_cascade_table if (OB_FAIL(alter_table_schema.add_depend_table_id(foreign_key_info.parent_table_id_))) { LOG_WARN("failed to add depend table id", K(ret), K(foreign_key_info)); } } } if (OB_SUCC(ret)) { if (OB_FAIL(alter_table_schema.add_foreign_key_info(foreign_key_info))) { LOG_WARN("failed to push foreign key info", K(ret), K(foreign_key_info)); } } } else { const ObTableSchema *parent_schema = NULL; if (OB_SUCC(ret)) { // get parent table schema // Determine whether it is self-referential if ture, partition table schema is child table schema if (0 == foreign_key_arg.parent_table_.case_compare(alter_table_schema.get_origin_table_name()) && 0 == foreign_key_arg.parent_database_.case_compare(alter_table_schema.get_origin_database_name())) { parent_schema = table_schema; } else if (OB_FAIL(schema_guard.get_table_schema(table_schema->get_tenant_id(), foreign_key_arg.parent_database_, foreign_key_arg.parent_table_, false, parent_schema))) { LOG_WARN("failed to get parent table schema", K(ret), K(foreign_key_arg)); } } if (OB_SUCC(ret)) { if (foreign_key_arg.is_parent_table_mock_) { if (OB_NOT_NULL(parent_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("parent_schema is exist", K(ret), KPC(parent_schema)); } else if (OB_FAIL(gen_mock_fk_parent_table_for_create_fk(schema_guard, table_schema->get_tenant_id(), foreign_key_arg, NULL, foreign_key_info, mock_fk_parent_table_schema))) { LOG_WARN("failed to generate_mock_fk_parent_table_schema", K(ret), K(table_schema->get_tenant_id()), K(foreign_key_arg)); } } else if (OB_ISNULL(parent_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("parent table is not exist", K(ret), K(foreign_key_arg)); } else if (parent_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("parent table is in recyclebin", K(ret), K(foreign_key_arg)); } else if (parent_schema->get_table_id() != table_schema->get_table_id()) { // If the reference table is itself, there is no need to update sync_versin_for_cascade_table if (OB_FAIL(alter_table_schema.add_depend_table_id(parent_schema->get_table_id()))) { LOG_WARN("failed to add depend table id", K(ret), K(foreign_key_arg)); } } } // get child column schema. if (OB_SUCC(ret)) { foreign_key_info.child_table_id_ = table_schema->get_table_id(); foreign_key_info.parent_table_id_ = foreign_key_arg.is_parent_table_mock_ ? mock_fk_parent_table_schema.get_mock_fk_parent_table_id() : parent_schema->get_table_id(); for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.child_columns_.count(); j++) { const ObString &column_name = foreign_key_arg.child_columns_.at(j); const ObColumnSchemaV2 *column_schema = table_schema->get_column_schema(column_name); if (OB_ISNULL(column_schema)) { ret = OB_ERR_COLUMN_NOT_FOUND; LOG_WARN("child column is not exist", K(ret), K(column_name)); } else if (OB_FAIL(foreign_key_info.child_column_ids_.push_back(column_schema->get_column_id()))) { LOG_WARN("failed to push child column id", K(ret), K(column_name)); } } } // get parent column schema. if (OB_SUCC(ret) && !foreign_key_arg.is_parent_table_mock_) { for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.parent_columns_.count(); j++) { const ObString &column_name = foreign_key_arg.parent_columns_.at(j); const ObColumnSchemaV2 *column_schema = parent_schema->get_column_schema(column_name); if (OB_ISNULL(column_schema)) { ret = OB_ERR_COLUMN_NOT_FOUND; LOG_WARN("parent column is not exist", K(ret), K(column_name)); } else if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(column_schema->get_column_id()))) { LOG_WARN("failed to push parent column id", K(ret), K(column_name)); } } } // get reference option and foreign key name. if (OB_SUCC(ret)) { foreign_key_info.update_action_ = foreign_key_arg.update_action_; foreign_key_info.delete_action_ = foreign_key_arg.delete_action_; foreign_key_info.foreign_key_name_ = foreign_key_arg.foreign_key_name_; foreign_key_info.enable_flag_ = foreign_key_arg.enable_flag_; foreign_key_info.validate_flag_ = foreign_key_arg.validate_flag_; foreign_key_info.rely_flag_ = foreign_key_arg.rely_flag_; foreign_key_info.ref_cst_type_ = foreign_key_arg.ref_cst_type_; foreign_key_info.ref_cst_id_ = foreign_key_arg.ref_cst_id_; foreign_key_info.is_parent_table_mock_ = foreign_key_arg.is_parent_table_mock_; } // add foreign key info. if (OB_SUCC(ret)) { ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_FAIL(schema_service->fetch_new_constraint_id(table_schema->get_tenant_id(), foreign_key_info.foreign_key_id_))) { LOG_WARN("failed to fetch new foreign key id", K(ret), K(foreign_key_arg)); } else if (OB_FAIL(alter_table_schema.add_foreign_key_info(foreign_key_info))) { LOG_WARN("failed to push foreign key info", K(ret), K(foreign_key_info)); } } } } // for return ret; } int ObDDLService::check_cst_name_dup_for_rename_table_mysql( share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObTableSchema *from_table_schema, const uint64_t to_database_id) { int ret = OB_SUCCESS; bool is_oracle_mode = false; if (OB_FAIL(from_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to check if oracle compat mode", K(ret)); } else if (is_oracle_mode) { // do nothing, not support rename table to another db in oracle mode } else if (from_table_schema->get_database_id() != to_database_id) { // check if constraint/foreign key name is exist when rename table to another database in mysql mode ObTableSchema tmp_schema; bool is_constraint_name_exist = true; tmp_schema.set_table_id(from_table_schema->get_table_id()); tmp_schema.set_tenant_id(from_table_schema->get_tenant_id()); tmp_schema.set_table_type(from_table_schema->get_table_type()); tmp_schema.set_database_id(to_database_id); // for check constraint ObTableSchema::const_constraint_iterator iter = from_table_schema->constraint_begin(); for (;OB_SUCC(ret) && iter != from_table_schema->constraint_end(); ++iter) { if (OB_FAIL(check_constraint_name_is_exist(schema_guard, tmp_schema, (*iter)->get_constraint_name_str(), false, is_constraint_name_exist))) { LOG_WARN("fail to check check constraint name is exist or not", K(ret), K((*iter)->get_constraint_name_str())); } else if (is_constraint_name_exist) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, (*iter)->get_constraint_name_str().length(), (*iter)->get_constraint_name_str().ptr()); LOG_WARN("check constraint name is duplicate", K(ret), K((*iter)->get_constraint_name_str())); } } // for foreign key constraint if (OB_SUCC(ret)) { const common::ObIArray& foreign_keys = from_table_schema->get_foreign_key_infos(); for (int i = 0; OB_SUCC(ret) && i < foreign_keys.count(); ++i) { if (OB_FAIL(check_constraint_name_is_exist( schema_guard, tmp_schema, foreign_keys.at(i).foreign_key_name_, true, is_constraint_name_exist))) { LOG_WARN("fail to check foreign key name is exist or not", K(ret), K(foreign_keys.at(i).foreign_key_name_)); } else if (is_constraint_name_exist) { ret = OB_ERR_DUP_KEY; LOG_USER_ERROR(OB_ERR_DUP_KEY, foreign_keys.at(i).foreign_key_name_.length(), foreign_keys.at(i).foreign_key_name_.ptr()); LOG_WARN("foreign key name is duplicate", K(ret), K(foreign_keys.at(i).foreign_key_name_)); } } } } return ret; } int ObDDLService::check_database_exist( const uint64_t tenant_id, const common::ObString &database_name, uint64_t &database_id) { int ret = OB_SUCCESS; bool exist = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(schema_service_->check_database_exist( tenant_id, database_name, database_id, exist))) { LOG_WARN("check database exist failed", "database", database_name, K(ret)); } else { if (!exist) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr()); } } return ret; } int ObDDLService::check_table_exist(ObTableSchema &table_schema) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { bool is_table_name_exist = false; bool is_table_id_exist = false; const int64_t table_schema_version = OB_INVALID_VERSION; // get the latest local schema_guard if (OB_FAIL(schema_service_->check_table_exist( table_schema.get_tenant_id(), table_schema.get_database_id(), table_schema.get_table_name_str(), table_schema.is_index_table(), table_schema_version, is_table_name_exist))) { LOG_WARN("failed to check is table exist", K(table_schema.get_tenant_id()), K(table_schema.get_database_id()), K(table_schema.is_index_table()), K(table_schema.get_table_name()), K(ret)); } else if (OB_INVALID_ID != table_schema.get_table_id() && OB_FAIL(schema_service_->check_table_exist( table_schema.get_tenant_id(), table_schema.get_table_id(), table_schema_version, is_table_id_exist))) { LOG_WARN("failed to check is table exist", K(table_schema.get_tenant_id()), K(table_schema.get_table_id()), K(ret)); } else if (is_table_name_exist || is_table_id_exist) { ret = OB_ERR_TABLE_EXIST; LOG_WARN("table is exist, cannot create it twice,", K(table_schema.get_tenant_id()), K(table_schema.get_database_id()), K(table_schema.get_table_id()), K(table_schema.get_table_name()), K(ret)); } } return ret; } int ObDDLService::check_inner_stat() const { int ret = OB_SUCCESS; if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret)); } else if (OB_ISNULL(schema_service_) || OB_ISNULL(sql_proxy_) || OB_ISNULL(rpc_proxy_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service_,sql_proxy_ or rpc_proxy_ is null", K(ret)); } else if (OB_ISNULL(server_mgr_) || OB_ISNULL(lst_operator_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("server_mgr_ or pt_operator_ or lst_operator_ is null", KR(ret)); } else if (OB_ISNULL(unit_mgr_) || OB_ISNULL(zone_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr_ or zone_mgr_ is null", K(ret)); } return ret; } int ObDDLService::get_sample_table_schema( common::ObIArray &table_schemas, const ObSimpleTableSchemaV2 *&sample_table_schema) { int ret = OB_SUCCESS; // find the first table schema, except all_core-table. for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); ++i) { if (OB_UNLIKELY(NULL == table_schemas.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema null", K(ret)); } else if (table_schemas.at(i)->has_partition() && OB_ALL_CORE_TABLE_TID != table_schemas.at(i)->get_table_id()) { sample_table_schema = table_schemas.at(i); } else {} // go on find } return ret; } int ObDDLService::set_tablegroup_id(ObTableSchema &table_schema) { int ret = OB_SUCCESS; uint64_t tablegroup_id = OB_INVALID_ID; uint64_t tenant_id = table_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (table_schema.get_tablegroup_name().empty()) { table_schema.set_tablegroup_id(OB_INVALID_ID); } else if (OB_FAIL(schema_guard.get_tablegroup_id(table_schema.get_tenant_id(), table_schema.get_tablegroup_name(), tablegroup_id))) { LOG_WARN("get_tablegroup_id failed", "tenant_id", table_schema.get_tenant_id(), "tablegroup_name", table_schema.get_tablegroup_name(), K(ret)); } else if (OB_INVALID_ID == tablegroup_id) { ret = OB_TABLEGROUP_NOT_EXIST; LOG_WARN("group name not exist ", K(ret)); } else { table_schema.set_tablegroup_id(tablegroup_id); } // if table's tablegroup does not exist, use database's default tablegroup if (OB_SUCC(ret) && OB_INVALID_ID == table_schema.get_tablegroup_id()) { const ObDatabaseSchema *db_schema = NULL; if (OB_FAIL(schema_guard.get_database_schema(tenant_id, table_schema.get_database_id(), db_schema))) { LOG_WARN("fail to get database schema", K(ret), K(tenant_id), "db_id", table_schema.get_database_id()); } else if (OB_UNLIKELY(NULL == db_schema)) { ret = OB_ERR_BAD_DATABASE; LOG_WARN("fail to get database schema", K(ret), K(tenant_id), "db_id", table_schema.get_database_id()); } else { table_schema.set_tablegroup_id(db_schema->get_default_tablegroup_id()); } } // if database's default_tablegroup_id does not exist, use tenant's default tablegroup if (OB_SUCC(ret) && OB_INVALID_ID == table_schema.get_tablegroup_id()) { const ObTenantSchema *tenant_schema = NULL; if (OB_FAIL(schema_guard.get_tenant_info(table_schema.get_tenant_id(), tenant_schema))) { LOG_WARN("fail to get tenant schema", K(ret), "tenant_id", table_schema.get_tenant_id()); } else if (OB_UNLIKELY(NULL == tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("fail to get tenant schema", K(ret), "tenant_id", table_schema.get_tenant_id()); } else { table_schema.set_tablegroup_id(tenant_schema->get_default_tablegroup_id()); } } // TODO: (2019.6.24 wendu) Cannot add replicated table to tablegroup if (OB_SUCC(ret)) { if (ObDuplicateScope::DUPLICATE_SCOPE_NONE != table_schema.get_duplicate_scope() && OB_INVALID_ID != table_schema.get_tablegroup_id()) { ret = OB_NOT_SUPPORTED; LOG_WARN("replicated table in tablegroup is not supported", K(ret), "table_id", table_schema.get_table_id(), "tablegroup_id", table_schema.get_tablegroup_id()); LOG_USER_ERROR(OB_NOT_SUPPORTED, "replicated table in tablegroup"); } } return ret; } template int ObDDLService::set_default_tablegroup_id(SCHEMA &schema) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; uint64_t tablegroup_id = OB_INVALID_ID; const uint64_t tenant_id = schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else { const ObString &tablegroup_name = schema.get_default_tablegroup_name(); if (tablegroup_name.empty()) { schema.set_default_tablegroup_id(OB_INVALID_ID); } else if (OB_FAIL(schema_guard.get_tablegroup_id(tenant_id, tablegroup_name, tablegroup_id))) { LOG_WARN("get_tablegroup_id failed", K(schema.get_tenant_id()), K(tablegroup_name), K(ret)); } else if (OB_INVALID_ID == tablegroup_id) { ret = OB_TABLEGROUP_NOT_EXIST; LOG_WARN("tablegroup not exist", K(ret), K(tablegroup_name)); } else { schema.set_default_tablegroup_id(tablegroup_id); } } return ret; } int ObDDLService::print_view_expanded_definition( const ObTableSchema &table_schema, ObString &ddl_stmt_str, common::ObIAllocator &allocator, ObSchemaGetterGuard &schema_guard, bool if_not_exist) { int ret = OB_SUCCESS; char *buf = NULL; int64_t buf_len = OB_MAX_VARCHAR_LENGTH; int64_t pos = 0; bool is_oracle_mode; const ObDatabaseSchema *database_schema = NULL; const uint64_t tenant_id = table_schema.get_tenant_id(); const int64_t database_id = table_schema.get_database_id(); if (OB_ISNULL(buf = static_cast(allocator.alloc(buf_len)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail to allocate memory", K(ret), K(OB_MAX_VARCHAR_LENGTH)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, database_id, database_schema))) { LOG_WARN("failed to get database schema", K(ret), K(tenant_id), K(database_id)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database not exist", K(ret), K(database_id)); } else if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, is_oracle_mode ? "CREATE%s %sVIEW \"%s\".\"%s\" AS %.*s" : "CREATE%s %sVIEW `%s`.`%s` AS %.*s", if_not_exist ? " OR REPLACE" : "", table_schema.is_materialized_view() ? "MATERIALIZED " : "", database_schema->get_database_name(), table_schema.get_table_name(), table_schema.get_view_schema().get_view_definition_str().length(), table_schema.get_view_schema().get_view_definition_str().ptr()))) { LOG_WARN("fail to print view definition", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, ";"))) { LOG_WARN("fail to print end", K(ret)); } else { ddl_stmt_str.assign_ptr(buf, static_cast(pos)); } return ret; } int ObDDLService::create_tables_in_trans(const bool if_not_exist, const ObString &ddl_stmt_str, const ObErrorInfo &error_info, ObIArray &table_schemas, const int64_t frozen_version, const obrpc::ObSequenceDDLArg &sequence_ddl_arg, const uint64_t last_replay_log_id, const ObIArray *dep_infos, ObIArray &mock_fk_parent_table_schema_array) { UNUSED(frozen_version); int ret = OB_SUCCESS; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); RS_TRACE(create_tables_in_trans_begin); bool is_standby = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("faile to get is standby cluster", K(ret)); } else if (table_schemas.count() < 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schemas have no element", K(ret)); } else if (OB_INVALID_ID == last_replay_log_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(ret), K(last_replay_log_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObSchemaGetterGuard schema_guard; uint64_t tenant_id = table_schemas.at(0).get_tenant_id(); share::schema::ObTableSchema &first_table = table_schemas.at(0); int64_t refreshed_schema_version = 0; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans", KR(ret), K(tenant_id), K(tenant_id), K(refreshed_schema_version)); } else { } if (OB_SUCC(ret)) { ObString tmp_ddl_stmt_str = ddl_stmt_str; if (table_schemas.count() > 0) { const ObTableSchema &table_schema = table_schemas.at(0); if (table_schema.is_view_table()) { if (OB_FAIL(print_view_expanded_definition( table_schema, tmp_ddl_stmt_str, allocator, schema_guard, if_not_exist))) { LOG_WARN("fail to print_view_expanded_definition", K(ret), K(table_schema.get_table_id())); } } //create or replace view xxx if (OB_FAIL(ret)) { } else if (table_schema.is_view_table() && if_not_exist) { const ObString &view_name = table_schema.get_table_name(); const ObTableSchema *old_view_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(table_schema.get_tenant_id(), table_schema.get_database_id(), view_name, false, //is index old_view_schema))) { LOG_WARN("failed to get table schema", K(view_name), K(ret)); } else if (OB_ISNULL(old_view_schema)) { ret = OB_SUCCESS; } else if (OB_FAIL(drop_trigger_in_drop_table(trans, ddl_operator, schema_guard, *old_view_schema, false))) { // 兼容oracle,create or replace view时drop trigger,且不进回收站 LOG_WARN("failed to drop trigger", K(old_view_schema->get_table_id()), K(ret)); } else if (OB_FAIL(ddl_operator.drop_table(*old_view_schema, trans))) { LOG_WARN("failed to drop old view schema", K(ret)); } } } RS_TRACE(operator_create_table_begin); for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) { ObTableSchema &table_schema = table_schemas.at(i); if (OB_FAIL(ddl_operator.create_sequence_in_create_table(table_schema, trans, schema_guard, &sequence_ddl_arg))) { LOG_WARN("create sequence in create table fail", K(ret)); } else if (OB_FAIL(ddl_operator.create_table(table_schema, trans, 0 == i ? &tmp_ddl_stmt_str : NULL, i == table_schemas.count() - 1))) { LOG_WARN("failed to create table schema, ", K(ret)); } else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, table_schema))) { LOG_WARN("failed to insert_temp_table_info!", K(ret)); } else if (GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_322 && table_schema.is_view_table() && dep_infos != nullptr && 0 == i) { for (int64_t i = 0 ; OB_SUCC(ret) && i < dep_infos->count(); ++i) { ObDependencyInfo dep; if (OB_FAIL(dep.assign(dep_infos->at(i)))) { LOG_WARN("failed to assign dependency info", K(ret)); } else { dep.set_tenant_id(tenant_id); dep.set_dep_obj_id(table_schema.get_table_id()); dep.set_dep_obj_owner_id(table_schema.get_table_id()); dep.set_schema_version(table_schema.get_schema_version()); OZ (dep.insert_schema_object_dependency(trans)); } } } } // add error info for create force view if (OB_SUCC(ret) && 1 == table_schemas.count() && first_table.is_user_view()) { bool is_oracle_mode = false; if (OB_LIKELY(ERROR_STATUS_HAS_ERROR != error_info.get_error_status())) { /* do nothing */ } else if (OB_FAIL(first_table.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check if tenant mode is oracle mode", K(ret)); } else if (OB_UNLIKELY(!is_oracle_mode)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected compat mode add create view error info", K(ret), K(is_oracle_mode)); } else { ObErrorInfo tmp_error_info; if (OB_FAIL(tmp_error_info.assign(error_info))) { LOG_WARN("failed to assign error info", K(ret)); } else { tmp_error_info.set_obj_id(first_table.get_table_id()); tmp_error_info.set_obj_type(static_cast(ObObjectType::VIEW)); tmp_error_info.set_database_id(first_table.get_database_id()); tmp_error_info.set_tenant_id(first_table.get_tenant_id()); tmp_error_info.set_schema_version(first_table.get_schema_version()); if (OB_FAIL(tmp_error_info.handle_error_info(trans, NULL))) { LOG_WARN("insert create error info failed.", K(ret)); } } } } if (FAILEDx(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) { LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(tenant_id)); } // Create a new indexed system table needs to write a schema if (OB_SUCC(ret) && ObSysTableChecker::is_sys_table_has_index(first_table.get_table_id())) { ObArray schemas; if (OB_FAIL(ObSysTableChecker::append_sys_table_index_schemas( tenant_id, first_table.get_table_id(), schemas))) { LOG_WARN("fail to add sys table index", K(ret), K(tenant_id), "table_id", first_table.get_table_id()); } else if (OB_FAIL(ddl_operator.create_table(schemas.at(0), trans, NULL, true /*need_sync_schema_version*/))) { LOG_WARN("failed to create table schema", K(ret), "schema", schemas.at(0)); } } if (OB_SUCC(ret) && is_system_table(first_table.get_table_id())) { HEAP_VARS_2((ObTableSchema, lob_meta_schema), (ObTableSchema, lob_piece_schema)) { if (OB_FAIL(add_sys_table_lob_aux(tenant_id, first_table.get_table_id(), lob_meta_schema, lob_piece_schema))) { LOG_WARN("fail to get sys table lob aux schema", KR(ret), K(first_table.get_table_id())); } else { if (OB_FAIL(ddl_operator.create_table(lob_meta_schema, trans, NULL, true /*need_sync_schema_version*/))) { LOG_WARN("failed to create lob aux meta table.", K(ret), "schema", lob_meta_schema); } else if (OB_FAIL(ddl_operator.create_table(lob_piece_schema, trans, NULL, true /*need_sync_schema_version*/))) { LOG_WARN("failed to create lob aux data table.", K(ret), "schema", lob_piece_schema); } } } } int64_t frozen_scn = 0; if (OB_FAIL(ret)) { } else if (OB_ISNULL(GCTX.root_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("root service is null", KR(ret)); } else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) { LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id)); } else { ObTableCreator table_creator( tenant_id, frozen_scn, *lst_operator_, trans); ObNewTableTabletAllocator new_table_tablet_allocator( tenant_id, schema_guard, sql_proxy_); common::ObArray ls_id_array; const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version(); if (OB_INVALID_VERSION == last_schema_version) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid last schema version", K(ret)); } else if (OB_FAIL(table_creator.init())) { LOG_WARN("fail to init table creator", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.init())) { LOG_WARN("fail to init new table tablet allocator", KR(ret)); } ObArray schemas; for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) { const share::schema::ObTableSchema &this_table = table_schemas.at(i); const int64_t table_id = this_table.get_table_id(); if (!this_table.has_tablet()) { } else if (!this_table.is_global_index_table()) { if (OB_FAIL(schemas.push_back(&this_table))) { LOG_WARN("failed to push_back", KR(ret), K(this_table)); } } else { if (OB_FAIL(new_table_tablet_allocator.prepare(this_table))) { LOG_WARN("fail to prepare ls for index schema tablets"); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array( ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_table_arg( this_table, ls_id_array))) { LOG_WARN("create table partitions failed", KR(ret), K(this_table), K(last_schema_version)); } } if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.insert_ori_schema_version( trans, tenant_id, table_id, last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(tenant_id), K(table_id), K(last_schema_version)); } } } if (OB_FAIL(ret)) { } else if (schemas.count() <= 0) { // virtual tablet and view skip } else if (OB_FAIL(new_table_tablet_allocator.prepare(first_table))) { LOG_WARN("fail to prepare ls for index schema tablets"); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array( ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg( schemas, ls_id_array))) { LOG_WARN("create table partitions failed", KR(ret), K(first_table), K(last_schema_version)); } else if (OB_FAIL(table_creator.execute())) { LOG_WARN("execute create partition failed", KR(ret)); } // finishing is always invoked for new table tablet allocator int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) { LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret)); } } RS_TRACE(operator_create_table_end); } DEBUG_SYNC(BEFORE_CREATE_TABLE_TRANS_COMMIT); if (OB_SUCC(ret) && THIS_WORKER.is_timeout_ts_valid() && THIS_WORKER.is_timeout()) { ret = OB_TIMEOUT; LOG_WARN("already timeout", KR(ret)); } if (OB_SUCC(ret)) { ret = E(EventTable::EN_CREATE_TABLE_TRANS_END_FAIL) OB_SUCCESS; } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } RS_TRACE(create_tables_in_trans_end); return ret; } // Create table information is written to the internal table within a transaction. // If sql_trans is NULL, it need to create a transaction inside the function. int ObDDLService::create_table_in_trans( ObTableSchema &table_schema, const int64_t frozen_version, const ObString *ddl_stmt_str, ObMySQLTransaction *sql_trans, share::schema::ObSchemaGetterGuard &schema_guard) { UNUSED(frozen_version); int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { const uint64_t tenant_id = table_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; ObDDLSQLTransaction tmp_trans(schema_service_); ObMySQLTransaction &trans = OB_ISNULL(sql_trans) ? tmp_trans : *sql_trans; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_ISNULL(sql_trans) && OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.create_table(table_schema, trans, ddl_stmt_str))) { LOG_WARN("failed to create table schema, ", KR(ret)); } else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, table_schema))) { LOG_WARN("failed to insert temp table info!", KR(ret)); } else { LOG_INFO("succeed to insert table schema in schema tables", K(table_schema.get_tenant_id()), K(table_schema.get_database_id()), K(table_schema.get_table_id()), K(table_schema.get_table_name())); } const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version(); if (OB_SUCC(ret)) { if (OB_INVALID_VERSION == last_schema_version) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid schema version" , K(last_schema_version)); } else if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans, tenant_id, table_schema.get_table_id(), last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", KR(ret), K(tenant_id), K(last_schema_version)); } } if (OB_SUCC(ret) && table_schema.has_tablet() && OB_FAIL(create_index_tablet(table_schema, trans, schema_guard))) { LOG_WARN("fail to create_index_tablet", KR(ret), K(table_schema)); } if (OB_ISNULL(sql_trans) && trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } return ret; } int ObDDLService::check_tablegroup_in_single_database( share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObTableSchema &table_schema) { int ret = OB_SUCCESS; ObArray table_schemas; const ObSimpleTableSchemaV2 *sample_table = NULL; if (OB_INVALID_ID == table_schema.get_tablegroup_id()) { // skip } else if (OB_FAIL(schema_guard.get_table_schemas_in_tablegroup( table_schema.get_tenant_id(), table_schema.get_tablegroup_id(), table_schemas))) { LOG_WARN("fail to get table schemas in tablegroup", K(ret), "tenant_id", table_schema.get_tenant_id(), "tablegroup_id", table_schema.get_tablegroup_id()); } else if (OB_FAIL(get_sample_table_schema(table_schemas, sample_table))) { LOG_WARN("fail to get sample table schema", K(ret)); } else if (NULL == sample_table) { // empty tablegroup, good } else { if (sample_table->get_database_id() != table_schema.get_database_id()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("tables in one tablegroup across more than one schema", K(ret), "sample_table_id", sample_table->get_table_id(), "sample_database_id", sample_table->get_database_id(), "tablegroup_id", table_schema.get_tablegroup_id(), "table_id", table_schema.get_table_id(), "table_database_id", table_schema.get_database_id()); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "tables in one tablegroup across more than one schema"); } else {} // good } return ret; } int ObDDLService::set_new_table_options( const obrpc::ObAlterTableArg &alter_table_arg, const share::schema::AlterTableSchema &alter_table_schema, const share::schema::ObTenantSchema &tenant_schema, share::schema::ObTableSchema &new_table_schema, const share::schema::ObTableSchema &orig_table_schema, share::schema::ObSchemaGetterGuard &schema_guard, bool &need_update_index_table, AlterLocalityOp &alter_locality_op) { int ret = OB_SUCCESS; if ((alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::LOCALITY) || alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::PRIMARY_ZONE)) && OB_INVALID_ID != new_table_schema.get_tablegroup_id()) { // 2.0 introduces constraints, it is not allowed to directly modify the primary_zone/locality // of the table in the tablegroup ret = OB_OP_NOT_ALLOW; LOG_WARN("modify primary_zone/locality of table in a tablegroup is not allowed", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify primary_zone/locality of table in a tablegroup"); } else if (alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::LOCALITY) && alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::TABLEGROUP_NAME)) { // 2.0 introduces constraints, It is not allowed to change tablegroup and locality at the same time ret = OB_OP_NOT_ALLOW; LOG_WARN("modify locality and tablegroup at the same time is not allowed", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify locality and tablegroup at the same time"); } else if (alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::STORAGE_FORMAT_VERSION) && alter_table_schema.get_storage_format_version() < orig_table_schema.get_storage_format_version()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("cannot modify storage format version to lower version", K(ret), K(alter_table_schema.get_storage_format_version()), K(orig_table_schema.get_storage_format_version())); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "target storage format version cannot be smaller than current version"); } else if (alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::PROGRESSIVE_MERGE_ROUND) && alter_table_schema.get_progressive_merge_round() <= orig_table_schema.get_progressive_merge_round()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("cannot modify progressive merge round to lower version", K(ret), K(alter_table_schema.get_progressive_merge_round()), K(orig_table_schema.get_progressive_merge_round())); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "target progressive merge round cannot be smaller or equal to current version"); } else if (OB_FAIL(set_raw_table_options( alter_table_schema, new_table_schema, schema_guard, need_update_index_table))) { LOG_WARN("fail to set raw table options", K(ret), K(new_table_schema), K(orig_table_schema)); } else if (ObDuplicateScope::DUPLICATE_SCOPE_NONE != new_table_schema.get_duplicate_scope() && OB_INVALID_ID != new_table_schema.get_tablegroup_id()) { ret = OB_NOT_SUPPORTED; LOG_WARN("replicated table in tablegroup is not supported", K(ret), "table_id", new_table_schema.get_table_id(), "tablegroup_id", new_table_schema.get_tablegroup_id()); LOG_USER_ERROR(OB_NOT_SUPPORTED, "replicated table in tablegroup"); } else { if (OB_SUCC(ret) && alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::TABLEGROUP_NAME)) { ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_); if (OB_FAIL(check_tablegroup_in_single_database(schema_guard, new_table_schema))) { LOG_WARN("fail to check tablegroup in single database", K(ret)); } else if (OB_FAIL(helper.check_table_alter_tablegroup( schema_guard, orig_table_schema, new_table_schema))) { LOG_WARN("fail to check table schema in tablegroup", K(ret)); } else {} // good } bool alter_collation = alter_table_schema.alter_option_bitset_.has_member( obrpc::ObAlterTableArg::COLLATION_TYPE); bool alter_charset = alter_table_schema.alter_option_bitset_.has_member( obrpc::ObAlterTableArg::CHARSET_TYPE); if (OB_SUCC(ret) && (alter_collation || alter_charset)) { ObCharsetType charset_type = alter_table_schema.get_charset_type(); ObCollationType collation_type = alter_table_schema.get_collation_type(); if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "modify character or collation"); } else if (alter_collation && alter_charset) { if (!ObCharset::is_valid_collation(charset_type, collation_type)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid charset", K(ret), K(charset_type), K(collation_type)); } else { new_table_schema.set_collation_type(collation_type); new_table_schema.set_charset_type(charset_type); } } else if (alter_collation) { new_table_schema.set_collation_type(collation_type); new_table_schema.set_charset_type(ObCharset::charset_type_by_coll(collation_type)); } else if (alter_charset) { new_table_schema.set_collation_type(ObCharset::get_default_collation(charset_type)); new_table_schema.set_charset_type(charset_type); } } } LOG_DEBUG("set new table options", K(alter_table_arg), K(alter_table_schema), K(tenant_schema), K(new_table_schema), K(orig_table_schema)); return ret; } //set new table options to new table schema int ObDDLService::set_raw_table_options( const AlterTableSchema &alter_table_schema, ObTableSchema &new_table_schema, ObSchemaGetterGuard &schema_guard, bool &need_update_index_table) { int ret = OB_SUCCESS; //replace alter options need_update_index_table = false; for (int32_t i = ObAlterTableArg::AUTO_INCREMENT; OB_SUCC(ret) && i < ObAlterTableArg::MAX_OPTION; ++i) { if (alter_table_schema.alter_option_bitset_.has_member(i)) { switch (i) { case ObAlterTableArg::TABLE_DOP: { new_table_schema.set_dop(alter_table_schema.get_dop()); break; } case ObAlterTableArg::AUTO_INCREMENT: { new_table_schema.set_auto_increment(alter_table_schema.get_auto_increment()); break; } case ObAlterTableArg::BLOCK_SIZE: { need_update_index_table = true; new_table_schema.set_block_size(alter_table_schema.get_block_size()); break; } case ObAlterTableArg::CHARSET_TYPE: { new_table_schema.set_charset_type(alter_table_schema.get_charset_type()); break; } case ObAlterTableArg::COLLATION_TYPE: { new_table_schema.set_collation_type(alter_table_schema.get_collation_type()); break; } case ObAlterTableArg::STORE_FORMAT: { new_table_schema.set_row_store_type(alter_table_schema.get_row_store_type()); new_table_schema.set_store_format(alter_table_schema.get_store_format()); need_update_index_table = true; break; } case ObAlterTableArg::STORAGE_FORMAT_VERSION: { new_table_schema.set_storage_format_version(alter_table_schema.get_storage_format_version()); need_update_index_table = true; break; } case ObAlterTableArg::COMPRESS_METHOD: { ret = new_table_schema.set_compress_func_name(alter_table_schema.get_compress_func_name()); need_update_index_table = true; break; } case ObAlterTableArg::COMMENT: { ret = new_table_schema.set_comment(alter_table_schema.get_comment_str()); break; } case ObAlterTableArg::EXPIRE_INFO: { ret = new_table_schema.set_expire_info(alter_table_schema.get_expire_info()); break; } case ObAlterTableArg::PRIMARY_ZONE: { LOG_INFO("changing table's primay_zone does not take effect"); ret = OB_SUCCESS; // do nothing break; } case ObAlterTableArg::REPLICA_NUM: { // ignore alter replica num break; } case ObAlterTableArg::TABLET_SIZE: { new_table_schema.set_tablet_size(alter_table_schema.get_tablet_size()); need_update_index_table = true; break; } case ObAlterTableArg::PCTFREE: { new_table_schema.set_pctfree(alter_table_schema.get_pctfree()); need_update_index_table = true; break; } case ObAlterTableArg::PROGRESSIVE_MERGE_NUM: { need_update_index_table = true; new_table_schema.set_progressive_merge_num(alter_table_schema.get_progressive_merge_num()); break; } case ObAlterTableArg::PROGRESSIVE_MERGE_ROUND: { need_update_index_table = true; new_table_schema.set_progressive_merge_round(alter_table_schema.get_progressive_merge_round()); break; } case ObAlterTableArg::TABLE_NAME: { //check table rename const ObString &new_table_name = alter_table_schema.get_table_name_str(); const ObString &origin_table_name = alter_table_schema.get_origin_table_name(); const ObString &new_database_name = alter_table_schema.get_database_name(); const ObString &origin_database_name = alter_table_schema.get_origin_database_name(); uint64_t tenant_id = new_table_schema.get_tenant_id(); ObNameCaseMode mode = OB_NAME_CASE_INVALID; bool is_oracle_mode = false; bool has_mv = false; const ObTableSchema *orig_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, origin_database_name, origin_table_name, false, orig_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(origin_table_name), K(origin_database_name)); } else if (NULL == orig_table_schema) { ret = OB_ERR_TABLE_EXIST; LOG_WARN("table not exist", K(ret)); } else if (OB_FAIL(check_table_has_materialized_view(schema_guard, *orig_table_schema, has_mv))) { LOG_WARN("fail to check table has materialized view", K(ret), K(*orig_table_schema)); } else if (has_mv) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support rename table has materialized view", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_name_case_mode(tenant_id, mode))) { LOG_WARN("fail to get tenant name case mode", K(tenant_id), K(ret)); } else if (OB_FAIL(orig_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to get compat mode", K(ret), K(tenant_id)); } else { //TODO rename datbase_name need to update all index table @hualong ObString database_name; if (!is_oracle_mode) { // mysql mode if (!new_database_name.empty() && !ObCharset::case_mode_equal(mode, new_database_name, origin_database_name)) { database_name = new_database_name; } else { database_name = origin_database_name; } } else { // oracle mode if (!new_database_name.empty() && !ObCharset::case_sensitive_equal(new_database_name, origin_database_name)) { database_name = new_database_name; } else { database_name = origin_database_name; } } ObString table_name; uint64_t database_id = OB_INVALID_ID; if (!is_oracle_mode) { // mysql mode if (!new_table_name.empty() && !ObCharset::case_mode_equal(mode, new_table_name, origin_table_name)) { table_name = new_table_name; } else { table_name = origin_table_name; } } else { // oracle mode if (!new_table_name.empty() && !ObCharset::case_sensitive_equal(new_table_name, origin_table_name)) { table_name = new_table_name; } else { table_name = origin_table_name; } } if (OB_FAIL(schema_guard.get_database_id(tenant_id, database_name, database_id))) { LOG_WARN("fail to get database id", K(tenant_id), K(database_name), K(ret)); } else if (database_name != origin_database_name || table_name != origin_table_name) { const ObTableSchema *tmp_schema = NULL; const ObSynonymInfo *synonym_info = NULL; bool is_index = false; if (OB_FAIL(schema_guard.get_synonym_info(tenant_id, database_id, table_name, synonym_info))) { LOG_WARN("fail to check synonym exist", K(database_name), K(table_name), K(ret)); } else if (NULL != synonym_info) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(database_name), K(table_name), K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, database_name, table_name, is_index, tmp_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(database_name), K(table_name)); } else if (NULL == tmp_schema) { LOG_INFO("table not exist, can rename to new table name", K(new_table_name)); } else { ret = OB_ERR_TABLE_EXIST; LOG_USER_ERROR(OB_ERR_TABLE_EXIST, table_name.length(), table_name.ptr()); } } if (OB_SUCC(ret)) { if (OB_FAIL(new_table_schema.set_table_name(table_name))) { LOG_WARN("fail to set table name", K(table_name), K(ret)); } else { need_update_index_table = true; new_table_schema.set_database_id(database_id); } } if (OB_SUCC(ret) && OB_FAIL(check_cst_name_dup_for_rename_table_mysql( schema_guard, orig_table_schema, new_table_schema.get_database_id()))) { LOG_WARN("fail to check cst name dup for rename table mysql", K(ret)); } } break; } case ObAlterTableArg::TABLEGROUP_NAME: { uint64_t tablegroup_id = OB_INVALID_ID; const ObString &tablegroup_name = alter_table_schema.get_tablegroup_name(); if (!tablegroup_name.empty()) { //tablegroup_id not set in resolver, only record tablegroup name if (OB_FAIL(schema_guard.get_tablegroup_id(alter_table_schema.get_tenant_id(), tablegroup_name, tablegroup_id))) { LOG_WARN("failed to get tablegroup id", K(ret), K(tablegroup_name)); } else if (OB_INVALID_ID == tablegroup_id) { ret = OB_TABLEGROUP_NOT_EXIST; LOG_WARN("invalid tablegroup name", K(ret), K(tablegroup_name)); } else { new_table_schema.set_tablegroup_id(tablegroup_id); need_update_index_table = true; } } else { new_table_schema.set_tablegroup_id(OB_INVALID_ID); need_update_index_table = true; } break; } case ObAlterTableArg::SEQUENCE_COLUMN_ID: { //TODO break; } case ObAlterTableArg::USE_BLOOM_FILTER: { new_table_schema.set_is_use_bloomfilter(alter_table_schema.is_use_bloomfilter()); break; } case ObAlterTableArg::READ_ONLY: { new_table_schema.set_read_only(alter_table_schema.is_read_only()); need_update_index_table = true; break; } case ObAlterTableArg::LOCALITY: { if (is_sys_table(new_table_schema.get_table_id())) { ret = OB_NOT_SUPPORTED; LOG_WARN("change system table's locality is not allowed", K(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "change system table's locality"); } else { LOG_INFO("changing table's locality does not take effect"); } break; } case ObAlterTableArg::SESSION_ID: { bool is_index = false; const ObString &table_name = new_table_schema.get_table_name_str(); const uint64_t database_id = new_table_schema.get_database_id(); uint64_t tenant_id = new_table_schema.get_tenant_id(); const ObTableSchema *found_table_schema = NULL; if (0 == new_table_schema.get_session_id() || alter_table_schema.get_session_id() == new_table_schema.get_session_id()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table session id is reset already", K(ret), K(new_table_schema), K(alter_table_schema)); } else { // bug18197606, In the process of querying or building a table, // another session creates a table with the same name, // Modify the first two can lead to the same name table and session_id = 0; // It is necessary to check whether there is a normal table with session_id = 0 // in the last step of querying the table creation. If it is error CTAS will be rolled back. uint64_t org_session_id = schema_guard.get_session_id(); schema_guard.set_session_id(alter_table_schema.get_session_id()); if (OB_FAIL(schema_guard.get_table_schema(tenant_id, database_id, table_name, is_index, found_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), K(database_id), K(table_name), K(is_index), K(ret)); } else if (OB_NOT_NULL(found_table_schema)) { ret = OB_ERR_TABLE_EXIST; LOG_USER_ERROR(OB_ERR_TABLE_EXIST, table_name.length(), table_name.ptr()); } else { new_table_schema.set_session_id(alter_table_schema.get_session_id()); } schema_guard.set_session_id(org_session_id); } break; } case ObAlterTableArg::SESSION_ACTIVE_TIME: { new_table_schema.set_sess_active_time(alter_table_schema.get_sess_active_time()); break; } case ObAlterTableArg::DUPLICATE_SCOPE: { new_table_schema.set_duplicate_scope(alter_table_schema.get_duplicate_scope()); break; } case ObAlterTableArg::ENABLE_ROW_MOVEMENT: { new_table_schema.set_enable_row_movement(alter_table_schema.is_enable_row_movement()); break; } case ObAlterTableArg::FORCE_LOCALITY: { // do nothing break; } case ObAlterTableArg::TABLE_MODE: { new_table_schema.set_table_mode(alter_table_schema.get_table_mode()); need_update_index_table = true; break; } case ObAlterTableArg::INCREMENT_MODE : { new_table_schema.set_table_auto_increment_mode( alter_table_schema.get_table_auto_increment_mode()); break; } case ObAlterTableArg::ENABLE_EXTENDED_ROWID: { new_table_schema.set_table_rowid_mode(alter_table_schema.get_table_rowid_mode()); break; } case ObAlterTableArg::ENCRYPTION: { new_table_schema.set_encryption_str(alter_table_schema.get_encryption_str()); break; } case ObAlterTableArg::TABLESPACE_ID: { new_table_schema.set_tablespace_id(alter_table_schema.get_tablespace_id()); new_table_schema.set_encryption_str(alter_table_schema.get_encryption_str()); break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unknown option!", K(i)); } } } } return ret; } int ObDDLService::parse_and_set_create_tenant_new_locality_options( share::schema::ObSchemaGetterGuard &schema_guard, ObTenantSchema &schema, const common::ObIArray &pools, const common::ObIArray &zone_list, const common::ObIArray &zone_region_list) { int ret = OB_SUCCESS; char locality_str[MAX_LOCALITY_LENGTH + 1]; int64_t pos = 0; ObLocalityDistribution locality_dist; ObArray unit_infos; if (OB_FAIL(locality_dist.init())) { LOG_WARN("fail to init locality dist", K(ret)); } else if (OB_FAIL(locality_dist.parse_locality( schema.get_locality_str(), zone_list, &zone_region_list))) { LOG_WARN("fail to parse locality", K(ret)); } else if (OB_FAIL(locality_dist.output_normalized_locality( locality_str, MAX_LOCALITY_LENGTH, pos))) { LOG_WARN("fail to normalized locality", K(ret)); } else if (OB_FAIL(schema.set_locality(locality_str))) { LOG_WARN("fail to set normalized locality back to schema", K(ret)); } else if (OB_FAIL(unit_mgr_->get_unit_infos(pools, unit_infos))) { LOG_WARN("fail to get unit infos", K(ret)); } else if (OB_FAIL(set_schema_replica_num_options(schema, locality_dist, unit_infos))) { LOG_WARN("fail to set schema replica num options", K(ret)); } else if (OB_FAIL(set_schema_zone_list( schema_guard, schema, zone_region_list))) { LOG_WARN("fail to set table schema zone list", K(ret)); } else {} // no more to do LOG_DEBUG("parse and set new locality", K(ret), K(locality_str), K(schema), K(pools), K(zone_list), K(zone_region_list), K(unit_infos)); return ret; } template int ObDDLService::set_schema_zone_list( share::schema::ObSchemaGetterGuard &schema_guard, T &schema, const common::ObIArray &zone_region_list) { int ret = OB_SUCCESS; common::ObArray zone_list; common::ObArray zone_locality; if (OB_FAIL(schema.get_zone_replica_attr_array_inherit(schema_guard, zone_locality))) { LOG_WARN("fail to get zone replica num arrary", K(ret)); } else if (OB_FAIL(generate_zone_list_by_locality( zone_locality, zone_region_list, zone_list))) { LOG_WARN("fail to generate zone list by locality", K(ret), K(zone_locality), K(zone_region_list)); } else if (OB_FAIL(schema.set_zone_list(zone_list))) { LOG_WARN("fail to set zone list", K(ret), K(zone_list)); } else {} // no more to do return ret; } int ObDDLService::generate_index_name(ObCreateIndexArg &create_index_arg, const ObTableSchema &origin_table_schema, const AddIndexNameHashSet &add_index_name_set, const DropIndexNameHashSet &drop_index_name_set, ObSchemaGetterGuard &schema_guard, ObArenaAllocator &allocator) { int ret = OB_SUCCESS; //inspect whether first_column_name is exist if (create_index_arg.index_columns_.count() < 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("create_index_arg.index_columns_ have no element", K(ret)); } else { const ObColumnSortItem &sort_column = create_index_arg.index_columns_.at(0); const ObString &first_col_name = sort_column.column_name_; //check in old database const uint64_t tenant_id = origin_table_schema.get_tenant_id(); const uint64_t database_id = origin_table_schema.get_database_id(); //short index name ObString index_name = first_col_name; bool exist = true; //use first column name as index name if (OB_FAIL(check_index_table_exist(tenant_id, database_id, origin_table_schema.get_table_id(), index_name, schema_guard, exist))) { LOG_WARN("failed to check index table exist!", K(index_name), K(origin_table_schema), K(ret)); } else { ObIndexNameHashWrapper index_key(index_name); if (!exist) { if (OB_HASH_EXIST == add_index_name_set.exist_refactored(index_key)) { exist = true; } } else { if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) { exist = false; } } } //genereate new index name use the first column c1_2, c1_3.. if (OB_SUCC(ret) && exist) { index_name.reset(); char buffer[number::ObNumber::MAX_PRINTABLE_SIZE]; for (int32_t i = 2; OB_SUCC(ret) && exist; ++i) { (void)snprintf(buffer, sizeof(buffer), "%.*s_%d", first_col_name.length(), first_col_name.ptr(), i); if (OB_FAIL(ob_write_string(allocator, ObString::make_string(buffer), index_name))) { LOG_WARN("Can not malloc space for index name", K(ret)); } else if (OB_FAIL(check_index_table_exist(tenant_id, database_id, origin_table_schema.get_table_id(), index_name, schema_guard, exist))) { LOG_WARN("failed to check table schema", K(database_id), K(tenant_id), K(index_name), K(origin_table_schema), K(ret)); } else if (exist) { continue; } else { ObIndexNameHashWrapper index_key(index_name); if (!exist) { if (OB_HASH_EXIST == add_index_name_set.exist_refactored(index_key)) { exist = true; } } else { if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) { exist = false; } } } } } if (OB_SUCC(ret)) { create_index_arg.index_name_ = index_name; } } return ret; } int ObDDLService::check_index_table_exist(const uint64_t tenant_id, const uint64_t database_id, const uint64_t table_id, const ObString &index_name, ObSchemaGetterGuard &schema_guard, bool &is_exist) { int ret = OB_SUCCESS; is_exist = false; ObString index_table_name; const ObTableSchema *index_schema = NULL; bool is_index = true; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); if (OB_FAIL(ObTableSchema::build_index_table_name(allocator, table_id, index_name, index_table_name))) { LOG_WARN("failed to build index table name", K(index_name), K(index_table_name), K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, database_id, index_table_name, is_index, index_schema))) { LOG_WARN("failed to check table schema", K(database_id), K(tenant_id), K(index_name), K(index_table_name), K(ret)); } else if (NULL != index_schema) { is_exist = true; } allocator.clear(); return ret; } // Used for alter table xxx drop primary key. // reset origin rowkey info and add heap table hidden pk column. int ObDDLService::drop_primary_key( ObTableSchema &new_table_schema) { int ret = OB_SUCCESS; // step1: clear origin primary key. const ObRowkeyInfo &rowkey_info = new_table_schema.get_rowkey_info(); for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_info.get_size(); i++) { const ObRowkeyColumn *rowkey_column = rowkey_info.get_column(i); ObColumnSchemaV2 *col = nullptr; if (OB_ISNULL(col = new_table_schema.get_column_schema(rowkey_column->column_id_))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is nullptr", K(ret), K(rowkey_column->column_id_), K(new_table_schema)); } else { col->set_rowkey_position(0); } } if (OB_SUCC(ret)) { // step2: set table pk mode and origanization mode. new_table_schema.set_table_pk_mode(ObTablePKMode::TPKM_TABLET_SEQ_PK); new_table_schema.set_table_organization_mode(ObTableOrganizationMode::TOM_HEAP_ORGANIZED); // step3: add hidden pk column. new_table_schema.reset_rowkey_info(); int32_t rowkey_position = 1; ObColumnSchemaV2 hidden_pk; hidden_pk.reset(); hidden_pk.set_column_id(OB_HIDDEN_PK_INCREMENT_COLUMN_ID); hidden_pk.set_data_type(ObUInt64Type); hidden_pk.set_nullable(false); hidden_pk.set_is_hidden(true); hidden_pk.set_charset_type(CHARSET_BINARY); hidden_pk.set_collation_type(CS_TYPE_BINARY); if (OB_FAIL(hidden_pk.set_column_name(OB_HIDDEN_PK_INCREMENT_COLUMN_NAME))) { LOG_WARN("failed to set column name", K(ret)); } else { hidden_pk.set_rowkey_position(rowkey_position); if (OB_FAIL(new_table_schema.add_column(hidden_pk))) { LOG_WARN("add column to table_schema failed", K(ret), K(hidden_pk)); } } } return ret; } int ObDDLService::add_primary_key(const ObIArray &pk_column_names, ObTableSchema &new_table_schema) { int ret = OB_SUCCESS; // step1: clear origin primary key ObTableSchema::const_column_iterator tmp_begin = new_table_schema.column_begin(); ObTableSchema::const_column_iterator tmp_end = new_table_schema.column_end(); for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) { ObColumnSchemaV2 *col = (*tmp_begin); if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; } else if (col->get_column_id() == OB_HIDDEN_PK_INCREMENT_COLUMN_ID) { // delete hidden primary key if (OB_FAIL(new_table_schema.delete_column(col->get_column_name_str()))) { LOG_WARN("fail to delete hidden primary key", K(ret)); } } else { col->set_rowkey_position(0); } new_table_schema.set_table_pk_mode(ObTablePKMode::TPKM_OLD_NO_PK); new_table_schema.set_table_organization_mode(ObTableOrganizationMode::TOM_INDEX_ORGANIZED); } if (OB_SUCC(ret)) { // step2: set new primary key rowkey_position int32_t rowkey_position = 1; new_table_schema.reset_rowkey_info(); ObTableSchema::const_column_iterator tmp_begin = new_table_schema.column_begin(); ObTableSchema::const_column_iterator tmp_end = new_table_schema.column_end(); for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) { ObColumnSchemaV2 *col = (*tmp_begin); if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is NULL", K(ret)); } else { for (int32_t i = 0; OB_SUCC(ret) && i < pk_column_names.count(); ++i) { const ObString &col_name = pk_column_names.at(i); if (0 == col->get_column_name_str().case_compare(col_name)) { col->set_rowkey_position(rowkey_position++); col->set_nullable(false); if (OB_FAIL(new_table_schema.set_rowkey_info((*col)))) { LOG_WARN("failed to set rowkey info", K(ret)); } } } } } if (OB_SUCC(ret) && OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) { LOG_WARN("fail to check primary key cover partition column", K(ret)); } } return ret; } int ObDDLService::create_hidden_table_with_pk_changed( const obrpc::ObAlterTableArg &alter_table_arg, const ObSArray &index_columns, const ObTableSchema &origin_table_schema, ObTableSchema &new_table_schema, const int64_t frozen_version, ObSchemaGetterGuard &schema_guard, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, ObArenaAllocator &allocator, const ObIndexArg::IndexActionType &index_action_type) { int ret = OB_SUCCESS; const bool bind_tablets = false; const bool is_drop_pk = ObIndexArg::DROP_PRIMARY_KEY == index_action_type; const bool is_add_or_alter_pk = (ObIndexArg::ADD_PRIMARY_KEY == index_action_type) || (ObIndexArg::ALTER_PRIMARY_KEY == index_action_type); // For add primary key and modify column in one sql, create user hidden table when modifing column. const bool create_user_hidden_table_now = !(ObIndexArg::ADD_PRIMARY_KEY == index_action_type && alter_table_arg.is_alter_columns_); if ((!is_drop_pk && !is_add_or_alter_pk) || (is_drop_pk && 0 != index_columns.count()) || (is_add_or_alter_pk && 0 == index_columns.count())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(index_action_type), K(index_columns)); } else if (is_add_or_alter_pk && OB_FAIL(add_primary_key(index_columns, new_table_schema))) { LOG_WARN("failed to add pk", K(ret), K(index_columns), K(new_table_schema)); } else if (is_drop_pk && OB_FAIL(drop_primary_key(new_table_schema))) { LOG_WARN("failed to add hidden pk column for heap table", K(ret)); } else if (create_user_hidden_table_now && OB_FAIL(create_user_hidden_table(origin_table_schema, new_table_schema, &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, frozen_version, ddl_operator, trans, allocator))) { LOG_WARN("failed to alter table offline", K(ret)); } return ret; } int ObDDLService::check_is_change_column_order( const ObTableSchema &table_schema, const AlterColumnSchema &alter_column_schema, bool &is_change_column_order) const { int ret = OB_SUCCESS; const ObString &this_name = alter_column_schema.get_origin_column_name(); const ObString &next_name = alter_column_schema.get_next_column_name(); const ObString &prev_name = alter_column_schema.get_prev_column_name(); const bool is_first = alter_column_schema.is_first_; const bool is_before = !next_name.empty(); const bool is_after = !prev_name.empty(); const int flag_cnt = static_cast(is_first) + static_cast(is_before) + static_cast(is_after); const ObSchemaOperationType op_type = alter_column_schema.alter_type_; if (OB_UNLIKELY(1 < flag_cnt)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("change column order flag conflict", K(ret), K(is_first), K(is_before), K(is_after)); } else { is_change_column_order = 0 != flag_cnt; } if (OB_SUCC(ret) && is_change_column_order) { bool is_oracle_mode = false; if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to get oracle mode", K(ret)); } else if (OB_DDL_CHANGE_COLUMN == op_type || OB_DDL_MODIFY_COLUMN == op_type) { // same order iff no non-hidden column exists in (left, right) const ObString &left_name = is_before ? this_name : prev_name; const ObString &right_name = is_before ? next_name : this_name; bool same_order = false; ObColumnIterByPrevNextID iter(table_schema); const ObColumnSchemaV2 *column = nullptr; // iter to left column, for is_first the left column is just iter start while (OB_SUCC(ret) && !same_order && !is_first) { if (OB_FAIL(iter.next(column))) { } else if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; } else if ((is_oracle_mode && left_name == column->get_column_name_str()) || (!is_oracle_mode && 0 == left_name.case_compare(column->get_column_name_str()))) { if (left_name == right_name) { same_order = true; } break; } } // iter from left to first non-shadow non-hidden column while (OB_SUCC(ret) && !same_order) { if (OB_FAIL(iter.next(column))) { } else if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; } else if (column->is_shadow_column() || column->is_hidden()) { // skip } else { if ((is_oracle_mode && right_name == column->get_column_name_str()) || (!is_oracle_mode && 0 == right_name.case_compare(column->get_column_name_str()))) { same_order = true; } break; } } if (same_order) { is_change_column_order = false; } if (OB_FAIL(ret)) { if (ret == OB_ITER_END) { ret = OB_SUCCESS; } else { LOG_WARN("failed to check is change column order", K(ret)); } } } else if (OB_DDL_ADD_COLUMN == op_type) { const ObColumnSchemaV2 *column = nullptr; if (is_after && nullptr != (column = table_schema.get_column_schema(prev_name)) && column->get_next_column_id() == BORDER_COLUMN_ID) { // add trailing column won't affect the order of old columns is_change_column_order = false; } } } return ret; } int ObDDLService::check_alter_column_is_offline( const ObTableSchema &orig_table_schema, ObSchemaGetterGuard &schema_guard, const ObColumnSchemaV2 &orig_column_schema, AlterColumnSchema &alter_column_schema, bool &is_offline) const { int ret = OB_SUCCESS; bool is_change_column_order = false; bool need_rewrite_data = false; bool add_pk = false; if (OB_FAIL(check_is_change_column_order(orig_table_schema, alter_column_schema, is_change_column_order))) { LOG_WARN("failed to check is change column order", K(ret)); } else if (OB_FAIL(orig_table_schema.check_alter_column_is_offline( &orig_column_schema, &alter_column_schema, schema_guard, need_rewrite_data))) { LOG_WARN("fail to check column can be altered", K(ret)); } else { add_pk = orig_table_schema.is_heap_table() && alter_column_schema.is_primary_key_; is_offline = is_change_column_order || need_rewrite_data || add_pk; } return ret; } // check wheter the hidden table of offline ddl contains stored generated column. int ObDDLService::check_exist_stored_gen_col( const ObTableSchema &orig_table_schema, const AlterTableSchema &alter_table_schema, bool &is_exist) { int ret = OB_SUCCESS; is_exist = false; int64_t column_cnt = 0; AlterColumnSchema *alter_column_schema = nullptr; ObTableSchema::const_column_iterator it_begin = orig_table_schema.column_begin(); ObTableSchema::const_column_iterator it_end = orig_table_schema.column_end(); for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) { ObColumnSchemaV2 *column_schema = (*it_begin); if (OB_ISNULL(column_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error", K(ret)); } else if (column_schema->is_stored_generated_column()) { column_cnt++; } else {/* do nothing. */} } it_begin = alter_table_schema.column_begin(); it_end = alter_table_schema.column_end(); for (; OB_SUCC(ret) && !is_exist && it_begin != it_end; it_begin++) { if (OB_ISNULL(alter_column_schema = static_cast(*it_begin))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin is NULL", K(ret)); } else if (alter_column_schema->is_stored_generated_column()) { if (OB_DDL_DROP_COLUMN == alter_column_schema->alter_type_) { column_cnt--; } else if (OB_DDL_ADD_COLUMN == alter_column_schema->alter_type_) { is_exist = true; } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected ddl type found here", K(ret), K(*alter_column_schema)); } } } if (OB_FAIL(ret)) { } else if (!is_exist && column_cnt > 0) { is_exist = true; } else {/* do nothing. */} return ret; } int ObDDLService::check_is_add_column_online(const ObTableSchema &table_schema, const AlterColumnSchema &alter_column_schema, ObDDLType &tmp_ddl_type) { int ret = OB_SUCCESS; tmp_ddl_type = ObDDLType::DDL_INVALID; bool is_change_column_order = false; if (OB_DDL_ADD_COLUMN != alter_column_schema.alter_type_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error", K(ret), K(alter_column_schema)); } else if (alter_column_schema.is_autoincrement_ || alter_column_schema.is_primary_key_ || alter_column_schema.has_not_null_constraint()) { tmp_ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } else if (OB_FAIL(check_is_change_column_order(table_schema, alter_column_schema, is_change_column_order))) { LOG_WARN("fail to check is change column order", K(ret)); } else if (is_change_column_order || alter_column_schema.is_stored_generated_column()) { tmp_ddl_type = ObDDLType::DDL_ADD_COLUMN_OFFLINE; } else { tmp_ddl_type = ObDDLType::DDL_ADD_COLUMN_ONLINE; } return ret; } int ObDDLService::check_is_modify_partition_key(const ObTableSchema &orig_table_schema, const AlterTableSchema &alter_table_schema, bool &is_modify_partition_key) { int ret = OB_SUCCESS; is_modify_partition_key = false; const common::ObPartitionKeyInfo &partition_keys = orig_table_schema.get_partition_key_info(); const common::ObPartitionKeyInfo &subpartition_keys = orig_table_schema.get_subpartition_key_info(); ObTableSchema::const_column_iterator iter = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator iter_end = alter_table_schema.column_end(); AlterColumnSchema *alter_column_schema = nullptr; for(; OB_SUCC(ret) && !is_modify_partition_key && iter != iter_end; iter++) { if (OB_ISNULL(alter_column_schema = static_cast(*iter))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("iter is NULL", K(ret)); } else if (OB_FAIL(orig_table_schema.is_column_in_partition_key(alter_column_schema->get_column_id(), is_modify_partition_key))) { LOG_WARN("fail to check if column in partition key", K(ret), "column_id", alter_column_schema->get_column_id()); } } return ret; } int ObDDLService::check_is_change_cst_column_name(const ObTableSchema &table_schema, const AlterTableSchema &alter_table_schema, bool &change_cst_column_name) { int ret = OB_SUCCESS; change_cst_column_name = false; ObTableSchema::const_column_iterator iter = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator iter_end = alter_table_schema.column_end(); AlterColumnSchema *alter_column_schema = nullptr; for(; OB_SUCC(ret) && !change_cst_column_name && iter != iter_end; iter++) { const ObColumnSchemaV2 *col_schema = nullptr; if (OB_ISNULL(alter_column_schema = static_cast(*iter))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("iter is NULL", K(ret)); } else if (OB_ISNULL(col_schema = table_schema.get_column_schema(alter_column_schema->get_column_id()))) { } else if (col_schema->get_column_name_str() != alter_column_schema->get_column_name_str()) { change_cst_column_name = true; } } return ret; } int ObDDLService::check_alter_table_column(obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, ObSchemaGetterGuard &schema_guard, const bool is_oracle_mode, ObDDLType &ddl_type) { int ret = OB_SUCCESS; bool is_modify_partition_key = false; common::ObIAllocator &allocator = alter_table_arg.allocator_; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0}; AlterColumnSchema *alter_column_schema = NULL; ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end(); for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) { if (OB_ISNULL(alter_column_schema = static_cast(*it_begin))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin is NULL", K(ret)); } else { const ObString &orig_column_name = alter_column_schema->get_origin_column_name(); const ObColumnSchemaV2 *orig_column_schema = orig_table_schema.get_column_schema(orig_column_name); const ObSchemaOperationType op_type = alter_column_schema->alter_type_; switch (op_type) { case OB_DDL_ADD_COLUMN: { ObDDLType tmp_ddl_type = ObDDLType::DDL_INVALID; if (OB_FAIL(check_is_add_column_online(orig_table_schema, *alter_column_schema, tmp_ddl_type))) { LOG_WARN("fail to check is add column online", K(ret)); } else if (tmp_ddl_type == ObDDLType::DDL_ADD_COLUMN_ONLINE) { if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type) { ddl_type = ObDDLType::DDL_ADD_COLUMN_ONLINE; } else if (ObDDLType::DDL_NORMAL_TYPE == ddl_type) { // ddl_type = ObDDLType::DDL_NORMAL_TYPE; } else if (ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type) { // ddl_type = ObDDLType::DDL_ADD_COLUMN_OFFLINE; } else if (ObDDLType::DDL_DROP_COLUMN == ddl_type || ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) { ddl_type = ObDDLType::DDL_COLUMN_REDEFINITION; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } } else if (tmp_ddl_type == ObDDLType::DDL_TABLE_REDEFINITION) { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } else { if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type || ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type) { ddl_type = ObDDLType::DDL_ADD_COLUMN_OFFLINE; } else if (ObDDLType::DDL_DROP_COLUMN == ddl_type || ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) { ddl_type = ObDDLType::DDL_COLUMN_REDEFINITION; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } } break; } case OB_DDL_DROP_COLUMN: { if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_DROP_COLUMN == ddl_type) { ddl_type = ObDDLType::DDL_DROP_COLUMN; } else if (ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type || ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type || ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) { ddl_type = ObDDLType::DDL_COLUMN_REDEFINITION; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } break; } case OB_DDL_CHANGE_COLUMN: case OB_DDL_MODIFY_COLUMN: { bool is_offline = false; bool add_pk = false; if (OB_FAIL(fill_column_collation(alter_table_schema.get_sql_mode(), is_oracle_mode, orig_table_schema, allocator, *alter_column_schema))) { LOG_WARN("failed to fill column collation", K(ret)); } else if (OB_ISNULL(orig_column_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid orig column schema", K(ret)); } else if (OB_FAIL(check_alter_column_is_offline( orig_table_schema, schema_guard, *orig_column_schema, *alter_column_schema, is_offline))) { LOG_WARN("failed to check is offline", K(ret)); } else if (is_offline) { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "Unsupported type modification"); } else { if (alter_column_schema->is_primary_key_) { if (orig_table_schema.get_rowkey_column_num() > 0) { if (!orig_table_schema.is_heap_table()) { ret = OB_ERR_MULTIPLE_PRI_KEY; RS_LOG(WARN, "multiple primary key defined", K(ret)); } else { add_pk = true; } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("rowkey_column_num must be greater than 0", K(ret), K(orig_table_schema.get_rowkey_column_num())); } } } } if (OB_FAIL(ret)) { } else if (add_pk) { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } else if (is_offline) { if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_MODIFY_COLUMN == ddl_type) { ddl_type = ObDDLType::DDL_MODIFY_COLUMN; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } } else { if (alter_column_schema->is_autoincrement() && !orig_column_schema->is_autoincrement()) { if (orig_column_schema->is_generated_column()) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "Changing the STORED status for generated columns"); } else if (orig_table_schema.get_autoinc_column_id() == 0) { if (orig_column_schema->is_nullable()) { // if the original table has null, we need to do double write to fill the nulls if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_MODIFY_COLUMN == ddl_type) { ddl_type = ObDDLType::DDL_MODIFY_COLUMN; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } } else { if (ObDDLType::DDL_INVALID == ddl_type) { ddl_type = ObDDLType::DDL_MODIFY_AUTO_INCREMENT; } else if (ObDDLType::DDL_MODIFY_COLUMN == ddl_type) { // ddl_type = ObDDLType::DDL_MODIFY_COLUMN; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } } } else { ret = OB_ERR_WRONG_AUTO_KEY; LOG_USER_ERROR(OB_ERR_WRONG_AUTO_KEY); } } else if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type || ObDDLType::DDL_NORMAL_TYPE == ddl_type) { ddl_type = ObDDLType::DDL_NORMAL_TYPE; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } } break; } case OB_DDL_ALTER_COLUMN: { if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type || ObDDLType::DDL_NORMAL_TYPE == ddl_type) { ddl_type = ObDDLType::DDL_NORMAL_TYPE; } else { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } break; } default: { ret = OB_INVALID_ARGUMENT; LOG_WARN("unhandled operator type!", K_(alter_column_schema->alter_type)); break; } } } } if (OB_FAIL(ret)) { } else if (ObDDLType::DDL_DROP_COLUMN == ddl_type || ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type || ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) { bool is_exist_stored_gen_col = false; // whether the target table contain stored generated column. if (OB_FAIL(check_exist_stored_gen_col(orig_table_schema, alter_table_schema, is_exist_stored_gen_col))) { LOG_WARN("fail to check exist stored generated column", K(ret)); } else if (is_exist_stored_gen_col) { // column redefinition cannot handle stored gen column, use table redefinition instead ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } else {/* do nothing. */} } if (OB_FAIL(ret)) { } else if (is_double_table_long_running_ddl(ddl_type) && OB_FAIL(check_is_modify_partition_key(orig_table_schema, alter_table_schema, is_modify_partition_key))) { LOG_WARN("fail to check is modify partition key", K(ret)); } else if (is_modify_partition_key) { ddl_type = ObDDLType::DDL_ALTER_PARTITION_BY; } return ret; } // Check whether alter primary key and alter column in one sql is allowed. // Currently, support MODIFY column and ADD PRIMARY KEY in one sql. int ObDDLService::check_support_alter_pk_and_columns( const obrpc::ObAlterTableArg &alter_table_arg, const obrpc::ObIndexArg::IndexActionType &index_action_type, bool &is_support) { int ret = OB_SUCCESS; is_support = true; if (ObIndexArg::ADD_PRIMARY_KEY != index_action_type) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(index_action_type), K(alter_table_arg)); } else if (!alter_table_arg.is_alter_columns_) { // without column operation in the sql. } else { const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; for (ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin(); OB_SUCC(ret) && is_support && it_begin != alter_table_schema.column_end(); it_begin++) { AlterColumnSchema *alter_column_schema = nullptr; if (OB_ISNULL(alter_column_schema = static_cast(*it_begin))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin is NULL", K(ret)); } else if (OB_DDL_MODIFY_COLUMN != alter_column_schema->alter_type_) { is_support = false; } else if (alter_column_schema->is_primary_key_) { ret = OB_ERR_MULTIPLE_PRI_KEY; LOG_WARN("multiple primary key defined", K(ret), KPC(alter_column_schema)); } else { // do nothing. } } } return ret; } int ObDDLService::check_alter_table_index(const obrpc::ObAlterTableArg &alter_table_arg, ObDDLType &ddl_type) { int ret = OB_SUCCESS; char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0}; ObIndexArg::IndexActionType last_type = ObIndexArg::INVALID_ACTION; const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) { ObIndexArg *index_arg = const_cast(index_arg_list.at(i)); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("index arg should not be null", K(ret)); } else { const ObIndexArg::IndexActionType type = index_arg->index_action_type_; switch(type) { case ObIndexArg::DROP_PRIMARY_KEY: { if (!is_invalid_ddl_type(ddl_type)) { ret = OB_NOT_SUPPORTED; (void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment", ObIndexArg::to_type_str(last_type), ObIndexArg::to_type_str(type)); LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); } else { ddl_type = DDL_DROP_PRIMARY_KEY; last_type = type; } break; } case ObIndexArg::ADD_PRIMARY_KEY: { bool is_support = true; if (ObDDLType::DDL_INVALID == ddl_type) { ddl_type = DDL_ADD_PRIMARY_KEY; last_type = type; } else if (OB_FAIL(check_support_alter_pk_and_columns(alter_table_arg, type, is_support))) { LOG_WARN("check support column operation and add primary key in one sql failed", K(ret), K(alter_table_arg)); } else if (is_support) { ddl_type = DDL_TABLE_REDEFINITION; last_type = type; } else { ret = OB_NOT_SUPPORTED; (void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment", ObIndexArg::to_type_str(last_type), ObIndexArg::to_type_str(type)); LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); LOG_WARN("Not Supported DDL", K(ret), K(ddl_type), K(last_type), K(type)); } break; } case ObIndexArg::ALTER_PRIMARY_KEY: { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter primary key"); } else { if (!is_invalid_ddl_type(ddl_type)) { ret = OB_NOT_SUPPORTED; (void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment", ObIndexArg::to_type_str(last_type), ObIndexArg::to_type_str(type)); LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); } else { ddl_type = DDL_ALTER_PRIMARY_KEY; last_type = type; } } break; } case ObIndexArg::ADD_INDEX: case ObIndexArg::REBUILD_INDEX: case ObIndexArg::DROP_INDEX: case ObIndexArg::ALTER_INDEX: case ObIndexArg::ALTER_INDEX_PARALLEL: case ObIndexArg::RENAME_INDEX: case ObIndexArg::ALTER_INDEX_TABLESPACE: { // offline ddl cannot appear at the same time with other ddl if ((DDL_MODIFY_COLUMN == ddl_type || DDL_ADD_COLUMN_OFFLINE == ddl_type || DDL_ADD_COLUMN_ONLINE == ddl_type || DDL_TABLE_REDEFINITION == ddl_type) && ObIndexArg::ADD_INDEX == type) { // TODO(shuangcan): distinguish simple table and double table ddl ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; last_type = type; } else if (is_long_running_ddl(ddl_type)) { ret = OB_NOT_SUPPORTED; (void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment", ddl_type_str(ddl_type), ObIndexArg::to_type_str(type)); LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); } else { ddl_type = ObDDLType::DDL_NORMAL_TYPE; last_type = type; } break; } case ObIndexArg::DROP_FOREIGN_KEY: { if (DDL_DROP_COLUMN == ddl_type) { // In oracle mode, we support to drop foreign key implicitly caused by drop column. } else if (is_long_running_ddl(ddl_type)) { ret = OB_NOT_SUPPORTED; (void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment", ddl_type_str(ddl_type), ObIndexArg::to_type_str(type)); LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); } else { ddl_type = ObDDLType::DDL_NORMAL_TYPE; last_type = type; } break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unknown index action type!", K(type), K(ret)); } } } } return ret; } int ObDDLService::check_can_convert_to_character(const share::schema::ObColumnSchemaV2 &column_schema, bool &can_convert) { int ret = OB_SUCCESS; can_convert = false; if (OB_UNLIKELY(!column_schema.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(column_schema)); } else { can_convert = (column_schema.is_string_type() || column_schema.is_enum_or_set()) && CS_TYPE_BINARY != column_schema.get_collation_type(); } return ret; } int ObDDLService::check_convert_to_character(obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, ObDDLType &ddl_type) { int ret = OB_SUCCESS; // alter table table_name CONVERT TO CHARACTER SET charset_name [COLLATE collation_name] AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; ObCharsetType charset_type = alter_table_schema.get_charset_type(); ObCollationType collation_type = alter_table_schema.get_collation_type(); if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "modify character or collation"); } else if (CS_TYPE_INVALID == collation_type) { // If collation_type is not given, the default collation_type of charset_type is used collation_type = ObCharset::get_default_collation(charset_type); alter_table_schema.set_collation_type(collation_type); alter_table_schema.set_charset_type(charset_type); } else if (!ObCharset::is_valid_collation(charset_type, collation_type)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid charset", K(ret), K(charset_type), K(collation_type)); } // This is to do a performance optimization. If the collation_type of the original table is // equivalent to the new collation_type, do nothing if (OB_SUCC(ret) && orig_table_schema.get_collation_type() != collation_type) { if (is_long_running_ddl(ddl_type)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement"); } else { bool can_convert = false; bool convert_partition_key = false; ObTableSchema::const_column_iterator tmp_begin = orig_table_schema.column_begin(); ObTableSchema::const_column_iterator tmp_end = orig_table_schema.column_end(); for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) { ObColumnSchemaV2 *col = (*tmp_begin); if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is NULL", K(ret)); } else if (OB_FAIL(check_can_convert_to_character(*col, can_convert))) { LOG_WARN("check can convert to character", K(ret)); } else if (can_convert) { if (orig_table_schema.is_column_in_foreign_key(col->get_column_id())) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "Alter column charset or collation with foreign key"); } else if (OB_FAIL(orig_table_schema.is_column_in_partition_key(col->get_column_id(), convert_partition_key))) { LOG_WARN("fail to check if column in partition key", K(ret), "column id", col->get_column_id()); } } } // change charset of a partitioned table can lead to repartition, we should handle it seperately. if (convert_partition_key) { if (is_long_running_ddl(ddl_type)) { // override the ret code here is by design. ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement"); } else { ddl_type = ObDDLType::DDL_ALTER_PARTITION_BY; } } else { ddl_type = ObDDLType::DDL_CONVERT_TO_CHARACTER; } } } return ret; } int ObDDLService::check_is_add_identity_column(const share::schema::ObTableSchema &orig_table_schema, const share::schema::ObTableSchema &hidden_table_schema, bool &is_add_identity_column) { int ret = OB_SUCCESS; ObTableSchema::const_column_iterator iter = orig_table_schema.column_begin(); ObTableSchema::const_column_iterator end = orig_table_schema.column_end(); ObTableSchema::const_column_iterator hidden_iter = hidden_table_schema.column_begin(); ObTableSchema::const_column_iterator hidden_end = hidden_table_schema.column_end(); is_add_identity_column = false; for (; OB_SUCC(ret) && hidden_iter != hidden_end; ++hidden_iter) { const ObColumnSchemaV2 *column = *hidden_iter; if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid column schema", K(column)); } else if (column->is_identity_column()) { if (is_add_identity_column) { ret = OB_ERR_IDENTITY_COLUMN_COUNT_EXCE_LIMIT; LOG_USER_ERROR(OB_ERR_IDENTITY_COLUMN_COUNT_EXCE_LIMIT); LOG_WARN("add more than one identity column is not allowed"); } else { is_add_identity_column = true; } } } for (; OB_SUCC(ret) && iter != end; ++iter) { const ObColumnSchemaV2 *column = *iter; if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid column schema", K(column)); } else if (column->is_identity_column()) { is_add_identity_column = false; } } return ret; } int ObDDLService::check_alter_table_partition(const obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const bool is_oracle_mode, ObDDLType &ddl_type) { int ret = OB_SUCCESS; const uint64_t tablegroup_id = orig_table_schema.get_tablegroup_id(); const ObPartitionLevel part_level = orig_table_schema.get_part_level(); if (obrpc::ObAlterTableArg::REPARTITION_TABLE == alter_table_arg.alter_part_type_) { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0 || (is_oracle_mode && PARTITION_LEVEL_ZERO != part_level)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "re-partition a patitioned table"); } else if (OB_INVALID_ID != tablegroup_id) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "can't modify table partition in tablegroup"); } else { if (is_long_running_ddl(ddl_type)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement"); } else { ddl_type = ObDDLType::DDL_ALTER_PARTITION_BY; } } } else { if (is_long_running_ddl(ddl_type)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement"); } else { ddl_type = ObDDLType::DDL_NORMAL_TYPE; } } return ret; } int ObDDLService::gen_alter_partition_new_table_schema_offline( const AlterTableSchema & alter_table_schema, const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema) { int ret = OB_SUCCESS; if (OB_FAIL(new_table_schema.assign(orig_table_schema))) { LOG_WARN("fail to assign table schema", K(ret)); } else if (OB_FAIL(new_table_schema.assign_partition_schema(alter_table_schema))) { LOG_WARN("fail to assign table partition schema", K(ret)); } else { new_table_schema.reset_column_info(); new_table_schema.reset_column_part_key_info(); ObTableSchema::const_column_iterator tmp_begin = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator tmp_end = alter_table_schema.column_end(); for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) { ObColumnSchemaV2 *col = (*tmp_begin); if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; } else if (OB_FAIL(new_table_schema.add_column(*col))) { LOG_WARN("fail to add column", K(ret)); } } } OZ (fill_interval_info_for_offline(alter_table_schema, new_table_schema), new_table_schema); return ret; } int ObDDLService::alter_table_partition_by( obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema, ObSchemaGetterGuard &schema_guard, const int64_t frozen_version, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; const bool bind_tablets = false; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; OZ (gen_alter_partition_new_table_schema_offline( alter_table_schema, orig_table_schema, new_table_schema)); OZ (create_user_hidden_table(orig_table_schema, new_table_schema, &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, frozen_version, ddl_operator, trans, alter_table_arg.allocator_)); return ret; } int ObDDLService::convert_to_character( obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema, ObSchemaGetterGuard &schema_guard, const int64_t frozen_version, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; const bool bind_tablets = false; bool can_convert = false; const ObSQLMode sql_mode = alter_table_arg.alter_table_schema_.get_sql_mode(); bool is_oracle_mode = false; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; ObCollationType collation_type = alter_table_schema.get_collation_type(); new_table_schema.set_collation_type(collation_type); new_table_schema.set_charset_type(ObCharset::charset_type_by_coll(collation_type)); ObTableSchema::const_column_iterator tmp_begin = orig_table_schema.column_begin(); ObTableSchema::const_column_iterator tmp_end = orig_table_schema.column_end(); if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to get oracle mode", K(ret)); } else { for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) { ObColumnSchemaV2 *orig_col = (*tmp_begin); if (OB_ISNULL(orig_col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is NULL", K(ret)); } else if (OB_FAIL(check_can_convert_to_character(*orig_col, can_convert))) { LOG_WARN("check can convert to character", K(ret)); } else if (can_convert) { ObColumnSchemaV2 *col = new_table_schema.get_column_schema(orig_col->get_column_name()); if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is NULL", K(ret)); } else { col->set_collation_type(collation_type); col->set_charset_type(ObCharset::charset_type_by_coll(collation_type)); if (OB_FAIL(fill_column_collation(sql_mode, is_oracle_mode, new_table_schema, alter_table_arg.allocator_, *col))) { LOG_WARN("failed to fill column collation", K(ret)); } } } } OZ (create_user_hidden_table(orig_table_schema, new_table_schema, &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, frozen_version, ddl_operator, trans, alter_table_arg.allocator_)); } return ret; } // check whether drop primary key is allowed. int ObDDLService::check_can_drop_primary_key(const ObTableSchema &origin_table_schema) { int ret = OB_SUCCESS; bool is_oracle_mode = false; const ObIArray &fk_infos = origin_table_schema.get_foreign_key_infos(); if (origin_table_schema.is_heap_table()) { const ObString pk_name = "PRIMAY"; ret = OB_ERR_CANT_DROP_FIELD_OR_KEY; LOG_WARN("can't DROP 'PRIMARY', check primary key exists", K(ret), K(origin_table_schema)); LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, pk_name.length(), pk_name.ptr()); } else if (fk_infos.empty()) { // allowed to drop primary key. } else if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check if oracle compat mode", K(ret)); } else if (is_oracle_mode) { // disallowed to drop pk if the table is parent table. for (int64_t i = 0; OB_SUCC(ret) && i < fk_infos.count(); i++) { if (fk_infos.at(i).parent_table_id_ == origin_table_schema.get_table_id()) { ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK; LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK); LOG_WARN("drop primary key is disallowed, due to referenced foreign key", K(ret)); } } } else { // disallow to drop pk if, // 1. the table is parent table, // 2. the table is child table and the primary key columns contain foreign key refernce columns. for (int64_t i = 0; OB_SUCC(ret) && i < fk_infos.count(); i++) { if (fk_infos.at(i).parent_table_id_ == origin_table_schema.get_table_id()) { ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK; LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK); LOG_WARN("drop primary key is disallowed, due to referenced foreign key", K(ret)); } else { bool is_all_rowkey_columns = true; // check the refernce columns is all part of rowkey columns. FOREACH_CNT_X(col_id, fk_infos.at(i).child_column_ids_, OB_SUCC(ret) && is_all_rowkey_columns) { const ObColumnSchemaV2 *col_schema = nullptr; if (OB_ISNULL(col_schema = origin_table_schema.get_column_schema(*col_id))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("child column of foreign key is nullptr", K(ret), K(*col_id)); } else if (!col_schema->is_rowkey_column()) { is_all_rowkey_columns = false; } } if (OB_SUCC(ret) && is_all_rowkey_columns) { ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK; LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK); LOG_WARN("drop primary key is disallowed, due to referenced foreign key", K(ret)); } } } } return ret; } int ObDDLService::alter_table_primary_key(obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &origin_table_schema, ObTableSchema &new_table_schema, ObSchemaGetterGuard &schema_guard, const int64_t frozen_version, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, common::ObArenaAllocator &allocator) { int ret = OB_SUCCESS; int64_t index_count = new_table_schema.get_index_tid_count(); const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) { ObIndexArg *index_arg = const_cast(index_arg_list.at(i)); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("index arg should not be null", K(ret)); } else { const ObIndexArg::IndexActionType type = index_arg->index_action_type_; switch(type) { case ObIndexArg::ADD_INDEX: { break; } case ObIndexArg::DROP_PRIMARY_KEY: { ObCreateIndexArg *create_index_arg = static_cast(index_arg); ObSArray index_columns; index_columns.reset(); if (OB_FAIL(check_can_drop_primary_key(origin_table_schema))) { LOG_WARN("fail to check whether to allow to drop primary key", K(ret)); } else if (OB_FAIL(create_hidden_table_with_pk_changed(alter_table_arg, index_columns, origin_table_schema, new_table_schema, frozen_version, schema_guard, ddl_operator, trans, allocator, type))) { LOG_WARN("failed to add hidden primary key for heap table", K(ret)); } break; } case ObIndexArg::ADD_PRIMARY_KEY: case ObIndexArg::ALTER_PRIMARY_KEY: { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_WARN("alter primary key not support yet", KR(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter primary key "); } else { if (ObIndexArg::ADD_PRIMARY_KEY == type) { if (OB_MAX_INDEX_PER_TABLE <= index_count) { ret = OB_ERR_TOO_MANY_KEYS; LOG_USER_ERROR(OB_ERR_TOO_MANY_KEYS, OB_MAX_INDEX_PER_TABLE); LOG_WARN("too many index for table!", K(index_count), K(OB_MAX_INDEX_PER_TABLE)); } else if (!new_table_schema.is_heap_table()) { ret = OB_ERR_MULTIPLE_PRI_KEY; LOG_WARN("multiple primary key defined", K(ret)); } } else { if (new_table_schema.is_heap_table()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("primary key does not exist!", K(ret)); } } if (OB_SUCC(ret)) { ObCreateIndexArg *create_index_arg = static_cast(index_arg); ObSArray index_columns; for (int32_t i = 0; OB_SUCC(ret) && i < create_index_arg->index_columns_.count(); ++i) { const ObColumnSortItem &col_item = create_index_arg->index_columns_.at(i); if (OB_FAIL(index_columns.push_back(col_item.column_name_))) { LOG_WARN("failed to add index column name", K(ret)); } } if (OB_SUCC(ret) && OB_FAIL(create_hidden_table_with_pk_changed(alter_table_arg, index_columns, origin_table_schema, new_table_schema, frozen_version, schema_guard, ddl_operator, trans, allocator, type))) { LOG_WARN("failed to add primary key", K(ret)); } } } break; } default: { ret = OB_NOT_SUPPORTED; LOG_WARN("offline ddl cannot appear at the same time with other ddl", K(ret)); } } } } return ret; } int ObDDLService::check_alter_table_constraint( const obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, share::ObDDLType &ddl_type) { int ret = OB_SUCCESS; char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0}; const ObAlterTableArg::AlterConstraintType type = alter_table_arg.alter_constraint_type_; bool change_cst_column_name = false; switch(type) { case obrpc::ObAlterTableArg::ADD_CONSTRAINT: case obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE: { if (OB_FAIL(check_is_change_cst_column_name(orig_table_schema, alter_table_arg.alter_table_schema_, change_cst_column_name))) { LOG_WARN("failed to check change cst column name", K(ret)); } else if ((ObDDLType::DDL_TABLE_REDEFINITION == ddl_type || ObDDLType::DDL_MODIFY_COLUMN == ddl_type) && !change_cst_column_name) { ddl_type = ObDDLType::DDL_TABLE_REDEFINITION; } else if (is_long_running_ddl(ddl_type)) { // if modify auto_increment and constraint together, treat it as normal modify column ret = OB_NOT_SUPPORTED; } else if (change_cst_column_name) { ddl_type = ObDDLType::DDL_CHANGE_COLUMN_NAME; ret = OB_NOT_SUPPORTED; } else { ddl_type = ObDDLType::DDL_NORMAL_TYPE; } break; } // to avoid ddl type being modified from DROP_COLUMN to NORMAL_TYPE case obrpc::ObAlterTableArg::DROP_CONSTRAINT: { if (ObDDLType::DDL_DROP_COLUMN == ddl_type) { // In oracle mode, we support to drop constraint implicitly caused by drop column. } else if (is_long_running_ddl(ddl_type)) { ret = OB_NOT_SUPPORTED; } else { ddl_type = ObDDLType::DDL_NORMAL_TYPE; } break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unknown alter constraint action type!", K(ret), K(type)); } } if (OB_NOT_SUPPORTED == ret) { (void)snprintf(err_msg, sizeof(err_msg), "%s and alter constraint in single statement", ddl_type_str(ddl_type)); LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); } return ret; } int ObDDLService::drop_constraint_caused_by_drop_column( const obrpc::ObAlterTableArg &alter_table_arg, share::schema::ObSchemaGetterGuard &schema_guard, const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans) { int ret = OB_SUCCESS; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; if (OB_SUCC(ret)) { if (OB_FAIL(check_can_alter_table_constraints(obrpc::ObAlterTableArg::DROP_CONSTRAINT, schema_guard, orig_table_schema, alter_table_schema))) { LOG_WARN("fail to check can alter constraints", K(ret), K(alter_table_schema)); } else if (OB_FAIL(ddl_operator.drop_table_constraints(orig_table_schema, alter_table_schema, new_table_schema, trans))) { LOG_WARN("failed to drop table constraints", K(ret), K(alter_table_schema)); } } return ret; } int ObDDLService::get_all_dropped_column_ids( const obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, common::ObIArray &drop_cols_id_arr, int64_t *new_table_cols_cnt) { int ret = OB_SUCCESS; int64_t columns_cnt_in_new_table = orig_table_schema.get_column_count(); ObTableSchema::const_column_iterator it = nullptr; AlterColumnSchema *alter_column_schema = nullptr; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; for (it = alter_table_schema.column_begin(); OB_SUCC(ret) && it != alter_table_schema.column_end(); it++) { if (OB_ISNULL(alter_column_schema = static_cast(*it))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("alter_column_schema is NULL", K(ret), K(alter_table_schema)); } else if (OB_DDL_ADD_COLUMN == alter_column_schema->alter_type_) { columns_cnt_in_new_table++; } else if (OB_DDL_DROP_COLUMN == alter_column_schema->alter_type_) { const ObString &orig_column_name = alter_column_schema->get_origin_column_name(); const ObColumnSchemaV2 *orig_column_schema = orig_table_schema.get_column_schema(orig_column_name); if (OB_ISNULL(orig_column_schema)) { ret = OB_ERR_CANT_DROP_FIELD_OR_KEY; LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, orig_column_name.length(), orig_column_name.ptr()); LOG_WARN("fail to find old column schema!", K(ret), K(orig_column_name), KPC(orig_column_schema)); } else if (OB_FAIL(drop_cols_id_arr.push_back(orig_column_schema->get_column_id()))) { LOG_WARN("fail to push back column id", K(ret), KPC(orig_column_schema)); } else { columns_cnt_in_new_table--; } } else {/* do nothing. */} } if (OB_SUCC(ret) && nullptr != new_table_cols_cnt) { *new_table_cols_cnt = columns_cnt_in_new_table; } return ret; } int ObDDLService::check_can_drop_column( const ObString &orig_column_name, const ObColumnSchemaV2 *orig_column_schema, const ObTableSchema &new_table_schema, const int64_t new_table_cols_cnt) { int ret = OB_SUCCESS; int64_t column_count = new_table_cols_cnt; if (OB_ISNULL(orig_column_schema) || OB_ISNULL(new_table_schema.get_column_schema(orig_column_name))) { ret = OB_ERR_CANT_DROP_FIELD_OR_KEY; LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, orig_column_name.length(), orig_column_name.ptr()); LOG_WARN("fail to find old column schema!", K(ret), K(orig_column_name), KP(orig_column_schema), K(new_table_schema)); } else if (orig_column_schema->has_generated_column_deps()) { ret = OB_ERR_DEPENDENT_BY_GENERATED_COLUMN; LOG_USER_ERROR(OB_ERR_DEPENDENT_BY_GENERATED_COLUMN, orig_column_name.length(), orig_column_name.ptr()); LOG_WARN("Dropping column has generated column deps", K(ret), K(orig_column_name)); } else if (orig_column_schema->is_tbl_part_key_column()) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop partitioning column is"); LOG_WARN("partitioning column is not allowed to be dropped", K(ret), K(orig_column_schema->get_column_name_str())); } else if ((!new_table_schema.is_heap_table() && column_count <= ObTableSchema::MIN_COLUMN_COUNT_WITH_PK_TABLE) || (new_table_schema.is_heap_table() && column_count <= ObTableSchema::MIN_COLUMN_COUNT_WITH_HEAP_TABLE)) { ret = OB_CANT_REMOVE_ALL_FIELDS; LOG_USER_ERROR(OB_CANT_REMOVE_ALL_FIELDS); LOG_WARN("Can not delete all columns in table", K(ret)); } else if (orig_column_schema->is_rowkey_column()) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop rowkey column is"); LOG_WARN("rowkey column is not allowed to be dropped", K(ret), K(orig_column_schema->get_column_name_str())); } return ret; } // to check whether the dropped column is related to constraint, and check can drop the column. int ObDDLService::check_drop_column_with_drop_constraint( const obrpc::ObAlterTableArg &alter_table_arg, share::schema::ObSchemaGetterGuard &schema_guard, const ObTableSchema &orig_table_schema, const common::ObIArray &drop_cols_id_arr) { int ret = OB_SUCCESS; bool is_oracle_mode = false; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const ObAlterTableArg::AlterConstraintType type = alter_table_arg.alter_constraint_type_; if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check if oracle compat mode", K(ret)); } else if (OB_FAIL(check_can_alter_table_constraints(type, schema_guard, orig_table_schema, alter_table_schema))) { LOG_WARN("fail to check can alter constraints", K(ret), K(type), K(alter_table_schema)); } else { FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret)) { for (ObTableSchema::const_constraint_iterator iter = orig_table_schema.constraint_begin(); OB_SUCC(ret) && iter != orig_table_schema.constraint_end(); iter++) { if (CONSTRAINT_TYPE_CHECK != (*iter)->get_constraint_type()) { } else if (0 == (*iter)->get_column_cnt()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("check cst don't have column info", K(ret), K(**iter)); } else { const ObString &cst_name = (*iter)->get_constraint_name_str(); for (ObConstraint::const_cst_col_iterator cst_col_iter = (*iter)->cst_col_begin(); OB_SUCC(ret) && (cst_col_iter != (*iter)->cst_col_end()); ++cst_col_iter) { if (*cst_col_iter == *dropped_col) { // the dropped column is related to check constraint. const ObString &dropped_column_name = orig_table_schema.get_column_schema(*dropped_col)->get_column_name_str(); ObConstraint* const* res = std::find_if(alter_table_schema.constraint_begin(), alter_table_schema.constraint_end(), [&cst_name](const ObConstraint* cst) { return 0 == cst_name.case_compare(cst->get_constraint_name_str()); }); if (alter_table_schema.constraint_end() == res) { ret = OB_ERR_DROP_COL_REFERENCED_MULTI_COLS_CONSTRAINT; LOG_WARN("the column is related to check constraint, can not be dropped", K(ret), K(cst_name), K(dropped_column_name)); if (is_oracle_mode) { LOG_USER_ERROR(OB_ERR_DROP_COL_REFERENCED_MULTI_COLS_CONSTRAINT, dropped_column_name.length(), dropped_column_name.ptr(), cst_name.length(), cst_name.ptr()); } else { LOG_USER_ERROR(OB_ERR_DROP_COL_REFERENCED_MULTI_COLS_CONSTRAINT, cst_name.length(), cst_name.ptr(), dropped_column_name.length(), dropped_column_name.ptr()); } } } } } } } } return ret; } // To be compatible with Mysql 5.6 and 8.0, follwing behavior on child table are allowed on OB 4.0: // 1. drop foreign key non-related columns and drop any foreign key in single stmt; // 2. drop the foreign key and its' some/all related columns in single stmt. // Notice that, drop fk related column on parent table has been processed in phase ddl resolver. // Here, only need to report OB_ERR_ALTER_COLUMN_FK if drop foreign key related columns without drop the fk. int ObDDLService::check_drop_column_with_drop_foreign_key( const obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const common::ObIArray &drop_cols_id_arr) { int ret = OB_SUCCESS; FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret)) { // 1. to iter all foreign keys related to the dropped column. const ObIArray &foreign_key_infos = orig_table_schema.get_foreign_key_infos(); for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { const ObForeignKeyInfo &fk_info = foreign_key_infos.at(i); const ObString &curr_fk_name = fk_info.foreign_key_name_; if (fk_info.child_table_id_ == orig_table_schema.get_table_id()) { bool is_drop_child_col = false; FOREACH_CNT_X(col_id, fk_info.child_column_ids_, OB_SUCC(ret) && !is_drop_child_col) { if (*dropped_col == *col_id) { is_drop_child_col = true; } } if (is_drop_child_col) { // 2. to check whether to drop the related foreign key. bool is_drop_curr_fk = false; const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; for (int64_t i = 0; OB_SUCC(ret) && !is_drop_curr_fk && i < index_arg_list.size(); ++i) { ObIndexArg *index_arg = const_cast(index_arg_list.at(i)); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument, index arg should not be null", K(ret), K(index_arg_list)); } else if (ObIndexArg::DROP_FOREIGN_KEY != index_arg->index_action_type_) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(index_arg)); } else { const ObDropForeignKeyArg *drop_foreign_key_arg = static_cast(index_arg); if (0 == curr_fk_name.case_compare(drop_foreign_key_arg->foreign_key_name_)) { is_drop_curr_fk = true; } } } // 3. drop child column of fk, but the fk is not dropped, should report error. if (OB_FAIL(ret)) { } else if (!is_drop_curr_fk) { const ObString &column_name = orig_table_schema.get_column_schema(*dropped_col)->get_column_name_str(); ret = OB_ERR_ALTER_COLUMN_FK; LOG_USER_ERROR(OB_ERR_ALTER_COLUMN_FK, column_name.length(), column_name.ptr()); LOG_WARN("the column is related to foreign key, and can not be dropped", K(ret), K(column_name), K(curr_fk_name)); } } } } } return ret; } int ObDDLService::delete_column_from_schema(obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObSchemaGetterGuard schema_guard; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; uint64_t tenant_id = alter_table_schema.get_tenant_id(); const ObString &orig_database_name = alter_table_schema.get_origin_database_name(); const ObString &orig_table_name = alter_table_schema.get_origin_table_name(); ObTableSchema new_table_schema; const ObTableSchema *orig_table_schema = nullptr; const ObTenantSchema *tenant_schema = nullptr; schema_guard.set_session_id(alter_table_arg.session_id_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id)); } else if (OB_ISNULL(tenant_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant schema is null", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, orig_database_name, orig_table_name, false, // is_index orig_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(orig_database_name), K(orig_table_name), KP(orig_table_schema)); } else if (OB_ISNULL(orig_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("err unexpected", K(ret), K(orig_database_name), K(orig_table_name)); } else if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; new_table_schema.set_in_offline_ddl_white_list(true); if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(drop_constraint_caused_by_drop_column(alter_table_arg, schema_guard, *orig_table_schema, new_table_schema, ddl_operator, trans))) { LOG_WARN("fail to drop constraint caused by drop column", K(ret), K(alter_table_arg)); } else if (OB_FAIL(delete_column_from_schema_in_trans(alter_table_schema, schema_guard, *orig_table_schema, new_table_schema, ddl_operator, trans))) { LOG_WARN("fail to set column to no minor", K(ret), K(new_table_schema)); } else { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema, trans, operation_type, &alter_table_arg.ddl_stmt_str_))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(delete_auto_increment_attribute(orig_table_schema, new_table_schema, alter_table_schema))) { LOG_WARN("fail to delete auto-increment attri", K(ret), K(new_table_schema), K(alter_table_schema)); } else if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("fail to publish schema", K(ret), K(tenant_id)); } } } } return ret; } int ObDDLService::delete_column_from_schema_in_trans( const AlterTableSchema &alter_table_schema, ObSchemaGetterGuard &schema_guard, const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans) { int ret = OB_SUCCESS; ObTableSchema::const_column_iterator it = nullptr; AlterColumnSchema *alter_column_schema = nullptr; for (it = alter_table_schema.column_begin(); OB_SUCC(ret) && it != alter_table_schema.column_end(); it++) { if (OB_ISNULL(alter_column_schema = static_cast(*it))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("alter_column_schema is NULL", K(ret), K(alter_table_schema)); } else { const ObString &orig_column_name = alter_column_schema->get_origin_column_name(); const ObColumnSchemaV2 *orig_column_schema = orig_table_schema.get_column_schema(orig_column_name); if (OB_ISNULL(orig_column_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ObColumnSchemaV2 should not be null", K(ret), K(*it)); } else if (OB_FAIL(ddl_operator.drop_sequence_in_drop_column( *orig_column_schema, trans, schema_guard))) { LOG_WARN("alter table drop identity column fail", K(ret)); } else if (OB_FAIL(ddl_operator.update_prev_id_for_delete_column(orig_table_schema, new_table_schema, *orig_column_schema, trans))) { LOG_WARN("failed to update column previous id for delele column", K(ret)); } else if (OB_FAIL(ddl_operator.delete_single_column(trans, new_table_schema, orig_column_name))) { LOG_WARN("fail to delete column", K(ret), K(alter_column_schema)); } else { LOG_INFO("delete column from schema", K(orig_column_name)); } } } return ret; } int ObDDLService::delete_auto_increment_attribute( const ObTableSchema *orig_table_schema, ObTableSchema &new_table_schema, AlterTableSchema &alter_table_schema) { int ret = OB_SUCCESS; uint64_t tenant_id = new_table_schema.get_tenant_id(); uint64_t table_id = new_table_schema.get_table_id(); uint64_t part_num = new_table_schema.get_all_part_num(); // drop auto-increment attr in drop column ObAutoincrementService &autoinc_service = ObAutoincrementService::get_instance(); if (0 != orig_table_schema->get_autoinc_column_id() && 0 == new_table_schema.get_autoinc_column_id()) { LOG_INFO("begin to clear all auto-increment cache", K(tenant_id), K(table_id), K(part_num), K(orig_table_schema->get_autoinc_column_id())); if (OB_FAIL(autoinc_service.clear_autoinc_cache_all(tenant_id, table_id, orig_table_schema->get_autoinc_column_id(), orig_table_schema->is_order_auto_increment_mode()))) { LOG_WARN("failed to clear auto-increment cache", K(tenant_id), K(table_id)); } } // sync sync_value(auto_increment) uint64_t auto_increment = alter_table_schema.get_auto_increment(); if (OB_SUCC(ret)) { if (new_table_schema.get_autoinc_column_id() > 0 && auto_increment > 0) { LOG_INFO("begin to sync auto_increment", K(tenant_id), K(table_id), K(part_num), K(auto_increment)); if (OB_FAIL(autoinc_service.sync_auto_increment_all(tenant_id, table_id, new_table_schema.get_autoinc_column_id(), 0 == auto_increment ? 0 : auto_increment - 1))) { LOG_WARN("failed to sync auto_increment", K(tenant_id), K(table_id), K(auto_increment)); } } } return ret; } // rename and remap all index tables. int ObDDLService::remap_index_tablets_to_new_indexs( obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObSArray &table_schemas, common::ObMySQLTransaction &trans, const int64_t frozen_version) { int ret = OB_SUCCESS; int64_t new_schema_version = OB_INVALID_VERSION; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObSchemaService *schema_service = schema_service_->get_schema_service(); ObSEArraysimple_index_infos; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not inited", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", K(ret)); } else if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("fail to get simple index infos failed", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = nullptr; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("fail to get index_table_schema", K(ret), K(tenant_id), "table_id", simple_index_infos.at(i).table_id_); } else if (OB_ISNULL(index_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("index table schema should not be null", K(ret), "table_id", simple_index_infos.at(i).table_id_); } else { ObString new_index_table_name; HEAP_VAR(ObTableSchema, new_index_table_schema) { if (OB_FAIL(new_index_table_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign new index table schema", K(ret)); } else if (OB_FAIL(gen_new_index_table_name(index_table_schema->get_table_name_str(), orig_table_schema.get_table_id(), hidden_table_schema.get_table_id(), alter_table_arg.allocator_, new_index_table_name))) { LOG_WARN("fail to build new index table name!", K(ret), K(hidden_table_schema.get_table_id()), K(new_index_table_name)); } else { uint64_t new_index_tid = OB_INVALID_ID; const ObString *ddl_stmt_str = nullptr; const bool need_sync_schema_version = true; const bool is_truncate_table = false; if (OB_FAIL(schema_service->fetch_new_table_id(tenant_id, new_index_tid))) { LOG_WARN("fail to fetch new table id", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema version", K(ret), K(tenant_id)); } else { new_index_table_schema.set_table_id(new_index_tid); new_index_table_schema.set_data_table_id(hidden_table_schema.get_table_id()); new_index_table_schema.set_table_name(new_index_table_name); new_index_table_schema.set_schema_version(new_schema_version); new_index_table_schema.set_tablet_id(index_table_schema->get_tablet_id()); new_index_table_schema.set_in_offline_ddl_white_list(true); new_index_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL); if (OB_FAIL(schema_service->get_table_sql_service().create_table(new_index_table_schema, trans, ddl_stmt_str, need_sync_schema_version, is_truncate_table))) { LOG_WARN("fail to create table", K(ret), K(need_sync_schema_version), K(is_truncate_table)); } else if (OB_FAIL(table_schemas.push_back(new_index_table_schema))) { LOG_WARN("fail to push back table schmea", K(ret)); } } } } } } } return ret; } // remap and rename index tables, and then take effect. int ObDDLService::swap_orig_and_hidden_table_state( obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans, ObSArray &table_schemas) { int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { HEAP_VARS_2((ObTableSchema, new_orig_table_schema), (ObTableSchema, new_hidden_table_schema)) { ObSEArray orig_simple_index_infos; if (OB_FAIL(new_orig_table_schema.assign(orig_table_schema)) || OB_FAIL(new_hidden_table_schema.assign(hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(orig_table_schema.get_simple_index_infos( orig_simple_index_infos))) { LOG_WARN("fail to get simple index infos", K(ret)); } else { // if the original table is a parent table, after swap the status of the two tables // need to rename the original fk name to the newly generated hidden fk name , and then // rename the newly generated hidden fk name to the original fk name on the child table if (orig_table_schema.is_parent_table()) { if (OB_FAIL(swap_all_child_table_fk_name(orig_table_schema, hidden_table_schema, schema_guard, trans, alter_table_arg.allocator_))) { LOG_WARN("fail to modify child table fk name", K(ret)); } } if (OB_SUCC(ret) && alter_table_arg.need_rebuild_trigger_) { if (OB_FAIL(rebuild_triggers_on_hidden_table(orig_table_schema, hidden_table_schema, schema_guard, ddl_operator, trans))) { LOG_WARN("fail to create triggers on hidden table", K(ret)); } } if (OB_SUCC(ret)) { ObTableSchema tmp_schema; for (int64_t i = 0; OB_SUCC(ret) && i < orig_simple_index_infos.count(); i++) { tmp_schema.reset(); const ObTableSchema *orig_index_table_schema = nullptr; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, orig_simple_index_infos.at(i).table_id_, orig_index_table_schema))) { LOG_WARN("fail to get orig index table schema", K(ret), K(tenant_id), "table_id", orig_simple_index_infos.at(i).table_id_); } else if (OB_ISNULL(orig_index_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("orig index table schema is null", K(ret), "table_id", orig_simple_index_infos.at(i).table_id_); } else if (OB_FAIL(tmp_schema.assign(*orig_index_table_schema))) { LOG_WARN("fail to assign orig index table schema", K(ret)); } else { tmp_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); if (OB_FAIL(table_schemas.push_back(tmp_schema))) { LOG_WARN("fail to add table schema", K(ret)); } } } if (OB_SUCC(ret)) { new_orig_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); new_hidden_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL); new_orig_table_schema.set_table_name(hidden_table_schema.get_table_name_str()); new_hidden_table_schema.set_table_name(orig_table_schema.get_table_name_str()); if (OB_FAIL(table_schemas.push_back(new_orig_table_schema)) || OB_FAIL(table_schemas.push_back(new_hidden_table_schema))) { LOG_WARN("fail to add table schema!", K(ret)); } } } for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) { const ObString *ddl_stmt_str = nullptr; ObTableSchema &tmp_schema = table_schemas.at(i); ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; tmp_schema.set_in_offline_ddl_white_list(true); if (tmp_schema.get_table_id() == orig_table_schema.get_table_id()) { ddl_stmt_str = &alter_table_arg.ddl_stmt_str_; } if (OB_FAIL(ddl_operator.update_table_attribute(tmp_schema, trans, operation_type, ddl_stmt_str))) { LOG_WARN("fail to update data table schema attribute", K(ret)); } } } } } return ret; } int ObDDLService::remap_index_tablets_and_take_effect( obrpc::ObAlterTableArg &alter_table_arg, const int64_t frozen_version) { int ret = OB_SUCCESS; ObSArray index_ids; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); ObSArray table_schemas; ObSchemaGetterGuard schema_guard; const ObTableSchema *orig_table_schema = nullptr; const ObTableSchema *hidden_table_schema = nullptr; schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("fail to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("fail to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { LOG_WARN("fail to get orig and hidden table schema", K(ret)); } else if (OB_FAIL(remap_index_tablets_to_new_indexs(alter_table_arg, *orig_table_schema, *hidden_table_schema, schema_guard, table_schemas, trans, frozen_version))) { LOG_WARN("fail to remap all index tables to the hidden table", K(ret)); } else if (OB_FAIL(swap_orig_and_hidden_table_state(alter_table_arg, *orig_table_schema, *hidden_table_schema, schema_guard, ddl_operator, trans, table_schemas))) { LOG_WARN("fail to swap origin and hidden table state", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } return ret; } int ObDDLService::update_autoinc_schema(obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObSchemaGetterGuard schema_guard; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; uint64_t tenant_id = alter_table_schema.get_tenant_id(); uint64_t table_id = alter_table_schema.get_table_id(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; const ObTableSchema *curr_table_schema; ObTableSchema new_table_schema; int64_t refreshed_schema_version = 0; ObAutoincrementService &auto_inc_service = ObAutoincrementService::get_instance(); if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, curr_table_schema))) { LOG_WARN("get data table schema failed", K(ret), K(tenant_id), K(table_id)); } else if (OB_ISNULL(curr_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("cannot find orig table", K(ret), K(alter_table_arg)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(new_table_schema.assign(*curr_table_schema))) { LOG_WARN("fail to assign table schema", K(ret)); } else { ObTableSchema::const_column_iterator iter = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator iter_end = alter_table_schema.column_end(); AlterColumnSchema *alter_column_schema = nullptr; ObColumnSchemaV2 *new_column_schema = nullptr; int64_t alter_column_num = 0; new_table_schema.set_in_offline_ddl_white_list(true); new_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL); for(; OB_SUCC(ret) && iter != iter_end; iter++) { if (alter_column_num != 0 || OB_ISNULL(alter_column_schema = static_cast(*iter))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected alter_column_num or iter is NULL", K(ret), K(alter_column_num)); } else { const ObString &orig_column_name = alter_column_schema->get_origin_column_name(); new_column_schema = new_table_schema.get_column_schema(orig_column_name); if (OB_ISNULL(new_column_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("alter column schema is null"); } else { new_column_schema->set_autoincrement(alter_column_schema->is_autoincrement()); new_column_schema->set_nullable(alter_column_schema->is_nullable()); new_table_schema.set_auto_increment(alter_table_schema.get_auto_increment()); new_table_schema.set_autoinc_column_id(alter_column_schema->get_column_id()); } alter_column_num++; } } if (OB_SUCC(ret) && OB_ISNULL(new_column_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("alter column schema is null"); } if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.update_single_column(trans, *curr_table_schema, new_table_schema, *new_column_schema))) { LOG_WARN("update single column failed", K(ret), K(*new_column_schema)); } else if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema, trans, operation_type, &alter_table_arg.ddl_stmt_str_))) { LOG_WARN("update table attribute failed", K(ret), K(new_table_schema)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_FAIL(ret)) { } else if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } } return ret; } int ObDDLService::lock_partitions(ObMySQLTransaction &trans, const ObTableSchema &table_schema) { int ret = OB_SUCCESS; ObTabletIDArray tablet_ids; const int64_t tenant_id = table_schema.get_tenant_id(); const int64_t table_id = table_schema.get_table_id(); // skip those type table for lock table if (!table_schema.has_tablet() || table_schema.is_aux_table() || table_schema.is_tmp_table() || table_schema.is_sys_table()) { } else if (OB_FAIL(table_schema.get_tablet_ids(tablet_ids))) { LOG_WARN("failed to get tablet ids", KR(ret), K(table_schema)); } else if (OB_FAIL(lock_tablets(trans, tenant_id, table_id, tablet_ids))) { LOG_WARN("failed to get tablet ids", KR(ret), K(table_schema), K(table_schema)); } return ret; } int ObDDLService::lock_tablets(ObMySQLTransaction &trans, const int64_t tenant_id, const int64_t table_id, const ObTabletIDArray &tablet_ids) { int ret = OB_SUCCESS; const int64_t timeout = 0; observer::ObInnerSQLConnection *conn = NULL; if (OB_ISNULL(conn = dynamic_cast (trans.get_connection()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("conn_ is NULL", KR(ret)); } else { LOG_INFO("lock tablet", KR(ret), K(tablet_ids), K(table_id), K(tenant_id), KPC(conn)); for (int i = 0; i < tablet_ids.count() && OB_SUCC(ret); i++) { if (OB_FAIL(conn->lock_tablet(tenant_id, table_id, tablet_ids.at(i), EXCLUSIVE, timeout))) { LOG_WARN("lock dest table failed", KR(ret), K(table_id), K(tenant_id)); } } } return ret; } int ObDDLService::lock_table(ObMySQLTransaction &trans, const ObTableSchema &table_schema) { int ret = OB_SUCCESS; const uint64_t table_id = table_schema.get_table_id(); const int64_t tenant_id = table_schema.get_tenant_id(); const int64_t timeout = 0; observer::ObInnerSQLConnection *conn = NULL; // skip those type table for lock table if (!table_schema.has_tablet() || table_schema.is_aux_table() || table_schema.is_tmp_table() || table_schema.is_sys_table()) { } else if (OB_ISNULL(conn = dynamic_cast (trans.get_connection()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("conn_ is NULL", KR(ret)); } else { LOG_INFO("lock table", KR(ret), K(table_id), K(tenant_id), KPC(conn)); if (OB_FAIL(conn->lock_table(tenant_id, table_id, EXCLUSIVE, timeout))) { LOG_WARN("lock dest table failed", KR(ret), K(table_schema)); } } return ret; } int ObDDLService::create_index_tablet(const ObTableSchema &index_schema, ObMySQLTransaction &trans, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; int64_t tenant_id = index_schema.get_tenant_id(); int64_t frozen_scn = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat error", K(is_inited()), KR(ret)); } else if (!index_schema.is_index_table()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("arg must be index table", KR(ret), K(tenant_id)); } else if (OB_ISNULL(GCTX.root_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("root service is null", KR(ret)); } else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) { LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id)); } else { int64_t start_usec = ObTimeUtility::current_time(); ObTableCreator table_creator( tenant_id, frozen_scn, *lst_operator_, trans); ObNewTableTabletAllocator new_table_tablet_allocator( tenant_id, schema_guard, sql_proxy_); common::ObArray ls_id_array; if (OB_FAIL(table_creator.init())) { LOG_WARN("fail to init table creator", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.init())) { LOG_WARN("fail to init new table tablet allocator", KR(ret)); } else if (index_schema.is_index_local_storage()) { const ObTableSchema *data_table_schema = NULL; const uint64_t data_table_id = index_schema.get_data_table_id(); ObSEArray schemas; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, data_table_schema))) { LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), K(data_table_id)); } else if (OB_ISNULL(data_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("data table schema not exists", KR(ret), K(data_table_id)); } else if (OB_FAIL(schemas.push_back(&index_schema))) { LOG_WARN("failed to push_back", KR(ret), K(index_schema)); } else if (OB_FAIL(new_table_tablet_allocator.prepare(index_schema))) { LOG_WARN("fail to prepare ls for index schema tablets", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array( ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_local_aux_tables_arg( schemas, data_table_schema, ls_id_array))) { LOG_WARN("create table tablet failed", KR(ret), K(index_schema)); } } else { if (OB_FAIL(new_table_tablet_allocator.prepare(index_schema))) { LOG_WARN("fail to prepare ls for index schema tablets"); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array( ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_table_arg( index_schema, ls_id_array))) { LOG_WARN("create table tablet failed", KR(ret), K(index_schema)); } } // execute create tablet if (OB_FAIL(ret)) { } else if (OB_FAIL(table_creator.execute())) { LOG_WARN("execute create partition failed", KR(ret)); } // finishing is always invoked for new table tablet allocator int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) { LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret)); } int64_t cost_usec = ObTimeUtility::current_time() - start_usec; LOG_INFO("create table partitions cost: ", K(cost_usec)); } return ret; } int ObDDLService::alter_table_index(const obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &origin_table_schema, ObTableSchema &new_table_schema, ObSchemaGetterGuard &schema_guard, const int64_t frozen_version, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, ObArenaAllocator &allocator, obrpc::ObAlterTableRes &res, ObIArray &ddl_tasks) { int ret = OB_SUCCESS; ObIndexBuilder index_builder(*this); const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; common::ObArray drop_parent_table_mock_foreign_key_infos_array; ObIArray &ddl_res_array = res.ddl_res_array_; // To many hashset will fill up the stack, construct them on heap instead HEAP_VAR(AddIndexNameHashSet, add_index_name_set) { HEAP_VAR(DropIndexNameHashSet, drop_index_name_set) { HEAP_VAR(RenameIndexNameHashSet, rename_ori_index_name_set) { HEAP_VAR(RenameIndexNameHashSet, rename_new_index_name_set) { HEAP_VAR(AlterIndexNameHashSet, alter_index_name_set) { int64_t index_count = new_table_schema.get_index_tid_count(); for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) { ObIndexArg *index_arg = const_cast(index_arg_list.at(i)); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("index arg should not be null", K(ret)); } else { if (index_arg->index_action_type_ == ObIndexArg::ADD_INDEX) { if (OB_MAX_INDEX_PER_TABLE <= index_count) { ret = OB_ERR_TOO_MANY_KEYS; LOG_USER_ERROR(OB_ERR_TOO_MANY_KEYS, OB_MAX_INDEX_PER_TABLE); LOG_WARN("too many index for table!", K(index_count), K(OB_MAX_INDEX_PER_TABLE)); } ObCreateIndexArg *create_index_arg = static_cast(index_arg); if (!new_table_schema.is_partitioned_table() && !create_index_arg->index_schema_.is_partitioned_table()) { if (INDEX_TYPE_NORMAL_GLOBAL == create_index_arg->index_type_) { create_index_arg->index_type_ = INDEX_TYPE_NORMAL_GLOBAL_LOCAL_STORAGE; } else if (INDEX_TYPE_UNIQUE_GLOBAL == create_index_arg->index_type_) { create_index_arg->index_type_ = INDEX_TYPE_UNIQUE_GLOBAL_LOCAL_STORAGE; } } if (OB_SUCC(ret)) { if (create_index_arg->index_type_ == INDEX_TYPE_PRIMARY) { if (new_table_schema.get_rowkey_column_num() > 0) { if (new_table_schema.is_heap_table()) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support to add primary key!", K(ret)); } else { ret = OB_ERR_MULTIPLE_PRI_KEY; LOG_WARN("multiple primary key defined", K(ret)); } } continue; } } if (create_index_arg->index_name_.empty()) { if (OB_FAIL(generate_index_name(*create_index_arg, new_table_schema, add_index_name_set, drop_index_name_set, schema_guard, allocator))) { LOG_WARN("generate index name failed", K(ret)); } } else { ObString index_name = create_index_arg->index_name_; ObString index_table_name; bool is_exist = false; if (OB_FAIL(check_index_table_exist(origin_table_schema.get_tenant_id(), origin_table_schema.get_database_id(), origin_table_schema.get_table_id(), index_name, schema_guard, is_exist))) { LOG_WARN("failed to check index table", K(ret)); } else { ObIndexNameHashWrapper index_key(index_name); if (!is_exist) { if (OB_HASH_EXIST == add_index_name_set.exist_refactored(index_key)) { //alter table t1 add index c1, add index c1 is_exist = true; } else if (OB_HASH_EXIST == rename_new_index_name_set.exist_refactored(index_key)) { //alter table t1 rename index ori_idx to new_idx, add index new_idx // MySQL supports this behavior is_exist = true; } } else { if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) { //alter table t1 drop index c1, add index c1 is_exist = false; } else if (OB_HASH_EXIST == rename_ori_index_name_set.exist_refactored(index_key)) { //alter table t1 rename index ori_idx to new_idx, add index ori_idx // MySQL supports this behavior is_exist = false; } } if (is_exist) { ret = OB_ERR_KEY_NAME_DUPLICATE; LOG_USER_ERROR(OB_ERR_KEY_NAME_DUPLICATE, index_name.length(), index_name.ptr()); LOG_WARN("duplicate index name", K(index_name), K(ret)); } } } if (OB_SUCC(ret)) { HEAP_VAR(ObTableSchema, index_schema) { ObArray gen_columns; bool global_storage = INDEX_TYPE_NORMAL_GLOBAL == create_index_arg->index_type_ || INDEX_TYPE_UNIQUE_GLOBAL == create_index_arg->index_type_; if (global_storage) { //FIXME: Cannot build index on tablegroup at present, // For a global index, its locality and primary take the value of the primary table // The current implementation has the following two problems: // 1. If the locality/primary_zone of the primary table is inherited semantics, // and the primary table is in the tablegroup, then the locality/primary_zone of // the inheritance relationship between the global index table and the primary table // is different, and the inherited values may be different; // 2. In addition, for the case where the primary_zone is random, // currently the leader_coordinator cannot guarantee that the results of // the random breakup of the primary table and the global index table primary_zone are consistent. if (OB_FAIL(index_schema.assign(create_index_arg->index_schema_))) { LOG_WARN("fail to assign schema", K(ret)); } else if (FALSE_IT(index_schema.set_tenant_id(origin_table_schema.get_tenant_id()))) { } } bool global_index_without_column_info = create_index_arg->index_schema_.is_partitioned_table() ? false : true; if (OB_FAIL(ret)) { } else if (OB_FAIL(ObIndexBuilderUtil::adjust_expr_index_args( *create_index_arg, new_table_schema, allocator, gen_columns))) { LOG_WARN("adjust fulltext args failed", K(ret)); } else if (OB_FAIL(index_builder.generate_schema(*create_index_arg, frozen_version, new_table_schema, global_index_without_column_info, index_schema))) { LOG_WARN("failed to generate index schema!", K(ret)); } else if (OB_FAIL(ddl_operator.alter_table_create_index(new_table_schema, frozen_version, gen_columns, index_schema, trans))) { LOG_WARN("failed to alter table add index!", K(index_schema), K(ret)); } else { // The index data is stored separately from the main table, // the partition needs to be built, and insert ori_schema_version in the outer insert if (index_schema.has_tablet() && OB_FAIL(create_index_tablet(index_schema, trans, schema_guard))) { LOG_WARN("fail to create_index_tablet", KR(ret), K(index_schema)); } if (OB_SUCC(ret)) { ObIndexNameHashWrapper index_key(create_index_arg->index_name_); if (OB_FAIL(create_index_arg->index_schema_.assign(index_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(add_index_name_set.set_refactored(index_key))) { LOG_WARN("set index name to hash set failed", K(create_index_arg->index_name_), K(ret)); } else { ++index_count; } } } } } } else if (ObIndexArg::DROP_INDEX == index_arg->index_action_type_) { ObDropIndexArg *drop_index_arg = static_cast(index_arg); drop_index_arg->tenant_id_ = origin_table_schema.get_tenant_id(); const ObString &index_name = drop_index_arg->index_name_; ObIndexNameHashWrapper index_key(index_name); if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) { //already drop in the same alter table clause ret = OB_ERR_KEY_COLUMN_DOES_NOT_EXITS; LOG_USER_ERROR(OB_ERR_KEY_COLUMN_DOES_NOT_EXITS, index_name.length(), index_name.ptr()); LOG_WARN("duplicate index name", K(index_name), K(ret)); } else if (OB_FAIL(rename_ori_index_name_set.set_refactored(index_key))) { // alter table rename ori_idx to new_idx, drop ori_idx ret = OB_ERR_KEY_DOES_NOT_EXISTS; const ObString &data_table_name = origin_table_schema.get_table_name_str(); LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, index_name.length(), index_name.ptr(), data_table_name.length(), data_table_name.ptr()); LOG_WARN("index does not exist in table", K(index_name), K(data_table_name), K(ret)); } else if (OB_FAIL(drop_index_name_set.set_refactored(index_key))) { LOG_WARN("set index name to hash set failed", K(ret), K(index_name)); } if (OB_SUCC(ret)) { const ObTableSchema *index_table_schema = nullptr; if (OB_FAIL(get_index_schema_by_name( origin_table_schema.get_table_id(), origin_table_schema.get_database_id(), *drop_index_arg, schema_guard, index_table_schema))) { LOG_WARN("get index schema by name failed", K(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_CANT_DROP_FIELD_OR_KEY; LOG_WARN("index table schema should not be null", K(*drop_index_arg), K(ret)); LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, drop_index_arg->index_name_.length(), drop_index_arg->index_name_.ptr()); } else if (index_table_schema->is_unavailable_index()) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support to drop a building index", K(ret), KPC(index_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "dropping a building index is"); } else if (drop_index_arg->is_add_to_scheduler_) { ObDDLRes ddl_res; ObDDLTaskRecord task_record; SMART_VAR(ObTableSchema, new_index_schema) { if (OB_FAIL(rename_dropping_index_name(origin_table_schema.get_table_id(), origin_table_schema.get_database_id(), *drop_index_arg, ddl_operator, trans, new_index_schema))) { LOG_WARN("submit drop index arg failed", K(ret)); } else { ObCreateDDLTaskParam param(new_index_schema.get_tenant_id(), DDL_DROP_INDEX, index_table_schema, nullptr, 0/*object_id*/, new_index_schema.get_schema_version(), 0L/*parallelism*/, &allocator, drop_index_arg); if (OB_FAIL(GCTX.root_service_->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) { LOG_WARN("submit ddl task failed", K(ret)); } else { ddl_res.task_id_ = task_record.task_id_; ddl_res.tenant_id_ = new_index_schema.get_tenant_id(); ddl_res.schema_id_ = new_index_schema.get_table_id(); if (OB_FAIL(ddl_tasks.push_back(task_record))) { LOG_WARN("push back ddl task failed", K(ret)); } else if (OB_FAIL(ddl_res_array.push_back(ddl_res))) { LOG_WARN("push back ddl res array failed", K(ret)); } } } } } else { if (OB_FAIL(ddl_operator.alter_table_drop_index( index_table_schema, new_table_schema, trans))) { LOG_WARN("failed to alter table drop index", K(*drop_index_arg), K(ret)); } else { --index_count; } } } } else if (index_arg->index_action_type_ == ObIndexArg::ALTER_INDEX) { ObAlterIndexArg *alter_index_arg = static_cast(index_arg); alter_index_arg->tenant_id_ = origin_table_schema.get_tenant_id(); const ObString &index_name = alter_index_arg->index_name_; ObIndexNameHashWrapper index_key(index_name); bool is_exist = false; if (OB_FAIL(check_index_table_exist(origin_table_schema.get_tenant_id(), origin_table_schema.get_database_id(), origin_table_schema.get_table_id(), index_name, schema_guard, is_exist))) { LOG_WARN("failed to check index table", K(ret)); } else { if (is_exist) { if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) { //alter table t1 drop index c1, alter index c1 is_exist = false; } else if ((OB_HASH_EXIST == rename_ori_index_name_set.exist_refactored(index_key)) || (OB_HASH_EXIST == rename_new_index_name_set.exist_refactored(index_key))) { //FIXME(juxanxue): we don't support alter visible and rename index operations in one statement //alter table t1 rename ori_idx to new_idx, alter ori_idx //alter table t1 rename ori_idx to new_idx, alter new_idx //In current cases, alter index will override the rename index operation is_exist = false; } } if (!is_exist) { ret = OB_ERR_KEY_DOES_NOT_EXISTS; const ObString &data_table_name = origin_table_schema.get_table_name_str(); LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, index_name.length(), index_name.ptr(), data_table_name.length(), data_table_name.ptr()); LOG_WARN("index does not exist in table", K(index_name), K(data_table_name), K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.alter_table_alter_index( origin_table_schema.get_tenant_id(), origin_table_schema.get_table_id(), origin_table_schema.get_database_id(), *alter_index_arg, trans))) { LOG_WARN("failed to alter table alter index!", K(alter_index_arg), K(ret)); } else if (OB_HASH_EXIST == alter_index_name_set.exist_refactored(index_key)) { ret = OB_ERR_ALTER_TABLE_ALTER_DUPLICATED_INDEX; LOG_USER_ERROR(OB_ERR_ALTER_TABLE_ALTER_DUPLICATED_INDEX, index_name.length(), index_name.ptr()); LOG_WARN("Duplicate alter index operations", K(ret), K(index_name)); } else if (OB_FAIL(alter_index_name_set.set_refactored(index_key))) { LOG_WARN("set index name to hash set failed", K(ret), K(index_name)); } } } else if (ObIndexArg::DROP_FOREIGN_KEY == index_arg->index_action_type_) { ObDropForeignKeyArg *drop_foreign_key_arg = static_cast(index_arg); const ObForeignKeyInfo *parent_table_mock_foreign_key_info = NULL; if (OB_FAIL(ddl_operator.alter_table_drop_foreign_key(origin_table_schema, *drop_foreign_key_arg, trans, parent_table_mock_foreign_key_info, origin_table_schema.get_in_offline_ddl_white_list()))) { LOG_WARN("failed to alter table drop foreign key", K(ret), K(drop_foreign_key_arg)); } else if (NULL != parent_table_mock_foreign_key_info) { if (OB_FAIL(drop_parent_table_mock_foreign_key_infos_array.push_back(parent_table_mock_foreign_key_info))) { LOG_WARN("failed to push back to drop_foreign_key_infos_array", K(ret), KPC(parent_table_mock_foreign_key_info)); } } } else if (ObIndexArg::ALTER_INDEX_PARALLEL == index_arg->index_action_type_) { ObAlterIndexParallelArg *alter_index_parallel_arg = static_cast(index_arg); bool is_exist = false; if (check_index_table_exist(origin_table_schema.get_tenant_id(), origin_table_schema.get_database_id(), origin_table_schema.get_table_id(), alter_index_parallel_arg->index_name_, schema_guard, is_exist)) { } else if (!is_exist) { ret = OB_ERR_UNEXPECTED; LOG_WARN("the index is not exist", K(ret), K(alter_index_parallel_arg)); } else if (OB_FAIL(ddl_operator.alter_index_table_parallel( origin_table_schema.get_tenant_id(), origin_table_schema.get_table_id(), origin_table_schema.get_database_id(), *alter_index_parallel_arg, trans))) { LOG_WARN("failed to alter index table parallel", K(ret), K(origin_table_schema.get_table_name())); } } else if (ObIndexArg::RENAME_INDEX == index_arg->index_action_type_) { ObRenameIndexArg *rename_index_arg = static_cast(index_arg); const ObString &ori_index_name = rename_index_arg->origin_index_name_; const ObString &new_index_name = rename_index_arg->new_index_name_; bool is_exist_ori = false; bool is_exist_new = false; ObIndexNameHashWrapper ori_index_key(ori_index_name); ObIndexNameHashWrapper new_index_key(new_index_name); // 1. check conflicts with origin index name // 2. check confilicts with new index name // 3. alter table rename index if (OB_FAIL(check_index_table_exist(origin_table_schema.get_tenant_id(), origin_table_schema.get_database_id(), origin_table_schema.get_table_id(), ori_index_name, schema_guard, is_exist_ori))) { LOG_WARN("failed to check index table", K(ret)); } else if (OB_FAIL(ret) || !is_exist_ori) { LOG_WARN("failed to check origin index name exist!", K(ori_index_name)); const ObString &data_table_name = origin_table_schema.get_table_name_str(); ret = OB_ERR_KEY_DOES_NOT_EXISTS; LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, ori_index_name.length(), ori_index_name.ptr(), data_table_name.length(), data_table_name.ptr()); } else if (OB_FAIL(ret) || (is_exist_ori && (OB_HASH_EXIST == add_index_name_set.exist_refactored(ori_index_key)))) { // add ori_idx, rename ori_idx to new_idx // ERROR 1176 (42000): Key 'ori_idx' doesn't exist in table LOG_WARN("failed to rename index that isn't added yet!", K(ori_index_name)); const ObString &data_table_name = origin_table_schema.get_table_name_str(); ret = OB_ERR_KEY_DOES_NOT_EXISTS; LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, ori_index_name.length(), ori_index_name.ptr(), data_table_name.length(), data_table_name.ptr()); } else if (OB_FAIL(ret) || (OB_HASH_EXIST == alter_index_name_set.exist_refactored(ori_index_key))) { // alter ori_idx, rename ori_idx to new_idx //FIXME(juxanxue): we don't support alter visible and rename index operations in one statement //In current cases, alter index will be overrided by the rename index operation LOG_WARN("failed to rename index that is altering index visible attribute!", K(ori_index_name)); ret = OB_ERR_WRONG_OBJECT; LOG_USER_ERROR(OB_ERR_WRONG_OBJECT, to_cstring(ori_index_name), "visible attribute", "updated"); } else if (OB_FAIL(check_index_table_exist(origin_table_schema.get_tenant_id(), origin_table_schema.get_database_id(), origin_table_schema.get_table_id(), new_index_name, schema_guard, is_exist_new))) { LOG_WARN("failed to check index table", K(ret)); } else if (OB_FAIL(ret) || (is_exist_new && !(OB_HASH_EXIST == drop_index_name_set.exist_refactored(new_index_key)))) { // Due to MySQL behavior, attemp to support scenario: drop idx_new, rename idx_ori to idx_new bool is_oracle_mode = false; if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check if tenant mode is oracle mode", K(ret)); } else if (is_oracle_mode) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("duplicate index name", K(ret), K(new_index_name)); } else { ret = OB_ERR_KEY_NAME_DUPLICATE; LOG_USER_ERROR(OB_ERR_KEY_NAME_DUPLICATE, new_index_name.length(), new_index_name.ptr()); LOG_WARN("duplicate index name", K(ret), K(new_index_name)); } } else if (OB_FAIL(ret) || (OB_HASH_EXIST == add_index_name_set.exist_refactored(new_index_key))) { // add new_idx, rename ori_idx to new_idx // ERROR 1061 (42000): Duplicate key name 'new_idx' ret = OB_ERR_KEY_NAME_DUPLICATE; LOG_WARN("duplicate index name", K(new_index_name), K(ret)); LOG_USER_ERROR(OB_ERR_KEY_NAME_DUPLICATE, new_index_name.length(), new_index_name.ptr()); } else if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(ori_index_key)) { // drop ori_idx, rename ori_idx to new_idx const ObString &data_table_name = origin_table_schema.get_table_name_str(); ret = OB_ERR_KEY_DOES_NOT_EXISTS; LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, ori_index_name.length(), ori_index_name.ptr(), data_table_name.length(), data_table_name.ptr()); } else { SMART_VAR(ObTableSchema, new_index_schema) { if (OB_FAIL(ddl_operator.alter_table_rename_index( origin_table_schema.get_tenant_id(), origin_table_schema.get_table_id(), origin_table_schema.get_database_id(), *rename_index_arg, trans, new_index_schema))) { LOG_WARN("failed to rename index", K(*rename_index_arg), K(ret)); } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(rename_ori_index_name_set.set_refactored(ori_index_key))) { LOG_WARN("set index name to hash set failed", K(ret), K(ori_index_name)); } else if (OB_FAIL(rename_new_index_name_set.set_refactored(new_index_key))) { LOG_WARN("set index name to hash set failed", K(ret), K(new_index_name)); } } else if (ObIndexArg::ALTER_INDEX_TABLESPACE == index_arg->index_action_type_) { ObAlterIndexTablespaceArg *alter_index_tablespace_arg = static_cast(index_arg); bool is_exist = false; if (OB_FAIL(check_index_table_exist(origin_table_schema.get_tenant_id(), origin_table_schema.get_database_id(), origin_table_schema.get_table_id(), alter_index_tablespace_arg->index_name_, schema_guard, is_exist))) { LOG_WARN("failed to check index table", K(ret)); } else if (!is_exist) { ret = OB_ERR_KEY_DOES_NOT_EXISTS; const ObString &index_name = alter_index_tablespace_arg->index_name_; const ObString &data_table_name = origin_table_schema.get_table_name_str(); LOG_WARN("the index is not exist", K(index_name), K(ret)); LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, index_name.length(), index_name.ptr(), data_table_name.length(), data_table_name.ptr()); } else if (OB_FAIL(ddl_operator.alter_index_table_tablespace(origin_table_schema.get_table_id(), origin_table_schema.get_database_id(), *alter_index_tablespace_arg, schema_guard, trans))) { LOG_WARN("failed to alter index table tablespace", K(ret), K(origin_table_schema.get_table_name())); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unknown index action type!", K_(index_arg->index_action_type), K(ret)); } } } if (OB_SUCC(ret) && !drop_parent_table_mock_foreign_key_infos_array.empty()) { ObArray mock_fk_parent_table_schema_array; ObArray mock_fk_parent_table_schema_ptr_array; ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", K(ret)); } else if (OB_FAIL(prepare_gen_mock_fk_parent_tables_for_drop_fks( schema_guard, origin_table_schema.get_tenant_id(), drop_parent_table_mock_foreign_key_infos_array, mock_fk_parent_table_schema_ptr_array, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to prepare_gen_mock_fk_parent_tables_for_drop_fks", K(ret)); } else if (OB_FAIL(gen_mock_fk_parent_tables_for_drop_fks( schema_guard, mock_fk_parent_table_schema_ptr_array, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to gen_mock_fk_parent_tables_for_drop_fks", K(ret)); } else if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) { LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(mock_fk_parent_table_schema_array)); } } } // end of alter_index_name_set } // end of rename_new_index_name_set } // end of rename_ori_index_name_set } // end of drop_index_name_set } // end of add_index_name_set return ret; } // MockFKParentTable begin int ObDDLService::gen_mock_fk_parent_table_for_create_fk( share::schema::ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, const ObCreateForeignKeyArg &foreign_key_arg, const ObMockFKParentTableSchema *tmp_mock_fk_parent_table_ptr, ObForeignKeyInfo &foreign_key_info, ObMockFKParentTableSchema &mock_fk_parent_table_schema) { int ret = OB_SUCCESS; if (!foreign_key_arg.is_parent_table_mock_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("foreign_key_arg is not parent_table_mock", K(ret)); } else { mock_fk_parent_table_schema.reset(); const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL; const ObDatabaseSchema *db_schema = NULL; if (OB_FAIL(schema_guard.get_database_schema(tenant_id, foreign_key_arg.parent_database_, db_schema))) { LOG_WARN("failed to get parent database schema", K(ret), K(foreign_key_arg)); } else if (NULL == db_schema) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, foreign_key_arg.parent_database_.length(), foreign_key_arg.parent_database_.ptr()); } else if (OB_INVALID_ID == db_schema->get_database_id()) { ret = OB_ERR_BAD_DATABASE; LOG_WARN("database id is invalid", K(ret), K(tenant_id), K(db_schema->get_database_id())); } else if (db_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("Can't not create outline of db in recyclebin", K(ret), K(*db_schema)); } else if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name( tenant_id, db_schema->get_database_id(), foreign_key_arg.parent_table_, mock_fk_parent_table_ptr))) { LOG_WARN("get_mock_fk_parent_table_schema_with_name failed", K(ret), K(tenant_id), K(db_schema->get_database_id()), K(foreign_key_arg.parent_table_)); } else if (NULL != tmp_mock_fk_parent_table_ptr && NULL != mock_fk_parent_table_ptr) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mock_fk_parent_table_ptr and tmp_mock_fk_parent_table_ptr are both not null", K(ret), KPC(mock_fk_parent_table_ptr), KPC(tmp_mock_fk_parent_table_ptr)); } else if (NULL != tmp_mock_fk_parent_table_ptr) { mock_fk_parent_table_ptr = tmp_mock_fk_parent_table_ptr; } if (OB_FAIL(ret)) { } else if (OB_ISNULL(mock_fk_parent_table_ptr)) { // mock_fk_parent_table is not exist, need to add a new one uint64_t mock_fk_parent_table_id = OB_INVALID_ID; ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must not null", K(ret)); } else if (OB_FAIL(schema_service->fetch_new_table_id(tenant_id, mock_fk_parent_table_id))) { LOG_WARN("fail to fetch_new_table_id", K(ret)); } else { mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_CREATE_TABLE_BY_ADD_FK_IN_CHILD_TBALE); mock_fk_parent_table_schema.set_tenant_id(tenant_id); mock_fk_parent_table_schema.set_database_id(db_schema->get_database_id()); mock_fk_parent_table_schema.set_mock_fk_parent_table_id(mock_fk_parent_table_id); mock_fk_parent_table_schema.set_mock_fk_parent_table_name(foreign_key_arg.parent_table_); uint64_t column_id = OB_INVALID_ID; uint64_t max_used_column_id = 0; bool is_column_exist = false; for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.parent_columns_.count(); ++j) { mock_fk_parent_table_schema.get_column_id_by_column_name(foreign_key_arg.parent_columns_.at(j), column_id, is_column_exist); if (is_column_exist) { ret = OB_ERR_COLUMN_DUPLICATE; LOG_USER_ERROR(OB_ERR_COLUMN_DUPLICATE, foreign_key_arg.parent_columns_.at(j).length(), foreign_key_arg.parent_columns_.at(j).ptr()); } else if (OB_FAIL(mock_fk_parent_table_schema.add_column_info_to_column_array(std::make_pair(++max_used_column_id, foreign_key_arg.parent_columns_.at(j))))) { LOG_WARN("fail to add_column_info_to_column_array for mock_fk_parent_table_schema", K(max_used_column_id), K(foreign_key_arg.parent_columns_.at(j))); } else if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(max_used_column_id))) { LOG_WARN("failed to push parent column id", K(ret), K(max_used_column_id)); } } } } else { // mock_fk_parent_table is already exist, need to add new columns or update schema version bool is_alter_table = (NULL == tmp_mock_fk_parent_table_ptr); mock_fk_parent_table_schema.assign(*mock_fk_parent_table_ptr); if (is_alter_table) { mock_fk_parent_table_schema.reset_column_array(); } uint64_t max_used_column_id = mock_fk_parent_table_ptr->get_column_array().at(mock_fk_parent_table_ptr->get_column_array().count() - 1).first; bool is_column_exist = false; uint64_t column_id = OB_INVALID_ID; for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.parent_columns_.count(); ++j) { mock_fk_parent_table_ptr->get_column_id_by_column_name(foreign_key_arg.parent_columns_.at(j), column_id, is_column_exist); if (is_column_exist) { if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(column_id))) { LOG_WARN("failed to push parent column id", K(ret), K(column_id)); } } else if (!is_column_exist) { if (OB_FAIL(mock_fk_parent_table_schema.add_column_info_to_column_array(std::make_pair(++max_used_column_id, foreign_key_arg.parent_columns_.at(j))))) { LOG_WARN("fail to add_column_info_to_column_array for mock_fk_parent_table_schema", K(max_used_column_id), K(foreign_key_arg.parent_columns_.at(j))); } else if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(max_used_column_id))) { LOG_WARN("failed to push parent column id", K(ret), K(max_used_column_id)); } } } if (OB_SUCC(ret)) { if (is_alter_table) { // alter table add fk if (mock_fk_parent_table_schema.get_column_array().empty()) { mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_UPDATE_SCHEMA_VERSION); } else { mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_ADD_COLUMN); } } else { // create table add fk mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_CREATE_TABLE_BY_ADD_FK_IN_CHILD_TBALE); } } } } return ret; } int ObDDLService::prepare_gen_mock_fk_parent_tables_for_drop_fks( share::schema::ObSchemaGetterGuard &schema_guard, const int64_t tenant_id, const ObIArray &foreign_key_info_array, ObIArray &mock_fk_parent_table_schema_ptr_array, ObIArray &mock_fk_parent_table_schema_array) { int ret = OB_SUCCESS; for (int64_t i = 0; i < foreign_key_info_array.count(); ++i) { const ObForeignKeyInfo* foreign_key_info = foreign_key_info_array.at(i); if (OB_ISNULL(foreign_key_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("foreign_key_info is null", K(ret)); } else if (!foreign_key_info->is_parent_table_mock_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("parent_table is not mock", K(ret), KPC(foreign_key_info)); } else { bool is_in_array = false; for (int64_t j = 0; OB_SUCC(ret) && !is_in_array && j < mock_fk_parent_table_schema_array.count(); ++j) { if (mock_fk_parent_table_schema_array.at(j).get_mock_fk_parent_table_id() == foreign_key_info->parent_table_id_) { is_in_array = true; if (OB_FAIL(mock_fk_parent_table_schema_array.at(j).add_foreign_key_info(*foreign_key_info))) { LOG_WARN("add_foreign_key_info failed", K(ret), K(foreign_key_info)); } } } if (OB_SUCC(ret) && !is_in_array) { ObMockFKParentTableSchema mock_fk_parent_table_schema; const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL; if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_id( tenant_id, foreign_key_info->parent_table_id_, mock_fk_parent_table_ptr))) { LOG_WARN("get_mock_fk_parent_table_schema_with_id failed", K(ret), K(tenant_id), K(foreign_key_info->parent_table_id_)); } else if (OB_ISNULL(mock_fk_parent_table_ptr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mock_fk_parent_table_ptr is null", K(ret)); } else if (OB_FAIL(mock_fk_parent_table_schema.assign(*mock_fk_parent_table_ptr))) { LOG_WARN("assign mock_fk_parent_table failed", K(ret)); } else if (FALSE_IT(mock_fk_parent_table_schema.reset_column_array())){ } else if (FALSE_IT(mock_fk_parent_table_schema.reset_foreign_key_infos())) { } else if (OB_FAIL(mock_fk_parent_table_schema.add_foreign_key_info(*foreign_key_info))) { LOG_WARN("add_foreign_key_info failed", K(ret), K(foreign_key_info)); } else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) { LOG_WARN("push_back mock_fk_parent_table_schema_array failed", K(ret), K(mock_fk_parent_table_schema)); } else if (OB_FAIL(mock_fk_parent_table_schema_ptr_array.push_back(mock_fk_parent_table_ptr))) { LOG_WARN("push_back mock_fk_parent_table_schema_ptr_array failed", K(ret), KPC(mock_fk_parent_table_ptr)); } } } } return ret; } int ObDDLService::gen_mock_fk_parent_tables_for_drop_fks( share::schema::ObSchemaGetterGuard &schema_guard, ObIArray &mock_fk_parent_table_schema_ptr_array, ObIArray &mock_fk_parent_table_schema_array) { int ret = OB_SUCCESS; if (mock_fk_parent_table_schema_ptr_array.count() != mock_fk_parent_table_schema_array.count()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("counts of mock_fk_parent_table_schema_ptr_array and mock_fk_parent_table_schema_array are not equal", K(ret), K(mock_fk_parent_table_schema_ptr_array.count()), K(mock_fk_parent_table_schema_array.count())); } if (OB_SUCC(ret) && !mock_fk_parent_table_schema_array.empty()) { for (int64_t i = 0; OB_SUCC(ret) && i < mock_fk_parent_table_schema_array.count(); ++i) { ObMockFKParentTableSchema &tmp_mock_fk_parent_table_schema = mock_fk_parent_table_schema_array.at(i); const ObMockFKParentTableSchema *orig_mock_fk_parent_table_ptr = mock_fk_parent_table_schema_ptr_array.at(i); ObArray foreign_key_infos_to_be_deleted; if (OB_FAIL(foreign_key_infos_to_be_deleted.assign(tmp_mock_fk_parent_table_schema.get_foreign_key_infos()))) { LOG_WARN("failed to assign foreign_key_infos_to_be_deleted", K(ret)); } else { tmp_mock_fk_parent_table_schema.reset_foreign_key_infos(); } for (int64_t j = 0; OB_SUCC(ret) && j < orig_mock_fk_parent_table_ptr->get_foreign_key_infos().count(); ++j) { bool is_to_be_remained = true; for (int64_t k = 0; is_to_be_remained && k < foreign_key_infos_to_be_deleted.count(); ++k) { if (orig_mock_fk_parent_table_ptr->get_foreign_key_infos().at(j).foreign_key_id_ == foreign_key_infos_to_be_deleted.at(k).foreign_key_id_) { is_to_be_remained = false; } } if (is_to_be_remained) { if (OB_FAIL(tmp_mock_fk_parent_table_schema.add_foreign_key_info( orig_mock_fk_parent_table_ptr->get_foreign_key_infos().at(j)))) { LOG_WARN("failed to add_foreign_key_info", K(ret), K(orig_mock_fk_parent_table_ptr->get_foreign_key_infos().at(j))); } } } if (OB_SUCC(ret)) { if (OB_FAIL(tmp_mock_fk_parent_table_schema.reconstruct_column_array_by_foreign_key_infos(orig_mock_fk_parent_table_ptr))) { LOG_WARN("reconstruct_column_array_by_foreign_key_infos failed", K(ret), K(tmp_mock_fk_parent_table_schema), KPC(orig_mock_fk_parent_table_ptr)); } else if (tmp_mock_fk_parent_table_schema.get_column_array().count() == orig_mock_fk_parent_table_ptr->get_column_array().count()) { // If all columns in mock_fk_parent_table_schema are still referenced by other fk, only update schema version tmp_mock_fk_parent_table_schema.set_operation_type(MOCK_FK_PARENT_TABLE_OP_UPDATE_SCHEMA_VERSION); } else if (tmp_mock_fk_parent_table_schema.get_column_array().count() == 0) { // If all columns in mock_fk_parent_table_schema will be dropped, drop this mock_fk_parent_table if (OB_FAIL(tmp_mock_fk_parent_table_schema.set_column_array(orig_mock_fk_parent_table_ptr->get_column_array()))) { LOG_WARN("set_column_array failed", K(ret), K(tmp_mock_fk_parent_table_schema), KPC(orig_mock_fk_parent_table_ptr)); } else { tmp_mock_fk_parent_table_schema.set_operation_type(MOCK_FK_PARENT_TABLE_OP_DROP_TABLE); } } else { // If part of columns in mock_fk_parent_table_schema will be dropped, drop columns in this mock_fk_parent_table tmp_mock_fk_parent_table_schema.set_operation_type(MOCK_FK_PARENT_TABLE_OP_DROP_COLUMN); ObMockFKParentTableColumnArray columns_to_be_remained; if (OB_FAIL(columns_to_be_remained.assign(tmp_mock_fk_parent_table_schema.get_column_array()))) { LOG_WARN("failed to assign columns_to_be_remained", K(ret)); } else { tmp_mock_fk_parent_table_schema.reset_column_array(); } for (int64_t i = 0; OB_SUCC(ret) && i < orig_mock_fk_parent_table_ptr->get_column_array().count(); ++i) { bool is_col_to_be_deleted = true; for (int64_t j = 0; is_col_to_be_deleted && j < columns_to_be_remained.count(); ++j) { if (orig_mock_fk_parent_table_ptr->get_column_array().at(i).first == columns_to_be_remained.at(j).first) { is_col_to_be_deleted = false; } } if (is_col_to_be_deleted) { if (OB_FAIL(tmp_mock_fk_parent_table_schema.add_column_info_to_column_array( std::make_pair(orig_mock_fk_parent_table_ptr->get_column_array().at(i).first, orig_mock_fk_parent_table_ptr->get_column_array().at(i).second)))) { LOG_WARN("fail to add_column_info_to_column_array", K(ret)); } } } } } } } return ret; } int ObDDLService::gen_mock_fk_parent_table_for_drop_table( share::schema::ObSchemaService *schema_service, share::schema::ObSchemaGetterGuard &schema_guard, const obrpc::ObDropTableArg &drop_table_arg, const DropTableIdHashSet &drop_table_set, const ObIArray &foreign_key_infos, const ObForeignKeyInfo &violated_foreign_key_info, const ObTableSchema *table_schema, ObMockFKParentTableSchema &mock_fk_parent_table_schema) { int ret = OB_SUCCESS; const ObTableSchema *child_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(drop_table_arg.tenant_id_, violated_foreign_key_info.child_table_id_, child_table_schema))) { LOG_WARN("get_table_schema failed", K(ret), K(violated_foreign_key_info)); } else if (OB_ISNULL(child_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("child_table is null", K(ret), K(violated_foreign_key_info)); } else if (drop_table_arg.foreign_key_checks_) { ret = OB_ERR_TABLE_IS_REFERENCED; LOG_USER_ERROR(OB_ERR_TABLE_IS_REFERENCED, table_schema->get_table_name_str().length(), table_schema->get_table_name_str().ptr(), violated_foreign_key_info.foreign_key_name_.length(), violated_foreign_key_info.foreign_key_name_.ptr(), child_table_schema->get_table_name_str().length(), child_table_schema->get_table_name_str().ptr()); } else if (!drop_table_arg.foreign_key_checks_) { ret = OB_SUCCESS; uint64_t mock_fk_parent_table_id = OB_INVALID_ID; if (drop_table_arg.tables_.count() > 1) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop fk parent table and more tables in one sql"); LOG_WARN("drop fk parent table and more tables in one sqll", K(ret), KPC(child_table_schema), KPC(table_schema)); } else if (child_table_schema->get_database_id() != table_schema->get_database_id()) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop fk parent table which has child tables in different database"); LOG_WARN("drop fk parent table which has child tables in different database not supported", K(ret), KPC(child_table_schema), KPC(table_schema)); } else if (OB_FAIL(schema_service->fetch_new_table_id(table_schema->get_tenant_id(), mock_fk_parent_table_id))) { LOG_WARN("fail to fetch_new_table_id", K(ret)); } else { // mock_fk_parent_table_ptr = &mock_fk_parent_table_schema; mock_fk_parent_table_schema.set_tenant_id(table_schema->get_tenant_id()); mock_fk_parent_table_schema.set_database_id(table_schema->get_database_id()); mock_fk_parent_table_schema.set_mock_fk_parent_table_id(mock_fk_parent_table_id); mock_fk_parent_table_schema.set_mock_fk_parent_table_name(table_schema->get_table_name_str()); } if (OB_SUCC(ret)) { for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); ++i) { ObForeignKeyInfo tmp_foreign_key_info = foreign_key_infos.at(i); if (OB_HASH_EXIST == drop_table_set.exist_refactored(tmp_foreign_key_info.child_table_id_)) { // do-nothing. // child table will be dropped in the same drop table SQL. multiple transactions update the schema, but the schema cache is not refreshed. } else if (tmp_foreign_key_info.child_table_id_ == tmp_foreign_key_info.parent_table_id_) { // do nothing, self referenced fk should be dropped. } else if (table_schema->get_table_id() != violated_foreign_key_info.child_table_id_) { // The difference of foreign_key_info between orig_parent_table and mock_fk_parent_table is only parent_table_id. // parent_column_ids are all the same. const int64_t invalid_cst_id = 0; tmp_foreign_key_info.set_parent_table_id(mock_fk_parent_table_schema.get_mock_fk_parent_table_id()); tmp_foreign_key_info.set_is_parent_table_mock(true); tmp_foreign_key_info.set_ref_cst_type(CONSTRAINT_TYPE_INVALID); tmp_foreign_key_info.set_ref_cst_id(invalid_cst_id); if (OB_FAIL(mock_fk_parent_table_schema.add_foreign_key_info(tmp_foreign_key_info))) { LOG_WARN("fail to add_foreign_key_info for mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema), K(tmp_foreign_key_info)); } else { // add column info of foreign_key_infos to mock_fk_parent_table_schema info for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_infos.at(i).parent_column_ids_.count(); ++j) { const ObColumnSchemaV2 * parent_column_schema = NULL; if (OB_ISNULL(parent_column_schema = table_schema->get_column_schema(foreign_key_infos.at(i).parent_column_ids_.at(j)))){ ret = OB_ERR_UNEXPECTED; LOG_WARN("got null column schema", KPC(table_schema), K(i), K(foreign_key_infos.at(i)), K(j), K(foreign_key_infos.at(i).parent_column_ids_.at(j))); } else { bool is_existed = false; // check if column info is in mock_fk_parent_table_schema already for (int64_t k = 0; !is_existed && k < mock_fk_parent_table_schema.get_column_array().count(); ++k) { if (parent_column_schema->get_column_id() == mock_fk_parent_table_schema.get_column_array().at(k).first && 0 == parent_column_schema->get_column_name_str().compare(mock_fk_parent_table_schema.get_column_array().at(k).second)) { is_existed = true; } } if (!is_existed) { if (OB_FAIL(mock_fk_parent_table_schema.add_column_info_to_column_array(std::make_pair(parent_column_schema->get_column_id(), parent_column_schema->get_column_name_str())))) { LOG_WARN("fail to add_column_info_to_column_array for mock_fk_parent_table_schema", KPC(parent_column_schema)); } } } } } } else { // This fk is only a self reference fk, do nothing } } } } return ret; } int ObDDLService::check_fk_columns_type_for_replacing_mock_fk_parent_table( share::schema::ObSchemaGetterGuard &schema_guard, const ObTableSchema &real_parent_table_schema, const ObMockFKParentTableSchema *&mock_parent_table_schema) { int ret = OB_SUCCESS; if (OB_ISNULL(mock_parent_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mock_parent_table_schema is not exist", K(ret)); } else { if (OB_ISNULL(mock_parent_table_schema)) { ret = OB_ERR_UNEXPECTED; SQL_RESV_LOG(WARN, "mock_parent_table_schema is null", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < mock_parent_table_schema->get_foreign_key_infos().count(); ++i) { const ObTableSchema *child_table_schema = NULL; const ObForeignKeyInfo &fk_info = mock_parent_table_schema->get_foreign_key_infos().at(i); if (OB_FAIL(schema_guard.get_table_schema(mock_parent_table_schema->get_tenant_id(), fk_info.child_table_id_, child_table_schema))) { LOG_WARN("table is not exist", K(ret), K(fk_info.child_table_id_)); } else if (OB_ISNULL(child_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("parent table schema is null", K(ret)); } else { // prepare params for check_foreign_key_columns_type ObArray child_columns; ObArray parent_columns; bool is_column_exist = false; for (int64_t j = 0; OB_SUCC(ret) && j < fk_info.child_column_ids_.count(); ++j) { ObString child_column_name; const ObColumnSchemaV2 *child_col = child_table_schema->get_column_schema(fk_info.child_column_ids_.at(j)); if (OB_ISNULL(child_col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column is not exist", K(ret), K(fk_info), K(j)); } else if (OB_FAIL(child_columns.push_back(child_col->get_column_name_str()))) { LOG_WARN("fail to push_back to child_columns", K(ret), K(child_col->get_column_name_str())); } } for (int64_t j = 0; OB_SUCC(ret) && j < fk_info.parent_column_ids_.count(); ++j) { ObString parent_column_name; mock_parent_table_schema->get_column_name_by_column_id(fk_info.parent_column_ids_.at(j), parent_column_name, is_column_exist); if (!is_column_exist) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column is not exist", K(ret), K(fk_info), K(j)); } else if (OB_FAIL(parent_columns.push_back(parent_column_name))) { LOG_WARN("fail to push_back to real_parent_table_schema_columns", K(ret)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(ObResolverUtils::check_foreign_key_columns_type( *child_table_schema, real_parent_table_schema, child_columns, parent_columns, NULL))) { ret = OB_ERR_CANNOT_ADD_FOREIGN; LOG_WARN("Failed to check_foreign_key_columns_type", K(ret)); } } } } } return ret; } int ObDDLService::get_uk_cst_id_for_replacing_mock_fk_parent_table( const ObIArray &index_table_schemas, share::schema::ObForeignKeyInfo &foreign_key_info) { int ret = OB_SUCCESS; bool is_match = false; for (int64_t i = 0; OB_SUCC(ret) && !is_match && i < index_table_schemas.count(); ++i) { const ObTableSchema *index_table_schema = index_table_schemas.at(i); if (index_table_schema->is_unique_index()) { const ObColumnSchemaV2 *index_col = NULL; const ObIndexInfo &index_info = index_table_schema->get_index_info(); ObArray uk_columns; for (int64_t j = 0; OB_SUCC(ret) && j < index_info.get_size(); ++j) { if (OB_ISNULL(index_col = index_table_schema->get_column_schema(index_info.get_column(j)->column_id_))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get index column schema failed", K(ret), K(index_info.get_column(j)->column_id_)); } else if (index_col->is_hidden() || index_col->is_shadow_column()) { // do nothing } else if (OB_FAIL(uk_columns.push_back(index_info.get_column(j)->column_id_))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("push back index column failed", K(ret), KPC(index_col)); } } if (OB_SUCC(ret)) { if (OB_FAIL(sql::ObResolverUtils::check_match_columns(foreign_key_info.parent_column_ids_, uk_columns, is_match))) { LOG_WARN("Failed to check_match_columns", K(ret)); } else if (is_match) { foreign_key_info.ref_cst_type_ = CONSTRAINT_TYPE_UNIQUE_KEY; foreign_key_info.ref_cst_id_ = index_table_schema->get_table_id(); } } } } return ret; } // replace mock fk parent table with real fk parent table int ObDDLService::gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table( share::schema::ObSchemaGetterGuard &schema_guard, const uint64_t drop_mock_fk_parent_table_id, const share::schema::ObTableSchema &real_parent_table, const ObIArray &uk_index_schemas, ObMockFKParentTableSchema &mock_fk_parent_table_schema) { int ret = OB_SUCCESS; const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL; if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_id( real_parent_table.get_tenant_id(), drop_mock_fk_parent_table_id, mock_fk_parent_table_ptr))) { LOG_WARN("check_mock_fk_parent_table_exist_by_id failed", K(ret), K(real_parent_table.get_tenant_id()), K(drop_mock_fk_parent_table_id)); } else if (OB_ISNULL(mock_fk_parent_table_ptr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mock_fk_parent_table is not exist", K(ret), K(mock_fk_parent_table_ptr)); } else if (OB_FAIL(mock_fk_parent_table_schema.assign(*mock_fk_parent_table_ptr))) { LOG_WARN("mock_fk_parent_table_schema assign failed", K(ret), KPC(mock_fk_parent_table_ptr)); } else if (FALSE_IT(mock_fk_parent_table_schema.set_operation_type(share::schema::ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_REPLACED_BY_REAL_PREANT_TABLE))) { } else if (OB_FAIL(check_fk_columns_type_for_replacing_mock_fk_parent_table(schema_guard, real_parent_table, mock_fk_parent_table_ptr))) { LOG_WARN("check_fk_columns_type_for_replacing_mock_fk_parent_table failed", K(ret), K(real_parent_table), KPC(mock_fk_parent_table_ptr)); } else { const ObIArray &ori_mock_fk_infos_array = mock_fk_parent_table_ptr->get_foreign_key_infos(); // modify the parent column id of fk,make it fit with real parent table // mock_column_id -> column_name -> real_column_id bool is_column_exist = false; for (int64_t i = 0; OB_SUCC(ret) && i < ori_mock_fk_infos_array.count(); ++i) { mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_column_ids_.reuse(); mock_fk_parent_table_schema.get_foreign_key_infos().at(i).ref_cst_type_ = CONSTRAINT_TYPE_INVALID; mock_fk_parent_table_schema.get_foreign_key_infos().at(i).is_parent_table_mock_ = false; mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_table_id_ = real_parent_table.get_table_id(); for (int64_t j = 0; OB_SUCC(ret) && j < ori_mock_fk_infos_array.at(i).parent_column_ids_.count(); ++j) { uint64_t mock_parent_table_column_id = ori_mock_fk_infos_array.at(i).parent_column_ids_.at(j); ObString column_name; mock_fk_parent_table_ptr->get_column_name_by_column_id(mock_parent_table_column_id, column_name, is_column_exist); if (!is_column_exist) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column is not exist", K(ret), K(mock_parent_table_column_id), KPC(mock_fk_parent_table_ptr)); } else { const ObColumnSchemaV2 *col_schema = NULL; if (OB_ISNULL(col_schema = real_parent_table.get_column_schema(column_name))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get column schema failed", K(ret), K(column_name)); } else if (OB_FAIL(mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_column_ids_.push_back(col_schema->get_column_id()))) { LOG_WARN("push_back to parent_column_ids failed", K(ret), K(col_schema->get_column_id())); } } } // check and mofidy ref cst type and ref cst id of fk const ObRowkeyInfo &rowkey_info = real_parent_table.get_rowkey_info(); common::ObArray pk_column_ids; for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_info.get_size(); ++i) { uint64_t column_id = 0; const ObColumnSchemaV2 *col_schema = NULL; if (OB_FAIL(rowkey_info.get_column_id(i, column_id))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to get rowkey info", K(ret), K(i), K(rowkey_info)); } else if (OB_ISNULL(col_schema = real_parent_table.get_column_schema(column_id))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get index column schema failed", K(ret)); } else if (col_schema->is_hidden() || col_schema->is_shadow_column()) { // do nothing } else if(OB_FAIL(pk_column_ids.push_back(col_schema->get_column_id()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("push back column_id failed", K(ret), K(col_schema->get_column_id())); } } bool is_match = false; if (OB_FAIL(ret)) { } else if (OB_FAIL(sql::ObResolverUtils::check_match_columns(pk_column_ids, mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_column_ids_, is_match))) { LOG_WARN("check_match_columns failed", K(ret)); } else if (is_match) { mock_fk_parent_table_schema.get_foreign_key_infos().at(i).ref_cst_type_ = CONSTRAINT_TYPE_PRIMARY_KEY; } else { // pk is not match, check if uk match if (OB_FAIL(get_uk_cst_id_for_replacing_mock_fk_parent_table( uk_index_schemas, mock_fk_parent_table_schema.get_foreign_key_infos().at(i)))) { LOG_WARN("fail to get_uk_cst_id_for_replacing_mock_fk_parent_table", K(ret)); } else if (CONSTRAINT_TYPE_INVALID == mock_fk_parent_table_schema.get_foreign_key_infos().at(i).ref_cst_type_) { ret = OB_ERR_CANNOT_ADD_FOREIGN; LOG_WARN("ref_cst_type is invalid", K(ret), KPC(mock_fk_parent_table_ptr)); } } } } return ret; } // MockFKParentTable end int ObDDLService::get_index_schema_by_name( const uint64_t data_table_id, const uint64_t database_id, const ObDropIndexArg &drop_index_arg, ObSchemaGetterGuard &schema_guard, const ObTableSchema *&index_table_schema) { int ret = OB_SUCCESS; const uint64_t tenant_id = drop_index_arg.tenant_id_; ObString index_table_name; ObArenaAllocator allocator(ObModIds::OB_SCHEMA); const ObString &index_name = drop_index_arg.index_name_; //build index name and get index schema if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("failed to get schema guard", K(ret)); } else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator, data_table_id, index_name, index_table_name))) { LOG_WARN("build_index_table_name failed", K(ret), K(data_table_id), K(index_name)); } else { const bool is_index = true; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, database_id, index_table_name, is_index, index_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(database_id), K(index_table_schema)); } else if (nullptr == index_table_schema) { ret = OB_ERR_CANT_DROP_FIELD_OR_KEY; LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, index_name.length(), index_name.ptr()); LOG_WARN("get index table schema failed", K(tenant_id), K(database_id), K(index_table_name), K(ret)); } else if (index_table_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("index table is in recyclebin", K(ret)); } } return ret; } // To avoid ddl hung when drop and add index with same index-name in single stmt, // should rename dropping index firstly, and then push it into ddl scheduler queue. int ObDDLService::rename_dropping_index_name( const uint64_t data_table_id, const uint64_t database_id, const ObDropIndexArg &drop_index_arg, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans, share::schema::ObTableSchema &new_index_schema) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTableSchema *index_table_schema = nullptr; int nwrite = 0; const int64_t buf_size = number::ObNumber::MAX_PRINTABLE_SIZE; char buf[buf_size] = {0}; ObString index_name = drop_index_arg.index_name_; if (OB_FAIL(get_index_schema_by_name(data_table_id, database_id, drop_index_arg, schema_guard, index_table_schema))) { LOG_WARN("get index schema by name", K(ret), K(data_table_id), K(database_id)); } else if ((nwrite = snprintf(buf, buf_size, "%s_%s_%lu", index_name.ptr(), "DELETING", ObTimeUtility::current_time())) >= buf_size || nwrite < 0) { ret = common::OB_BUF_NOT_ENOUGH; LOG_WARN("buf is not large enough", K(ret), K(buf_size)); } else { ObString new_index_name = ObString::make_string(buf); obrpc::ObRenameIndexArg rename_index_arg; rename_index_arg.tenant_id_ = index_table_schema->get_tenant_id(); rename_index_arg.origin_index_name_ = index_name; rename_index_arg.new_index_name_ = new_index_name; if (OB_FAIL(ddl_operator.alter_table_rename_index(index_table_schema->get_tenant_id(), index_table_schema->get_data_table_id(), index_table_schema->get_database_id(), rename_index_arg, trans, new_index_schema))) { LOG_WARN("rename index failed", K(ret)); } } return ret; } int ObDDLService::generate_tmp_idx_schemas( const ObTableSchema &new_table_schema, ObIArray &idx_schemas, ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = new_table_schema.get_tenant_id(); ObSEArray simple_index_infos; if (OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { RS_LOG(WARN, "get_table_schema failed", K(ret), K(tenant_id), "table id", simple_index_infos.at(i).table_id_); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; RS_LOG(WARN, "table schema should not be null", K(ret)); } else { if (OB_FAIL(idx_schemas.push_back(*index_table_schema))) { RS_LOG(WARN, "fail to push back to idx_schemas", K(ret)); } } } } return ret; } int ObDDLService::fill_column_collation( const ObSQLMode sql_mode, const bool is_oracle_mode, const ObTableSchema &table_schema, common::ObIAllocator &allocator, ObColumnSchemaV2 &column_schema) { int ret = OB_SUCCESS; ObObjTypeClass col_tc = column_schema.get_data_type_class(); ObCollationType collation_type = table_schema.get_collation_type(); ObCharsetType charset_type = table_schema.get_charset_type(); const ObCollationType cur_extended_type_info_collation = ObCharset::get_system_collation(); if (ObStringTC == col_tc) { if (OB_FAIL(ObDDLResolver::check_and_fill_column_charset_info( column_schema, charset_type, collation_type))) { RS_LOG(WARN, "failed to fill column charset info", K(ret)); } else if (OB_FAIL(ObDDLResolver::check_string_column_length( column_schema, is_oracle_mode))) { RS_LOG(WARN, "failed to check string column length", K(ret)); } } else if (ObRawTC == col_tc) { if (OB_FAIL(ObDDLResolver::check_raw_column_length(column_schema))) { RS_LOG(WARN, "failed to check raw column length", K(ret), K(column_schema)); } } else if (ob_is_text_tc(column_schema.get_data_type())) { if (OB_FAIL(ObDDLResolver::check_and_fill_column_charset_info( column_schema, table_schema.get_charset_type(), table_schema.get_collation_type()))) { RS_LOG(WARN, "failed to fill column charset info", K(ret)); } else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(column_schema, table_schema.get_table_id()))) { RS_LOG(WARN, "failed to check text or blob column length", K(ret)); } } else if (ObEnumSetTC == col_tc) { if (OB_FAIL(ObDDLResolver::check_and_fill_column_charset_info(column_schema, charset_type, collation_type))) { LOG_WARN("fail to check and fill column charset info", K(ret), K(column_schema)); } else if (OB_FAIL(ObResolverUtils::check_extended_type_info( allocator, column_schema.get_extended_type_info(), cur_extended_type_info_collation, column_schema.get_column_name_str(), column_schema.get_data_type(), column_schema.get_collation_type(), sql_mode))) { LOG_WARN("fail to fill extended type info", K(ret), K(column_schema)); } else if (OB_FAIL(ObDDLResolver::calc_enum_or_set_data_length(column_schema))) { LOG_WARN("fail to calc data length", K(ret), K(column_schema)); } } return ret; } int ObDDLService::resolve_orig_default_value(ObColumnSchemaV2 &alter_column_schema, const ObTimeZoneInfoWrap &tz_info_wrap, const common::ObString *nls_formats, ObIAllocator &allocator) { int ret = OB_SUCCESS; const ObObj &cur_default_value = alter_column_schema.get_cur_default_value(); if (alter_column_schema.is_identity_column()) { // When column is identity column or generated column, cur_default_value and orig_default_value is different // cur_default_value:"SEQUENCE.NEXTVAL" // orig_default_value:pure_sequnece_id } else if (!cur_default_value.is_null()) { if (OB_FAIL(alter_column_schema.set_orig_default_value(cur_default_value))) { LOG_WARN("fail to set orig default value for alter table", K(ret), K(cur_default_value)); } } else if (alter_column_schema.is_nullable()) { ObObj null_obj; null_obj.set_null(); if (OB_FAIL(alter_column_schema.set_orig_default_value(null_obj))) { LOG_WARN("fail to set origin default value", K(ret)); } } else { ObObj default_value; default_value.set_type(alter_column_schema.get_data_type()); if (OB_FAIL(default_value.build_not_strict_default_value())) { LOG_WARN("failed to build not strict default value", K(ret)); } else if (OB_FAIL(alter_column_schema.set_orig_default_value(default_value))) { LOG_WARN("failed to set orig default value", K(ret)); } } if (OB_SUCC(ret) && !alter_column_schema.is_identity_column()) { ObObj orig_default_value = alter_column_schema.get_orig_default_value(); if (IS_DEFAULT_NOW_OBJ(orig_default_value) || alter_column_schema.is_default_expr_v2_column()) { if (OB_FAIL(ObDDLResolver::calc_default_value(alter_column_schema, orig_default_value, tz_info_wrap, nls_formats, allocator))) { LOG_WARN("fail to calc default now expr", K(ret)); } else if (OB_FAIL(alter_column_schema.set_orig_default_value(orig_default_value))) { LOG_WARN("fail to set orig default value", K(orig_default_value), K(ret)); } } } return ret; } int ObDDLService::resolve_timestamp_column(AlterColumnSchema *alter_column_schema, ObTableSchema &new_table_schema, ObColumnSchemaV2 &new_column_schema, const common::ObTimeZoneInfoWrap &tz_info_wrap, const common::ObString *nls_formats, ObIAllocator &allocator) { int ret = OB_SUCCESS; if (OB_ISNULL(alter_column_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("alter_column_schema is NULL", K(ret)); } else if (ObTimestampType != new_column_schema.get_data_type() || new_column_schema.is_generated_column() || false == alter_column_schema->check_timestamp_column_order_) { //nothing to do } else { bool is_first_timestamp = false; ObTableSchema::const_column_iterator it_begin = new_table_schema.column_begin(); ObTableSchema::const_column_iterator it_end = new_table_schema.column_end(); bool found = false; for(; OB_SUCC(ret) && it_begin != it_end && !found; it_begin++) { if (OB_ISNULL(it_begin)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("it_begin should not be NULL", K(ret)); } else if (OB_ISNULL(*it_begin)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin should not be NULL", K(ret)); } else { if (ObTimestampType == (*it_begin)->get_data_type()) { if (0 == (*it_begin)->get_column_name_str().case_compare( new_column_schema.get_column_name_str())) { is_first_timestamp = true; } found = true; } } } if (OB_SUCC(ret)) { if (OB_DDL_ALTER_COLUMN == alter_column_schema->alter_type_) { //drop default or set default //mysql seem like set default now will couse a parser error; if (is_first_timestamp && alter_column_schema->is_drop_default_) { //new_column_schema is orig_column_schema; //if default value is now(), on update current timestamp is false; if (!new_column_schema.is_nullable() && !IS_DEFAULT_NOW_OBJ(new_column_schema.get_cur_default_value())) { new_column_schema.set_on_update_current_timestamp(true); } else { //do nothing } } } else { bool is_set_default = alter_column_schema->is_set_default_; bool is_set_null = alter_column_schema->is_set_nullable_; if (is_first_timestamp && !is_set_null && !is_set_default && !new_column_schema.is_on_update_current_timestamp()) { new_column_schema.set_nullable(false); new_column_schema.get_cur_default_value().set_ext( ObActionFlag::OP_DEFAULT_NOW_FLAG); new_column_schema.set_on_update_current_timestamp(true); } else if (!is_set_null) { new_column_schema.set_nullable(false); if (!is_set_default) { if (alter_column_schema->is_no_zero_date_) { ret = OB_INVALID_DEFAULT; LOG_USER_ERROR(OB_INVALID_DEFAULT, alter_column_schema->get_column_name_str().length(), alter_column_schema->get_column_name_str().ptr()); } else { int64_t zero_date = ObTimeConverter::ZERO_DATETIME; ObTimeConverter::round_datetime(alter_column_schema->get_data_scale(), zero_date); new_column_schema.get_cur_default_value().set_timestamp(zero_date); } } else if (new_column_schema.get_cur_default_value().is_null()) { ret = OB_INVALID_DEFAULT; LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(), new_column_schema.get_column_name_str().ptr()); } } else { new_column_schema.set_nullable(true); if (!is_set_default) { new_column_schema.get_cur_default_value().set_null(); } } if(OB_SUCC(ret) && OB_DDL_ADD_COLUMN == alter_column_schema->alter_type_) { ObObj cur_default_value = new_column_schema.get_cur_default_value(); if (IS_DEFAULT_NOW_OBJ(cur_default_value) || alter_column_schema->is_default_expr_v2_column()) { if (OB_FAIL(ObDDLResolver::calc_default_value(*alter_column_schema, cur_default_value, tz_info_wrap, nls_formats, allocator))) { LOG_WARN("fail to calc default now expr", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(alter_column_schema->set_orig_default_value(cur_default_value))) { OB_LOG(WARN, "fail to set orig default value", K(cur_default_value), K(ret)); } } } } } } return ret; } int ObDDLService::deal_default_value_padding(ObColumnSchemaV2 &column_schema, ObIAllocator &allocator) { int ret = OB_SUCCESS; ObString str; if (column_schema.get_orig_default_value().is_null() || column_schema.get_data_type() != ObCharType || column_schema.get_collation_type() != CS_TYPE_BINARY) { //nothing to do; } else if (OB_FAIL(column_schema.get_orig_default_value().get_string(str))) { LOG_WARN("fail to get string", K(ret)); } else { int64_t strlen = ObCharset::strlen_char(column_schema.get_collation_type(), str.ptr(), str.length()); if (strlen >= column_schema.get_data_length()) { //nothing to do //check_default_value_length will check length; } else { char* ptr = NULL; int64_t real_size = str.length() + column_schema.get_data_length() - strlen; if (NULL == (ptr = static_cast(allocator.alloc(real_size)))) { ret = OB_ALLOCATE_MEMORY_FAILED; } else { MEMCPY(ptr, str.ptr(), str.length()); memset(ptr + str.length(), OB_PADDING_BINARY, column_schema.get_data_length() - strlen); ObString new_string(real_size, ptr); ObObj new_default_value; new_default_value.set_binary(new_string); if (OB_FAIL(column_schema.set_orig_default_value(new_default_value))) { LOG_WARN("fail to set orig default value", K(ret), K(new_default_value)); } } } } return ret; } int ObDDLService::pre_check_orig_column_schema( const AlterColumnSchema &alter_column_schema, const ObTableSchema &origin_table_schema, common::hash::ObHashSet &update_column_name_set) { int ret = OB_SUCCESS; const ObString &orig_column_name = alter_column_schema.get_origin_column_name(); const ObColumnSchemaV2 *orig_column_schema = origin_table_schema.get_column_schema(orig_column_name); const ObColumnSchemaV2 *column_schema_from_old_table_schema = origin_table_schema.get_column_schema(orig_column_name); ObColumnNameHashWrapper orig_column_key(orig_column_name); if (NULL == column_schema_from_old_table_schema) { ret = OB_ERR_BAD_FIELD_ERROR; LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(), origin_table_schema.get_table_name_str().length(), origin_table_schema.get_table_name_str().ptr()); LOG_WARN("failed to find old column schema", K(ret), K(orig_column_name)); } else if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) { //column that has been modified, can't not modify again ret = OB_ERR_BAD_FIELD_ERROR; LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(), origin_table_schema.get_table_name_str().length(), origin_table_schema.get_table_name_str().ptr()); LOG_WARN("column that has been altered, can't not update again", K(ret)); } else if (OB_FAIL(check_generated_column_modify_authority(*orig_column_schema, alter_column_schema))) { LOG_WARN("check generated column modify authority", K(ret), KPC(orig_column_schema), K(alter_column_schema)); } return ret; } int ObDDLService::check_generated_column_modify_authority( const ObColumnSchemaV2 &old_column_schema, const AlterColumnSchema &alter_column_schema) { int ret = OB_SUCCESS; if (old_column_schema.is_generated_column() && alter_column_schema.is_generated_column()) { if ((old_column_schema.is_virtual_generated_column() && alter_column_schema.is_virtual_generated_column()) || (old_column_schema.is_stored_generated_column() && alter_column_schema.is_stored_generated_column())) { ObString old_def; ObString alter_def; if (OB_FAIL(old_column_schema.get_cur_default_value().get_string(old_def))) { LOG_WARN("get old generated column definition failed", K(ret), K(old_column_schema)); } else if (OB_FAIL(alter_column_schema.get_cur_default_value().get_string(alter_def))) { LOG_WARN("get new generated column definition failed", K(ret), K(alter_column_schema)); } else if (!ObCharset::case_insensitive_equal(old_def, alter_def)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "Modify generated column definition"); LOG_WARN("generated column schema definition changed", K(ret), K(old_column_schema), K(alter_column_schema)); } } else { ret = OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN; LOG_USER_ERROR(OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN, "Changing the STORED status"); } } else if (old_column_schema.is_generated_column() || alter_column_schema.is_generated_column()) { ret = OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN; LOG_USER_ERROR(OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN, "Changing the STORED status"); } return ret; } int ObDDLService::update_generated_column_schema( const AlterColumnSchema &alter_column_schema, const ObColumnSchemaV2 &orig_column_schema, const ObTableSchema &origin_table_schema, const ObTimeZoneInfoWrap &tz_info_wrap, ObTableSchema &new_table_schema, ObDDLOperator *ddl_operator, common::ObMySQLTransaction *trans) { int ret = OB_SUCCESS; ObTableSchema::const_column_iterator col_iter = new_table_schema.column_begin(); ObTableSchema::const_column_iterator col_end = new_table_schema.column_end(); ObColumnSchemaV2 *column = nullptr; if (OB_ISNULL(tz_info_wrap.get_time_zone_info())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid tz info", K(ret)); } for (; OB_SUCC(ret) && col_iter != col_end; col_iter++) { if (OB_ISNULL(column = *col_iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*col_iter is NULL", K(ret)); } else if (column->has_cascaded_column_id(orig_column_schema.get_column_id())) { ObColumnSchemaV2 new_generated_column_schema = *column; if (OB_FAIL(new_generated_column_schema.get_err_ret())) { LOG_WARN("failed to copy new gen column", K(ret)); } else if (OB_FAIL(modify_generated_column_default_value(new_generated_column_schema, const_cast(orig_column_schema.get_column_name_str()), alter_column_schema.get_column_name_str(), new_table_schema, *tz_info_wrap.get_time_zone_info()))) { LOG_WARN("modify generated column value failed", K(ret)); } else if (OB_FAIL(new_table_schema.alter_column(new_generated_column_schema, ObTableSchema::CHECK_MODE_ONLINE))) { // alter_column must not change column_array_ because we are still looping the column_array_ LOG_WARN("failed to change column", K(ret)); } else if (OB_NOT_NULL(ddl_operator) && OB_NOT_NULL(trans)) { if (OB_FAIL(ddl_operator->update_single_column(*trans, origin_table_schema, new_table_schema, new_generated_column_schema))) { LOG_WARN("generated column failed to alter column", K(ret)); } } } } return ret; } int ObDDLService::modify_generated_column_default_value(ObColumnSchemaV2 &generated_column, common::ObString &column_name, const ObString &new_column_name, const ObTableSchema &table_schema, const ObTimeZoneInfo &tz_info) { int ret = OB_SUCCESS; if (generated_column.is_generated_column()) { ObString col_def; ObArenaAllocator allocator(ObModIds::OB_SCHEMA); ObRawExprFactory expr_factory(allocator); SMART_VAR(ObSQLSessionInfo, default_session) { uint64_t tenant_id = table_schema.get_tenant_id(); const ObTenantSchema *tenant_schema = NULL; ObSchemaGetterGuard schema_guard; ObRawExpr *expr = NULL; lib::Worker::CompatMode compat_mode; if (OB_FAIL(default_session.init(0, 0, &allocator))) { LOG_WARN("init empty session failed", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("get tenant_schema failed", K(ret)); } else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) { LOG_WARN("init tenant failed", K(ret)); } else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) { LOG_WARN("session load system variable failed", K(ret)); } else if (OB_FAIL(default_session.load_default_configs_in_pc())) { LOG_WARN("session load default configs failed", K(ret)); } else if (OB_FAIL(generated_column.get_cur_default_value().get_string(col_def))) { LOG_WARN("get cur default value failed", K(ret)); } else if (OB_FAIL(ObRawExprUtils::build_generated_column_expr(NULL, col_def, expr_factory, default_session, table_schema, expr))) { LOG_WARN("build generated column expr failed", K(ret)); } else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(table_schema.get_tenant_id(), table_schema.get_table_id(), compat_mode))) { LOG_WARN("failed to get table compat mode", K(ret)); } if (OB_SUCC(ret)) { ObRawExprModifyColumnName modifyColumnName(new_column_name, column_name, compat_mode); if (OB_FAIL(modifyColumnName.modifyColumnName(*expr))) { LOG_WARN("modifyColumnName modify column name failed", K(ret)); } else { SMART_VAR(char[OB_MAX_DEFAULT_VALUE_LENGTH], expr_str_buf) { MEMSET(expr_str_buf, 0, sizeof(expr_str_buf)); ObString expr_def; int64_t pos = 0; ObObj default_value; ObRawExprPrinter expr_printer(expr_str_buf, OB_MAX_DEFAULT_VALUE_LENGTH, &pos, &tz_info); if (OB_FAIL(expr_printer.do_print(expr, T_NONE_SCOPE, true))) { LOG_WARN("print expr definition failed", K(ret)); } else if (FALSE_IT(expr_def.assign_ptr(expr_str_buf, static_cast(pos)))) { } else if (FALSE_IT(default_value.set_varchar(expr_def))) { } else if (OB_FAIL(generated_column.set_cur_default_value(default_value))) { LOG_WARN("set cur default value failed", K(ret)); } else if (OB_FAIL(generated_column.set_orig_default_value(default_value))) { LOG_WARN("set original default value failed", K(ret)); } } } } } } return ret; } // don't allow alter materialized view related columns // this rule will be change in the next implemation. int ObDDLService::validate_update_column_for_materialized_view( const ObTableSchema &orig_table_schema, const ObColumnSchemaV2 &orig_column_schema) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; ObArray mv_ids; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_mv_ids(tenant_id, mv_ids))) { LOG_WARN("fail to get all mv ids", K(ret), "vesion", orig_table_schema.get_schema_version()); } else { uint64_t mv_id = OB_INVALID_ID; const ObTableSchema *mv = NULL; for (int64_t i = 0; i < mv_ids.count() && OB_SUCC(ret); i++) { mv_id = mv_ids.at(i); if (OB_FAIL(schema_guard.get_table_schema(tenant_id, mv_id, mv))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(mv_id)); } else if (mv && mv->has_table(orig_table_schema.get_table_id()) && mv->get_column_schema(orig_table_schema.get_table_id(), orig_column_schema.get_column_id())) { ret = OB_NOT_SUPPORTED; LOG_WARN("don't allow update column if it's in materialized view", K(ret), K(*mv), K(orig_column_schema)); } } } return ret; } int ObDDLService::modify_part_func_expr( const ObString &orig_column_name, const ObString &alter_column_name, share::schema::ObTableSchema &table_schema, const common::ObTimeZoneInfo &tz_info, common::ObIAllocator &allocator) { int ret = OB_SUCCESS; if (!table_schema.is_partitioned_table()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid part level", K(ret), K(table_schema.get_part_level())); } else if (PARTITION_LEVEL_ONE == table_schema.get_part_level()) { if (OB_FAIL(modify_func_expr_column_name( orig_column_name, alter_column_name, table_schema, tz_info, allocator, false))) { LOG_WARN("fail to modify func expr column name", K(ret), K(orig_column_name), K(alter_column_name), K(table_schema)); } } else if (PARTITION_LEVEL_TWO == table_schema.get_part_level()) { if (OB_FAIL(modify_func_expr_column_name( orig_column_name, alter_column_name, table_schema, tz_info, allocator, false))) { LOG_WARN("fail to modify func expr column name for partition level one", K(ret), K(orig_column_name), K(alter_column_name), K(table_schema)); } else if (OB_FAIL(modify_func_expr_column_name( orig_column_name, alter_column_name, table_schema, tz_info, allocator, true))) { LOG_WARN("fail to modify func expr column name for partition level two", K(ret), K(orig_column_name), K(alter_column_name), K(table_schema)); } } return ret; } int ObDDLService::modify_func_expr_column_name( const ObString &orig_column_name, const ObString &alter_column_name, ObTableSchema &table_schema, const ObTimeZoneInfo &tz_info, common::ObIAllocator &allocator, bool is_sub_part) { int ret = OB_SUCCESS; ObRawExprFactory expr_factory(allocator); uint64_t tenant_id = table_schema.get_tenant_id(); const ObTenantSchema *tenant_schema = NULL; ObSchemaGetterGuard schema_guard; ObRawExpr *expr = NULL; ObArray columns; const ObColumnSchemaV2 *col_schema = NULL; ObPartitionOption &part_option = table_schema.get_part_option(); ObPartitionOption &sub_part_option = table_schema.get_sub_part_option(); ObString orig_part_expr; ObArray expr_strs; SMART_VAR(ObSQLSessionInfo, default_session) { if (!is_sub_part) { orig_part_expr = part_option.get_part_func_expr_str(); } else { orig_part_expr = sub_part_option.get_part_func_expr_str(); } if (OB_FAIL(ret)) { } else if (OB_FAIL(default_session.init(0, 0, &allocator))) { LOG_WARN("init empty session failed", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("get tenant_schema failed", K(ret)); } else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) { LOG_WARN("init tenant failed", K(ret)); } else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) { LOG_WARN("session load system variable failed", K(ret)); } else if (OB_FAIL(default_session.load_default_configs_in_pc())) { LOG_WARN("session load default configs failed", K(ret)); } else if (orig_part_expr.empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("partition func expr is empty", K(ret)); } else if (OB_FAIL(split_on(orig_part_expr, ',', expr_strs))) { LOG_WARN("fail to split func expr", K(ret), K(orig_part_expr)); } else { char *new_part_func_expr_buf = NULL; int64_t outer_pos = 0; lib::Worker::CompatMode compat_mode; if (OB_ISNULL(new_part_func_expr_buf = static_cast(allocator.alloc(OB_MAX_SQL_LENGTH)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail to alloc new_part_func_expr", K(ret)); } else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(table_schema.get_tenant_id(), table_schema.get_table_id(), compat_mode))) { LOG_WARN("failed to get table compat mode", K(ret)); } else { ObRawExprModifyColumnName modifyColumnName( alter_column_name, orig_column_name, compat_mode); for (int64_t i = 0; OB_SUCC(ret) && i < expr_strs.count(); ++i) { expr = NULL; columns.reset(); if (OB_FAIL(ObRawExprUtils::build_generated_column_expr( expr_strs.at(i), expr_factory, default_session, expr, columns, &table_schema, false /* allow_sequence */, NULL, NULL, sql::ObResolverUtils::DISABLE_CHECK, false))) { LOG_WARN("build generated column expr failed", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i set_ref_id(table_schema.get_table_id(), col_schema->get_column_id()); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(modifyColumnName.modifyColumnName(*expr))) { LOG_WARN("modifyColumnName modify column name failed", K(ret)); } else { SMART_VAR(char[OB_MAX_SQL_LENGTH], expr_str_buf) { MEMSET(expr_str_buf, 0, sizeof(expr_str_buf)); int64_t inner_pos = 0; ObRawExprPrinter expr_printer(expr_str_buf, OB_MAX_SQL_LENGTH, &inner_pos, &tz_info); if (OB_FAIL(expr_printer.do_print(expr, T_NONE_SCOPE, true))) { LOG_WARN("print expr definition failed", K(ret)); } else if (0 == i && OB_FAIL(databuff_printf(new_part_func_expr_buf, OB_MAX_SQL_LENGTH, outer_pos, "%.*s", static_cast(inner_pos), expr_str_buf))) { LOG_WARN("fail to print expr_str_buf", K(ret), K(i), K(expr_str_buf)); } else if (0 != i && OB_FAIL(databuff_printf(new_part_func_expr_buf, OB_MAX_SQL_LENGTH, outer_pos, ", %.*s", static_cast(inner_pos), expr_str_buf))) { LOG_WARN("fail to print expr_str_buf", K(ret), K(i), K(expr_str_buf)); } } } } if (OB_SUCC(ret)) { ObString new_part_func_expr_str; if (FALSE_IT(new_part_func_expr_str.assign_ptr( new_part_func_expr_buf, static_cast(outer_pos)))) { } else if (!is_sub_part && OB_FAIL(part_option.set_part_expr(new_part_func_expr_str))) { LOG_WARN("set part expr failed", K(ret)); } else if (is_sub_part && OB_FAIL(sub_part_option.set_part_expr(new_part_func_expr_str))) { LOG_WARN("set sub part expr failed", K(ret)); } } } } } return ret; } int ObDDLService::modify_part_func_expr_for_global_index( const share::schema::ObColumnSchemaV2 &orig_column, const share::schema::ObColumnSchemaV2 &alter_column, share::schema::ObTableSchema &new_table_schema, const common::ObTimeZoneInfo &tz_info, common::ObIAllocator &allocator, ObDDLOperator *ddl_operator, common::ObMySQLTransaction *trans, ObIArray *global_idx_schema_array/*=NULL*/) { int ret = OB_SUCCESS; const bool update_inner_table = nullptr != ddl_operator && nullptr != trans; ObSchemaGetterGuard schema_guard; const uint64_t tenant_id = new_table_schema.get_tenant_id(); ObSchemaService *schema_service = schema_service_->get_schema_service(); ObSEArray simple_index_infos; if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service is NULL", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } else { const ObTableSchema *index_schema = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { if (OB_FAIL(schema_guard.get_table_schema( tenant_id, simple_index_infos.at(i).table_id_, index_schema))) { LOG_WARN("get table_schema failed", K(ret), K(tenant_id), K(simple_index_infos.at(i).table_id_)); } else if (OB_ISNULL(index_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index schema should not be null", K(ret)); } else if (index_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("index table is in recyclebin", K(ret)); } else if (index_schema->is_global_index_table() && index_schema->is_partitioned_table()) { const ObColumnSchemaV2 *origin_column_schema = index_schema->get_column_schema(orig_column.get_column_id()); if (OB_ISNULL(origin_column_schema)) { // skip, this column is not in global index } else if (origin_column_schema->is_tbl_part_key_column()) { ObTableSchema new_index_schema; if (OB_FAIL(new_index_schema.assign(*index_schema))) { LOG_WARN("assign index_schema failed", K(ret)); } else if (OB_FAIL(modify_part_func_expr(origin_column_schema->get_column_name_str(), alter_column.get_column_name_str(), new_index_schema, tz_info, allocator))) { LOG_WARN("modify part func expr failed", K(ret)); } else if (update_inner_table && OB_FAIL(ddl_operator->update_partition_option(*trans, new_index_schema))) { LOG_WARN("update partition option failed", K(ret), K(new_index_schema), K(*index_schema)); } else if (nullptr != global_idx_schema_array && OB_FAIL(global_idx_schema_array->push_back(new_index_schema))) { LOG_WARN("fail to push_back to global_idx_schema_array", K(ret), K(new_index_schema)); } } } else { // skip } } // end of for } // end of else return ret; } int ObDDLService::modify_constraint_check_expr( const share::schema::ObColumnSchemaV2 &orig_column, const share::schema::ObColumnSchemaV2 &alter_column, share::schema::ObTableSchema &table_schema, const common::ObTimeZoneInfo &tz_info, common::ObIAllocator &allocator, ObDDLOperator *ddl_operator, common::ObMySQLTransaction *trans) { int ret = OB_SUCCESS; const bool update_inner_table = nullptr != ddl_operator && nullptr != trans; bool need_modify_check_expr = false; ObString new_check_expr_str; ObTableSchema::const_constraint_iterator iter = table_schema.constraint_begin(); for (; OB_SUCC(ret) && iter != table_schema.constraint_end(); ++iter) { need_modify_check_expr = false; if ((*iter)->get_check_expr_str().empty()) { continue; } if (OB_FAIL(rebuild_constraint_check_expr(orig_column, alter_column, **iter, table_schema, tz_info, allocator, new_check_expr_str, need_modify_check_expr))) { LOG_WARN("fail to gen constraint check expr", K(ret)); } else if (need_modify_check_expr) { (*iter)->set_check_expr(new_check_expr_str); (*iter)->set_is_modify_check_expr(true); if (update_inner_table && OB_FAIL(ddl_operator->update_check_constraint_state(*trans, table_schema, **iter))) { LOG_WARN("update check expr constraint failed", K(ret)); } } } return ret; } // Oracle mode, to refill column id for not null constraint that is referenced to newly-added column. // Case: alter table table_name add column_name not null. int ObDDLService::refill_columns_id_for_not_null_constraint( const share::schema::ObTableSchema &alter_table_schema, const share::schema::ObColumnSchemaV2 &alter_column_schema) { int ret = OB_SUCCESS; ObTableSchema::constraint_iterator target_cst_iter = nullptr; if (alter_column_schema.has_not_null_constraint()) { ObTableSchema::constraint_iterator cst_iter = alter_table_schema.constraint_begin_for_non_const_iter(); for (; NULL == target_cst_iter && OB_SUCC(ret) && cst_iter != alter_table_schema.constraint_end_for_non_const_iter(); cst_iter++) { if (OB_ISNULL(*cst_iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected", K(ret)); } else if (CONSTRAINT_TYPE_NOT_NULL == (*cst_iter)->get_constraint_type()) { ObString cst_col_name; if (OB_FAIL((*cst_iter)->get_not_null_column_name(cst_col_name))) { LOG_WARN("get not null cst column name failed", K(ret)); } else if (alter_column_schema.get_column_name_str().length() == cst_col_name.length() && 0 == alter_column_schema.get_column_name_str().compare(cst_col_name)) { target_cst_iter = cst_iter; } LOG_DEBUG("alter table add column not null", K((*cst_iter)->get_check_expr_str()), K(cst_col_name), K(alter_column_schema.get_column_name_str())); } } ObSEArray column_id_array; if (OB_FAIL(ret)) { } else if (OB_ISNULL(target_cst_iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("constraint not found for added not null column", K(ret), K(alter_column_schema)); } else if (OB_FAIL(column_id_array.push_back(alter_column_schema.get_column_id()))) { LOG_WARN("push back failed", K(ret)); } else if (OB_FAIL((*target_cst_iter)->assign_column_ids(column_id_array))) { LOG_WARN("assign column ids failed", K(ret)); } } return ret; } // column_id_array is invalid if newly-added check constraint is referenced to // newly-added column, due to invalid column id in resolver phase. // Case: alter table table_name add column_name constraint check(column_name < 10) check(column_name > 0). int ObDDLService::refill_columns_id_for_check_constraint( const share::schema::ObTableSchema &orig_table_schema, const share::schema::ObTableSchema &alter_table_schema, const share::schema::ObColumnSchemaV2 &alter_column_schema, const bool is_oracle_mode, common::ObIAllocator &allocator) { int ret = OB_SUCCESS; bool need_refill_columns_id = false; ObTableSchema::const_constraint_iterator check_cst_iter = nullptr; ObTableSchema::constraint_iterator target_cst_iter = nullptr; // check whether there are some unexpected case firstly. for (check_cst_iter = alter_table_schema.constraint_begin(); OB_SUCC(ret) && check_cst_iter != alter_table_schema.constraint_end(); check_cst_iter++) { if (OB_ISNULL(*check_cst_iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected", K(ret)); } else if (CONSTRAINT_TYPE_CHECK == (*check_cst_iter)->get_constraint_type()) { if (is_oracle_mode) { // set error code here to avoid forgetting refill columns id // under oracle mode if supported in the future. ret = OB_NOT_SUPPORTED; LOG_WARN("add column and alter check constraint in single stmt is not supported", K(ret), K(alter_table_schema)); } else if (1 == (*check_cst_iter)->get_column_cnt() && nullptr == orig_table_schema.get_column_schema(*((*check_cst_iter)->cst_col_begin()))) { // Currently, newly-added check constraint is referenced to newly-added column under Mysql mode. need_refill_columns_id = true; } } } // refill columns id array. if (OB_SUCC(ret) && need_refill_columns_id) { ObRawExprFactory expr_factory(allocator); uint64_t tenant_id = alter_column_schema.get_tenant_id(); const ObTenantSchema *tenant_schema = nullptr; ObSchemaGetterGuard schema_guard; SMART_VAR(ObSQLSessionInfo, default_session) { if (OB_FAIL(default_session.init(0, 0, &allocator))) { LOG_WARN("init empty session failed", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("get tenant_schema failed", K(ret), K(tenant_id)); } else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) { LOG_WARN("init tenant failed", K(ret), K(tenant_id)); } else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) { LOG_WARN("session load system variable failed", K(ret)); } else if (OB_FAIL(default_session.load_default_configs_in_pc())) { LOG_WARN("session load default configs failed", K(ret)); } else { for (target_cst_iter = alter_table_schema.constraint_begin_for_non_const_iter(); OB_SUCC(ret) && target_cst_iter != alter_table_schema.constraint_end_for_non_const_iter(); target_cst_iter++) { if (OB_ISNULL(*target_cst_iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected", K(ret)); } else if (CONSTRAINT_TYPE_CHECK == (*target_cst_iter)->get_constraint_type() && 1 == (*target_cst_iter)->get_column_cnt() && nullptr == orig_table_schema.get_column_schema(*((*target_cst_iter)->cst_col_begin()))) { ObString check_expr = (*target_cst_iter)->get_check_expr_str(); ObArray columns; ObSEArray column_id_array; ObRawExpr *expr = nullptr; const ParseNode *node = nullptr; if (check_expr.empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("check expr is empty", K(ret)); } else if (OB_FAIL(ObRawExprUtils::parse_bool_expr_node_from_str(check_expr, expr_factory.get_allocator(), node))) { LOG_WARN("parse expr node from string failed", K(ret)); } else if (OB_ISNULL(node)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("node is null", K(ret)); } else if (OB_FAIL(ObRawExprUtils::build_check_constraint_expr(expr_factory, default_session, *node, expr, columns))) { LOG_WARN("build generated column expr failed", K(ret), K(check_expr)); } else if (1 != columns.count()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected column count of the new-built check constraint", K(ret), K(check_expr)); } else if (0 != alter_column_schema.get_column_name_str().case_compare(columns.at(0).col_name_)) { // may be newly-added column without check constraint, do nothing. } else if (OB_FAIL(column_id_array.push_back(alter_column_schema.get_column_id()))) { LOG_WARN("push back failed", K(ret)); } else if (OB_FAIL((*target_cst_iter)->assign_column_ids(column_id_array))) { LOG_WARN("assign column ids failed", K(ret)); } } } } } } return ret; } int ObDDLService::rebuild_constraint_check_expr( const share::schema::ObColumnSchemaV2 &orig_column, const share::schema::ObColumnSchemaV2 &alter_column, const share::schema::ObConstraint &cst, share::schema::ObTableSchema &table_schema, const common::ObTimeZoneInfo &tz_info, common::ObIAllocator &allocator, ObString &new_check_expr_str, bool &need_modify_check_expr) { int ret = OB_SUCCESS; ObRawExprFactory expr_factory(allocator); uint64_t tenant_id = orig_column.get_tenant_id(); const ObTenantSchema *tenant_schema = NULL; ObSchemaGetterGuard schema_guard; ObRawExpr *expr = NULL; const ParseNode *node = NULL; ObArray columns; const ObColumnSchemaV2 *col_schema = NULL; ObString orig_check_expr = cst.get_check_expr_str(); SMART_VAR(ObSQLSessionInfo, default_session) { if (OB_FAIL(default_session.init(0, 0, &allocator))) { LOG_WARN("init empty session failed", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("get tenant_schema failed", K(ret)); } else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) { LOG_WARN("init tenant failed", K(ret)); } else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) { LOG_WARN("session load system variable failed", K(ret)); } else if (OB_FAIL(default_session.load_default_configs_in_pc())) { LOG_WARN("session load default configs failed", K(ret)); } else if (orig_check_expr.empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("check expr is empty", K(ret)); } else { char *new_check_expr_buf = NULL; int64_t outer_pos = 0; if (OB_ISNULL(new_check_expr_buf = static_cast(allocator.alloc(OB_MAX_SQL_LENGTH)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail to alloc new_check_expr_buf", K(ret)); } else { char *new_check_expr_buf = NULL; int64_t outer_pos = 0; lib::Worker::CompatMode compat_mode; if (OB_ISNULL(new_check_expr_buf = static_cast(allocator.alloc(OB_MAX_SQL_LENGTH)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail to alloc new_check_expr_buf", K(ret)); } else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(table_schema.get_tenant_id(), table_schema.get_table_id(), compat_mode))) { LOG_WARN("failed to get table compat mode", K(ret)); } else { ObRawExprModifyColumnName modifyColumnName( alter_column.get_column_name_str(), orig_column.get_column_name_str(), compat_mode); if (OB_FAIL(ObRawExprUtils::parse_bool_expr_node_from_str(orig_check_expr, expr_factory.get_allocator(), node))) { LOG_WARN("parse expr node from string failed", K(ret)); } else if (OB_ISNULL(node)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("node is null", K(ret)); } else if (OB_FAIL(ObRawExprUtils::build_check_constraint_expr(expr_factory, default_session, *node, expr, columns))) { LOG_WARN("build generated column expr failed", K(ret), K(orig_check_expr)); } for (int64_t i = 0; OB_SUCC(ret) && i set_ref_id(table_schema.get_table_id(), col_schema->get_column_id()); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(modifyColumnName.modifyColumnName(*expr))) { LOG_WARN("modifyColumnName modify column name failed", K(ret)); } else { SMART_VAR(char[OB_MAX_SQL_LENGTH], expr_str_buf) { MEMSET(expr_str_buf, 0, sizeof(expr_str_buf)); int64_t inner_pos = 0; ObRawExprPrinter expr_printer(expr_str_buf, OB_MAX_SQL_LENGTH, &inner_pos, &tz_info); if (OB_FAIL(expr_printer.do_print(expr, T_NONE_SCOPE, true))) { LOG_WARN("print expr definition failed", K(ret)); } else if (OB_FAIL(databuff_printf(new_check_expr_buf, OB_MAX_SQL_LENGTH, outer_pos, "%.*s", static_cast(inner_pos), expr_str_buf))) { LOG_WARN("fail to print expr_str_buf", K(ret), K(expr_str_buf)); } } } if (OB_SUCC(ret)) { new_check_expr_str.assign_ptr(new_check_expr_buf, static_cast(outer_pos)); } } } } } return ret; } int ObDDLService::check_can_alter_column_type( const share::schema::ObColumnSchemaV2 &src_column, const share::schema::ObColumnSchemaV2 &dst_column, const share::schema::ObTableSchema &table_schema) { int ret = OB_SUCCESS; bool is_change_column_type = false; bool is_in_index = false; if (OB_FAIL(check_is_change_column_type(src_column, dst_column, is_change_column_type))) { LOG_WARN("fail to check is change column type", K(ret), K(src_column), K(dst_column)); } else if (is_change_column_type) { if (OB_FAIL(check_column_in_index(src_column.get_column_id(), table_schema, is_in_index))) { LOG_WARN("fail to check column is in index table", K(ret)); } else if (is_in_index) { ret = OB_NOT_SUPPORTED; LOG_WARN("cannot modify column in index table", K(ret)); } } return ret; } int ObDDLService::check_is_change_column_type( const share::schema::ObColumnSchemaV2 &src_column, const share::schema::ObColumnSchemaV2 &dst_column, bool &is_change_column_type) { int ret = OB_SUCCESS; is_change_column_type = src_column.get_data_type() != dst_column.get_data_type(); LOG_INFO("check is change column type", K(is_change_column_type), K(src_column), K(dst_column)); return ret; } int ObDDLService::check_column_in_index( const uint64_t column_id, const share::schema::ObTableSchema &table_schema, bool &is_in_index) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; ObArray column_ids; is_in_index = false; if (OB_UNLIKELY(!table_schema.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(table_schema)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret)); } else if (OB_FAIL(table_schema.get_rowkey_column_ids(column_ids))) { LOG_WARN("fail to get rowkey column ids", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < column_ids.count() && !is_in_index; ++i) { if (column_id == column_ids.at(i).col_id_) { is_in_index = true; LOG_WARN("column in data table rowkey columns", K(column_id)); } } if (OB_SUCC(ret) && !is_in_index) { ObSEArray simple_index_infos; if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count() && !is_in_index; ++i) { const ObTableSchema *index_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), "table id", simple_index_infos.at(i).table_id_); } else if (OB_ISNULL(index_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("index table schema must not be NULL", K(ret)); } else { column_ids.reuse(); if (OB_FAIL(index_table_schema->get_column_ids(column_ids))) { LOG_WARN("fail to get column ids", K(ret)); } for (int64_t j = 0; OB_SUCC(ret) && !is_in_index && j < column_ids.count(); ++j) { if (column_id == column_ids.at(j).col_id_) { is_in_index = true; } } } } } if (OB_SUCC(ret) && !is_in_index) { column_ids.reuse(); if (OB_FAIL(table_schema.get_column_ids(column_ids))) { LOG_WARN("fail to get column ids", K(ret)); } else { for (int64_t j = 0; OB_SUCC(ret) && !is_in_index && j < column_ids.count(); ++j) { const ObColumnSchemaV2 *column_schema = NULL; if (OB_ISNULL(column_schema = table_schema.get_column_schema(column_ids.at(j).col_id_))) { ret = OB_SCHEMA_ERROR; LOG_WARN("column schema must not be NULL", K(ret), K(column_ids.at(j))); } else if (column_schema->is_generated_column()) { ObArray ref_column_ids; if (OB_FAIL(column_schema->get_cascaded_column_ids(ref_column_ids))) { LOG_WARN("fail to get cascade column ids", K(ret)); } else { is_in_index = has_exist_in_array(ref_column_ids, column_id); } } } } } } return ret; } int ObDDLService::fill_new_column_attributes( const AlterColumnSchema &alter_column_schema, ObColumnSchemaV2 &new_column_schema) { int ret = OB_SUCCESS; int64_t new_schema_version = OB_INVALID_VERSION; const uint64_t tenant_id = new_column_schema.get_tenant_id(); if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id)); } else { new_column_schema.set_column_name(alter_column_schema.get_column_name_str()); new_column_schema.set_charset_type(alter_column_schema.get_charset_type()); new_column_schema.set_collation_type(alter_column_schema.get_collation_type()); new_column_schema.set_data_type(alter_column_schema.get_data_type()); new_column_schema.set_data_length(alter_column_schema.get_data_length()); new_column_schema.set_data_precision(alter_column_schema.get_data_precision()); new_column_schema.set_data_scale(alter_column_schema.get_data_scale()); if (!is_oracle_mode() || alter_column_schema.is_set_default_) { new_column_schema.set_cur_default_value(alter_column_schema.get_cur_default_value()); } new_column_schema.set_is_hidden(alter_column_schema.is_hidden()); new_column_schema.set_nullable(alter_column_schema.is_nullable()); new_column_schema.set_autoincrement(alter_column_schema.is_autoincrement()); new_column_schema.set_column_flags(alter_column_schema.get_column_flags()); new_column_schema.set_comment(alter_column_schema.get_comment_str()); new_column_schema.set_schema_version(new_schema_version); new_column_schema.set_on_update_current_timestamp( alter_column_schema.is_on_update_current_timestamp()); new_column_schema.set_extended_type_info(alter_column_schema.get_extended_type_info()); } return ret; } int ObDDLService::check_modify_column_when_upgrade( const share::schema::ObColumnSchemaV2 &new_column, const share::schema::ObColumnSchemaV2 &orig_column) { int ret = OB_SUCCESS; if (obrpc::OB_UPGRADE_STAGE_DBUPGRADE != GCTX.get_upgrade_stage()) { // do nothing } else { ObColumnSchemaV2 tmp_column = new_column; tmp_column.set_schema_version(orig_column.get_schema_version()); tmp_column.set_data_length(orig_column.get_data_length()); tmp_column.set_data_precision(orig_column.get_data_precision()); tmp_column.set_data_scale(orig_column.get_data_scale()); if (OB_FAIL(tmp_column.get_assign_ret())) { LOG_WARN("assign failed", K(ret), K(new_column)); } else if (tmp_column != orig_column) { ret = OB_OP_NOT_ALLOW; LOG_WARN("can only modify column's length", K(ret), K(new_column), K(orig_column)); } else if (new_column.get_data_length() < orig_column.get_data_length() || new_column.get_data_precision() < orig_column.get_data_precision() || new_column.get_data_scale() < orig_column.get_data_scale()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("can only increase column's length", K(ret), K(new_column), K(orig_column)); } } return ret; } int ObDDLService::check_new_column_for_index( ObIArray &idx_schemas, const ObColumnSchemaV2 &new_column_schema) { int ret = OB_SUCCESS; int idx_cnt = idx_schemas.count(); ObTableSchema *index_table_schema = NULL; ObColumnSchemaV2 copy_index_column_schema; for (int64_t i = 0; OB_SUCC(ret) && i < idx_cnt; ++i) { index_table_schema = &idx_schemas.at(i); if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; RS_LOG(WARN, "table schema should not be null", K(ret)); } else { const ObColumnSchemaV2 *origin_idx_column_schema = index_table_schema->get_column_schema(new_column_schema.get_column_id()); if (NULL == origin_idx_column_schema) { RS_LOG(INFO, "index table do not contain this column", "column_name", new_column_schema.get_column_name_str(), "index_table", index_table_schema->get_table_name_str()); continue; } else if (!origin_idx_column_schema->is_rowkey_column()) { RS_LOG(INFO, "ingore not rowkey column", "column_name", new_column_schema.get_column_name_str(), "index_table", index_table_schema->get_table_name_str()); } else { copy_index_column_schema.reset(); copy_index_column_schema = new_column_schema; copy_index_column_schema.set_rowkey_position(origin_idx_column_schema->get_rowkey_position()); copy_index_column_schema.set_index_position(origin_idx_column_schema->get_index_position()); copy_index_column_schema.set_tbl_part_key_pos(origin_idx_column_schema->get_tbl_part_key_pos()); if (OB_FAIL(index_table_schema->alter_column(copy_index_column_schema, ObTableSchema::CHECK_MODE_ONLINE))) { RS_LOG(WARN, "failed to alter index column schema", K(copy_index_column_schema), K(ret)); } else if (!index_table_schema->is_valid()) { ret = OB_SCHEMA_ERROR; RS_LOG(WARN, "idx table schema is invalid!", K(ret)); } } } } return ret; } // wrapper for alter column effects // if column is in // 1. update index if modified column is in index // 2. update materialized view if modified column is in materialized view // but 2 is disabled for now int ObDDLService::alter_table_update_index_and_view_column( const ObTableSchema &new_table_schema, const ObColumnSchemaV2 &new_column_schema, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans, const ObIArray *global_idx_schema_array/*=NULL*/) { int ret = OB_SUCCESS; if (OB_FAIL(alter_table_update_aux_column(new_table_schema, new_column_schema, ddl_operator, trans, USER_INDEX, global_idx_schema_array))) { LOG_WARN("fail to update index column", K(ret), K(new_table_schema), K(new_column_schema)); } else if (OB_FAIL(alter_table_update_aux_column(new_table_schema, new_column_schema, ddl_operator, trans, AUX_VERTIAL_PARTITION_TABLE))) { LOG_WARN("fail to update aux vp column", K(ret), K(new_table_schema), K(new_column_schema)); } return ret; } // aux schema column int ObDDLService::alter_table_update_aux_column( const ObTableSchema &new_table_schema, const ObColumnSchemaV2 &new_column_schema, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans, const ObTableType table_type, const ObIArray *global_idx_schema_array/*=NULL*/) { int ret = OB_SUCCESS; //update column in aux table ObSchemaGetterGuard schema_guard; ObColumnSchemaV2 new_aux_column_schema; const bool is_index = USER_INDEX == table_type; ObSEArray aux_vp_tid_array; ObSEArray simple_index_infos; const uint64_t tenant_id = new_table_schema.get_tenant_id(); if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { RS_LOG(WARN, "get schema guard failed", K(ret)); } else if (is_index && OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } else if (!is_index && OB_FAIL(new_table_schema.get_aux_vp_tid_array(aux_vp_tid_array))) { LOG_WARN("get_aux_tid_array failed", K(ret), K(is_index)); } else { //update all aux table schema const ObTableSchema *aux_table_schema = NULL; int64_t N = is_index ? simple_index_infos.count() : aux_vp_tid_array.count(); for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { aux_table_schema = NULL; if (is_index && OB_NOT_NULL(global_idx_schema_array) && !global_idx_schema_array->empty()) { for (int64_t j = 0; OB_SUCC(ret) && j < global_idx_schema_array->count(); ++j) { if (simple_index_infos.at(i).table_id_ == global_idx_schema_array->at(j).get_table_id()) { aux_table_schema = &(global_idx_schema_array->at(j)); break; } } } uint64_t tid = is_index ? simple_index_infos.at(i).table_id_ : aux_vp_tid_array.at(i); if (OB_FAIL(ret)) { } else if (OB_ISNULL(aux_table_schema) && OB_FAIL(schema_guard.get_table_schema( tenant_id, tid, aux_table_schema))) { RS_LOG(WARN, "get_table_schema failed", K(ret), K(tenant_id), K(tid)); } else if (OB_ISNULL(aux_table_schema)) { ret = OB_ERR_UNEXPECTED; RS_LOG(WARN, "aux schema should not be null", K(ret)); } else if (aux_table_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("aux table is in recyclebin", K(ret)); } else { const ObColumnSchemaV2 *origin_column_schema = aux_table_schema->get_column_schema(new_column_schema.get_column_id()); if (NULL != origin_column_schema) { // exist such column in aux schema new_aux_column_schema = new_column_schema; new_aux_column_schema.set_table_id(aux_table_schema->get_table_id()); new_aux_column_schema.set_autoincrement(false); //save the rowkey postion and aux postion if (is_index) { new_aux_column_schema.set_rowkey_position(origin_column_schema->get_rowkey_position()); new_aux_column_schema.set_index_position(origin_column_schema->get_index_position()); new_aux_column_schema.set_tbl_part_key_pos(origin_column_schema->get_tbl_part_key_pos()); } if (!is_index) { // VP column of primary table need not update. new_aux_column_schema.set_column_flags(AUX_VP_COLUMN_FLAG); } //will only update some attribute, not include rowkey postion or aux position if (OB_FAIL(ret)) { } else if (OB_FAIL(ddl_operator.update_single_column(trans, *aux_table_schema, *aux_table_schema, new_aux_column_schema))) { RS_LOG(WARN, "schema service update aux column failed failed", "table schema", *aux_table_schema, K(ret)); } else if (OB_FAIL(ddl_operator.sync_aux_schema_version_for_history( trans, *aux_table_schema))) { RS_LOG(WARN, "fail to update aux schema version for update column"); } } } } // end of for } // end of else return ret; } int ObDDLService::alter_sequence_in_alter_column(const ObTableSchema &table_schema, ObColumnSchemaV2 &column_schema, ObMySQLTransaction &trans, ObSchemaGetterGuard &schema_guard, ObSequenceDDLArg &sequence_ddl_arg) { int ret = OB_SUCCESS; // stmt type check add for modify from identity column to normal column and it's still a identity column if (column_schema.is_identity_column() && sequence_ddl_arg.get_stmt_type() != common::OB_INVALID_ID) { ObSequenceDDLProxy ddl_operator(*schema_service_); ObSequenceSchema sequence_schema = sequence_ddl_arg.sequence_schema(); char temp_sequence_name[OB_MAX_SEQUENCE_NAME_LENGTH + 1] = { 0 }; int32_t len = snprintf(temp_sequence_name, sizeof(temp_sequence_name), "%s%lu%c%lu", "ISEQ$$_", ObSchemaUtils::get_extract_schema_id(column_schema.get_tenant_id(), column_schema.get_table_id()), '_', column_schema.get_column_id()); if (OB_UNLIKELY(len < 0)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("create sequence name fail", K(ret), K(column_schema)); } else { ObString sequence_name = ObString::make_string(temp_sequence_name); sequence_schema.set_database_id(table_schema.get_database_id()); sequence_schema.set_sequence_name(sequence_name); if (OB_FAIL(ddl_operator.alter_sequence(sequence_schema, sequence_ddl_arg.option_bitset_, trans, schema_guard, NULL, FROM_TABLE_DDL))) { LOG_WARN("alter sequence fail", K(ret)); } } } return ret; } int ObDDLService::redistribute_column_ids( ObTableSchema &new_table_schema) { int ret = OB_SUCCESS; hash::ObHashMap column_id_map; if (OB_FAIL(column_id_map.create(OB_MAX_COLUMN_NUMBER / 2, lib::ObLabel("DDLSrvTmp")))) { LOG_WARN("failed to create column id map", K(ret)); } else if (OB_FAIL(new_table_schema.generate_new_column_id_map(column_id_map))) { LOG_WARN("failed to generate new column id map", K(ret)); } else if (OB_FAIL(new_table_schema.convert_column_ids_for_ddl(column_id_map))) { LOG_WARN("failed to convert new table schema column id", K(ret)); } else { // do nothing } return ret; } int ObDDLService::delete_constraint_update_new_table( const AlterTableSchema &alter_table_schema, ObTableSchema &new_table_schema) { int ret = OB_SUCCESS; // remove constraint from new table schema caused by drop column. for (ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin(); OB_SUCC(ret) && iter != alter_table_schema.constraint_end(); iter ++) { if (OB_FAIL(new_table_schema.delete_constraint((*iter)->get_constraint_name_str()))) { LOG_WARN("fail to remove constraint from new table schema", K(ret), K(*iter)); } else {/* do nothing */} } return ret; } int ObDDLService::drop_column_update_new_table( const ObTableSchema &origin_table_schema, ObTableSchema &new_table_schema, const ObColumnSchemaV2 &orig_column_schema) { int ret = OB_SUCCESS; const uint64_t tenant_id = origin_table_schema.get_tenant_id(); int64_t new_schema_version = OB_INVALID_VERSION; ObColumnSchemaV2 *new_origin_col = new_table_schema.get_column_schema(orig_column_schema.get_column_name()); ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_ISNULL(new_origin_col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to get column from new table schema", K(ret)); } else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id)); } else { ObColumnSchemaV2 *next_col = new_table_schema.get_column_schema_by_prev_next_id(new_origin_col->get_next_column_id()); if (OB_ISNULL(next_col)) { // do nothing since local_column is tail column } else { next_col->set_prev_column_id(new_origin_col->get_prev_column_id()); next_col->set_schema_version(new_schema_version); } if (OB_FAIL(ret)) { } else if (OB_FAIL(new_table_schema.delete_column(new_origin_col->get_column_name_str()))) { LOG_WARN("fail to delete column", K(ret), K(new_origin_col->get_column_name_str())); } } return ret; } // update relevant inner table if all of schema_guard, ddl_operator and trans are not null int ObDDLService::add_new_column_to_table_schema( const ObTableSchema &origin_table_schema, const AlterTableSchema &alter_table_schema, const common::ObTimeZoneInfoWrap &tz_info_wrap, const common::ObString &nls_formats, obrpc::ObSequenceDDLArg &sequence_ddl_arg, common::ObIAllocator &allocator, ObTableSchema &new_table_schema, AlterColumnSchema &alter_column_schema, ObSchemaGetterGuard &schema_guard, ObDDLOperator *ddl_operator, common::ObMySQLTransaction *trans) { int ret = OB_SUCCESS; const ObSQLMode sql_mode = alter_table_schema.get_sql_mode(); const bool update_inner_table = nullptr != ddl_operator && nullptr != trans; bool is_oracle_mode = false; bool is_contain_part_key = false; ObSEArray gen_col_expr_arr; LOG_DEBUG("check before alter table column", K(origin_table_schema), K(alter_table_schema), K(new_table_schema)); if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to get oracle mode", K(ret)); } else if (OB_ISNULL(tz_info_wrap.get_time_zone_info()) || OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret)); } else { ObTableSchema::const_column_iterator iter = origin_table_schema.column_begin(); ObTableSchema::const_column_iterator end = origin_table_schema.column_end(); for (; OB_SUCC(ret) && iter != end; ++iter) { const ObColumnSchemaV2 *column = *iter; if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid column schema", K(column)); } else if (column->is_generated_column()) { const common::ObObj* ObObjtmp = &column->get_cur_default_value(); if (OB_FAIL(gen_col_expr_arr.push_back(ObObjtmp->get_string()))) { LOG_WARN("failed to add col expr", K(ret)); } } } } // fill column collation if (OB_FAIL(ret)) { } else if (OB_FAIL(fill_column_collation(sql_mode, is_oracle_mode, new_table_schema, allocator, alter_column_schema))) { LOG_WARN("failed to fill column collation", K(ret)); } else { int64_t max_used_column_id = new_table_schema.get_max_used_column_id(); const uint64_t tenant_id = new_table_schema.get_tenant_id(); if (is_inner_table(new_table_schema.get_table_id()) && (OB_INVALID_ID == alter_column_schema.get_column_id() || alter_column_schema.get_column_id() != max_used_column_id + 1)) { // 225 is barrier version, after this adding column in system table need specify column_id ret = OB_OP_NOT_ALLOW; LOG_WARN("inner table should add column at last and specify column_id", K(ret), K(alter_column_schema), K(max_used_column_id)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "inner table add column without column_id"); } else { alter_column_schema.set_column_id(++max_used_column_id); alter_column_schema.set_rowkey_position(0); alter_column_schema.set_index_position(0); alter_column_schema.set_not_part_key(); alter_column_schema.set_table_id(new_table_schema.get_table_id()); alter_column_schema.set_tenant_id(new_table_schema.get_tenant_id()); if (new_table_schema.is_primary_vp_table()) { // The last column add in the primary VP alter_column_schema.add_column_flag(PRIMARY_VP_COLUMN_FLAG); } new_table_schema.set_max_used_column_id(max_used_column_id); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(refill_columns_id_for_check_constraint(origin_table_schema, alter_table_schema, alter_column_schema, is_oracle_mode, allocator))) { LOG_WARN("fail to refill columns id for check constraint", K(ret)); } else if (is_oracle_mode && OB_FAIL(refill_columns_id_for_not_null_constraint(alter_table_schema, alter_column_schema))) { LOG_WARN("fail to refill column id to constraints", K(ret)); } else if (OB_FAIL(update_prev_id_for_add_column(origin_table_schema, new_table_schema, alter_column_schema, ddl_operator, trans))) { LOG_WARN("failed to update prev id", K(ret)); } else if (update_inner_table) { if (OB_FAIL(ddl_operator->create_sequence_in_add_column(new_table_schema, alter_column_schema, *trans, schema_guard, sequence_ddl_arg))) { LOG_WARN("alter table add identity column fail", K(alter_column_schema), K(ret)); } } if (OB_SUCC(ret)) { ObSchemaChecker schema_checker; if (OB_FAIL(schema_checker.init(schema_guard))) { LOG_WARN("failed to init schema guard", K(ret)); } else if (OB_FAIL(ObDDLResolver::check_default_value( alter_column_schema.get_cur_default_value(), tz_info_wrap, &nls_formats, allocator, new_table_schema, alter_column_schema, gen_col_expr_arr, alter_table_schema.get_sql_mode(), false, /* allow_sequence */ &schema_checker))) { LOG_WARN("fail to check default value", K(alter_column_schema), K(ret)); } else if (OB_FAIL(resolve_orig_default_value(alter_column_schema, tz_info_wrap, &nls_formats, allocator))) { LOG_WARN("fail to resolve default value", K(ret)); } else if (alter_column_schema.is_primary_key_) { if (new_table_schema.get_rowkey_column_num() > 0) { if (new_table_schema.is_heap_table()) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support to add primary key!", K(ret)); } else { ret = OB_ERR_MULTIPLE_PRI_KEY; LOG_WARN("multiple primary key defined", K(ret)); } } } } if (OB_SUCC(ret)) { const ObColumnSchemaV2 *mem_col = NULL; if (OB_FAIL(new_table_schema.add_column(alter_column_schema))) { if (OB_ERR_COLUMN_DUPLICATE == ret) { const ObString &column_name = alter_column_schema.get_column_name_str(); LOG_USER_ERROR(OB_ERR_COLUMN_DUPLICATE, column_name.length(), column_name.ptr()); LOG_WARN("duplicate column name", K(column_name), K(ret)); } LOG_WARN("failed to add new column", K(ret)); } else if (OB_FAIL(resolve_timestamp_column(&alter_column_schema, new_table_schema, alter_column_schema, tz_info_wrap, &nls_formats, allocator))) { LOG_WARN("fail to resolve timestamp column", K(ret)); } else if (OB_FAIL(deal_default_value_padding(alter_column_schema, allocator))) { LOG_WARN("fail to deal default value padding", K(alter_column_schema), K(ret)); } else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) { LOG_WARN("fail to check primary key cover partition column", K(ret)); } else if (OB_ISNULL(mem_col = new_table_schema.get_column_schema( alter_column_schema.get_column_id()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mem_col is NULL", K(ret)); } else { alter_column_schema.set_prev_column_id(mem_col->get_prev_column_id()); if (update_inner_table) { if (OB_FAIL(ddl_operator->insert_single_column(*trans, new_table_schema, alter_column_schema))) { LOG_WARN("failed to add column", K(ret), K(alter_column_schema)); } } } } return ret; } int ObDDLService::gen_alter_column_new_table_schema_offline( const ObTableSchema &origin_table_schema, AlterTableSchema &alter_table_schema, ObTableSchema &new_table_schema, obrpc::ObAlterTableArg &alter_table_arg, ObSchemaGetterGuard &schema_guard, bool &need_redistribute_column_id) { int ret = OB_SUCCESS; const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_; const common::ObString *nls_formats = alter_table_arg.nls_formats_; common::ObIAllocator &allocator = alter_table_arg.allocator_; need_redistribute_column_id = false; bool is_contain_part_key = false; // drop column related. int64_t new_table_cols_cnt = 0; ObArray drop_cols_id_arr; LOG_DEBUG("check before alter table column", K(origin_table_schema), K(alter_table_schema), K(new_table_schema)); if (OB_ISNULL(tz_info_wrap.get_time_zone_info()) || OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid tz_info_wrap", K(ret), K(tz_info_wrap)); } else if (OB_ISNULL(nls_formats)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid nls_formats", K(ret)); } else { AlterColumnSchema *alter_column_schema; ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end(); common::hash::ObHashSet update_column_name_set; if (OB_FAIL(update_column_name_set.create(32))) { LOG_WARN("failed to create update column name set", K(ret)); } else if (OB_FAIL(get_all_dropped_column_ids(alter_table_arg, origin_table_schema, drop_cols_id_arr, &new_table_cols_cnt/*final columns count of new table*/))) { LOG_WARN("fail to prefetch all drop columns id", K(ret), K(alter_table_arg)); } else if (drop_cols_id_arr.size() > 0) { if (OB_FAIL(check_drop_column_with_drop_foreign_key(alter_table_arg, origin_table_schema, drop_cols_id_arr))) { LOG_WARN("fail to check drop foreign key caused by drop column", K(ret), K(alter_table_arg)); } else if (OB_FAIL(check_drop_column_with_drop_constraint(alter_table_arg, schema_guard, origin_table_schema, drop_cols_id_arr))) { LOG_WARN("fail to check drop constraint caused by drop column", K(ret), K(alter_table_arg)); } else if (OB_FAIL(delete_constraint_update_new_table(alter_table_schema, new_table_schema))) { LOG_WARN("fail to delete constraints from new table", K(ret)); } else {/* do nothing. */} } for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) { if (OB_ISNULL(alter_column_schema = static_cast(*it_begin))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin is NULL", K(ret)); } else { const ObString &orig_column_name = alter_column_schema->get_origin_column_name(); // column that has been add, alter, change or modify const ObColumnSchemaV2 *orig_column_schema = NULL; const ObSchemaOperationType op_type = alter_column_schema->alter_type_; switch (op_type) { case OB_DDL_DROP_COLUMN: { orig_column_schema = origin_table_schema.get_column_schema(orig_column_name); ObColumnNameHashWrapper orig_column_key(orig_column_name); if (OB_FAIL(check_can_drop_column(orig_column_name, orig_column_schema, new_table_schema, new_table_schema.get_column_count() > new_table_cols_cnt + 1 ? new_table_schema.get_column_count() : new_table_cols_cnt + 1))) { LOG_WARN("column cannot be dropped", K(ret), K(new_table_cols_cnt), KPC(orig_column_schema)); } else if (OB_FAIL(drop_column_update_new_table(origin_table_schema, new_table_schema, *orig_column_schema))) { LOG_WARN("fail to drop column update new table", K(ret)); } else { need_redistribute_column_id = true; } break; } case OB_DDL_ADD_COLUMN: { if (OB_FAIL(add_new_column_to_table_schema(origin_table_schema, alter_table_schema, tz_info_wrap, *nls_formats, alter_table_arg.sequence_ddl_arg_, alter_table_arg.allocator_, new_table_schema, *alter_column_schema, schema_guard, nullptr, nullptr))) { LOG_WARN("failed to add new column to table schema", K(ret)); } else { ObSEArray new_pk_column; if (alter_column_schema->is_primary_key_) { if (!new_table_schema.is_heap_table()) { ret = OB_ERR_MULTIPLE_PRI_KEY; LOG_WARN("multiple primary key defined", K(ret)); } else if (OB_FAIL(new_pk_column.push_back(alter_column_schema->get_column_name_str()))){ LOG_WARN("failed to push back pk col name", K(ret)); } else if (OB_FAIL(add_primary_key(new_pk_column, new_table_schema))) { LOG_WARN("failed to add pk to table", K(ret), K(new_pk_column), K(new_table_schema)); } else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) { LOG_WARN("failed to check primary key cover partition column", K(ret)); } } } if (OB_FAIL(ret)) { } else { const ObString &column_name = alter_column_schema->get_column_name_str(); ObColumnNameHashWrapper column_key(column_name); if (OB_HASH_EXIST == update_column_name_set.exist_refactored(column_key)) { ret = OB_HASH_EXIST; LOG_WARN("duplicate column name", K(ret), K(column_name)); } else if (OB_FAIL(update_column_name_set.set_refactored(column_key))) { LOG_WARN("failed to add column name to hash set.", K(column_name), K(ret)); } else { need_redistribute_column_id = true; } } break; } case OB_DDL_CHANGE_COLUMN: case OB_DDL_MODIFY_COLUMN: { ObSchemaChecker schema_checker; const ObString &new_column_name = alter_column_schema->get_column_name_str(); orig_column_schema = new_table_schema.get_column_schema(orig_column_name); if (OB_FAIL(schema_checker.init(schema_guard))) { LOG_WARN("failed to init schema guard", K(ret)); } else if (OB_ISNULL(orig_column_schema)) { ret = OB_ERR_BAD_FIELD_ERROR; LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(), origin_table_schema.get_table_name_str().length(), origin_table_schema.get_table_name_str().ptr()); LOG_WARN("unknown column", KR(ret), K(orig_column_name), K(new_table_schema)); } else if (orig_column_schema->is_tbl_part_key_column() && !is_contain_part_key) { is_contain_part_key = true; if (OB_FAIL(new_table_schema.assign_partition_schema(alter_table_schema))) { LOG_WARN("failed to assign partition schema", K(ret)); } } if (OB_SUCC(ret)) { if (alter_column_schema->is_generated_column() && OB_FAIL(ObDDLResolver::reformat_generated_column_expr(alter_column_schema->get_cur_default_value(), tz_info_wrap, nls_formats, allocator, new_table_schema, *alter_column_schema, alter_table_schema.get_sql_mode(), &schema_checker))) { LOG_WARN("fail to check default value", KPC(alter_column_schema), K(ret)); } } ObColumnNameHashWrapper orig_column_key(orig_column_name); if (OB_FAIL(ret)) { } else if (OB_FAIL(pre_check_orig_column_schema(*alter_column_schema, origin_table_schema, update_column_name_set))) { RS_LOG(WARN, "failed to pre check orig column schema", K(ret)); } else if (OB_DDL_CHANGE_COLUMN == op_type && orig_column_schema->has_generated_column_deps()) { if (OB_FAIL(update_generated_column_schema(*alter_column_schema, *orig_column_schema, origin_table_schema, tz_info_wrap, new_table_schema))) { RS_LOG(WARN, "failed to rebuild generated column schema", K(ret)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(validate_update_column_for_materialized_view( origin_table_schema, *orig_column_schema))) { LOG_WARN("fail to validate update column for materialized view", K(ret)); } if (OB_SUCC(ret)) { /* rename column, no need to check gen col dup */ ObSEArray empty_expr_arr; if (OB_FAIL(ObDDLResolver::check_default_value(alter_column_schema->get_cur_default_value(), tz_info_wrap, nls_formats, allocator, new_table_schema, *alter_column_schema, empty_expr_arr, alter_table_schema.get_sql_mode(), !alter_column_schema->is_generated_column(), /* allow_sequence */ &schema_checker))) { LOG_WARN("fail to check default value", KPC(alter_column_schema),K(ret)); } } if (OB_SUCC(ret) && orig_column_schema->get_column_name_str() != alter_column_schema->get_column_name_str()) { if (orig_column_schema->is_tbl_part_key_column() && OB_FAIL(modify_part_func_expr( orig_column_schema->get_column_name_str(), alter_column_schema->get_column_name_str(), new_table_schema, *tz_info_wrap.get_time_zone_info(), allocator))) { LOG_WARN("modify part func expr failed", K(ret)); } else if (OB_FAIL(modify_part_func_expr_for_global_index( *orig_column_schema, *alter_column_schema, new_table_schema, *tz_info_wrap.get_time_zone_info(), allocator, nullptr/* ddl_operator */, nullptr/* trans */, nullptr/* global_idx_schema_array */))) { LOG_WARN("failed to modify part_func_expr for global_index", K(ret)); } else if (OB_FAIL(modify_constraint_check_expr( *orig_column_schema, *alter_column_schema, new_table_schema, *tz_info_wrap.get_time_zone_info(), allocator, nullptr/* ddl_operator */, nullptr/* trans */))) { LOG_WARN("failed to modify check_expr constraint", K(ret)); } } if (OB_SUCC(ret)) { //copy attributes that can be change by alter table change ... ObColumnSchemaV2 new_column_schema = *orig_column_schema; bool is_change_column_order = false; ObSEArray new_pk_column; if (alter_column_schema->is_primary_key_) { if (!new_table_schema.is_heap_table()) { ret = OB_ERR_MULTIPLE_PRI_KEY; LOG_WARN("multiple primary key defined", K(ret)); } else if (OB_FAIL(new_pk_column.push_back(alter_column_schema->get_column_name_str()))){ LOG_WARN("failed to push back pk col name", K(ret)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(fill_new_column_attributes(*alter_column_schema, new_column_schema))) { RS_LOG(WARN, "fail to fill new column attributes", K(ret)); } else if (OB_FAIL(new_column_schema.set_orig_default_value(alter_column_schema->get_orig_default_value()))) { LOG_WARN("failed to set orig default value", K(ret)); } else if (OB_FAIL(resolve_timestamp_column(alter_column_schema, new_table_schema, new_column_schema, tz_info_wrap, nls_formats, allocator))) { RS_LOG(WARN, "fail to resolve timestamp column", K(ret)); } else if (OB_FAIL(new_table_schema.alter_column(new_column_schema, ObTableSchema::CHECK_MODE_OFFLINE))) { RS_LOG(WARN, "failed to change column", K(ret)); } else if (!new_pk_column.empty() && OB_FAIL(add_primary_key(new_pk_column, new_table_schema))) { LOG_WARN("failed to add pk to table", K(ret), K(new_pk_column), K(new_table_schema)); } else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) { RS_LOG(WARN, "fail to check primary key cover partition column", K(ret)); } else { if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) { ret = OB_HASH_EXIST; RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name)); } else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) { RS_LOG(WARN, "failed to add index_name to hash set.", K(orig_column_name), K(ret)); } else if (OB_FAIL(check_is_change_column_order(new_table_schema, *alter_column_schema, is_change_column_order))) { LOG_WARN("failed to check is change column order", K(ret)); } else if (is_change_column_order) { if (alter_column_schema->get_column_id() != orig_column_schema->get_column_id()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid alter column schema column id", K(ret), K(*alter_column_schema), K(*orig_column_schema)); } else if (OB_FAIL(new_table_schema.reorder_column(new_column_schema.get_column_name_str(), alter_column_schema->is_first_, alter_column_schema->get_prev_column_name(), alter_column_schema->get_next_column_name()))) { LOG_WARN("failed to reorder column", K(ret)); } else { need_redistribute_column_id = true; } } } } break; } default: { ret = OB_ERR_UNEXPECTED; RS_LOG(WARN, "invalid offline ddl operator type!", K_(alter_column_schema->alter_type)); break; } } } } if (OB_SUCC(ret) && OB_FAIL(check_has_multi_autoinc(new_table_schema))) { LOG_WARN("failed to check table has multi autoinc", K(ret)); } if (OB_SUCC(ret) && need_redistribute_column_id) { if (OB_FAIL(redistribute_column_ids(new_table_schema))) { LOG_WARN("failed to redistribute column ids", K(ret)); } else { // do nothing } } if (OB_SUCC(ret)) { if (OB_FAIL(new_table_schema.sort_column_array_by_column_id())) { LOG_WARN("failed to sort column", K(ret), K(new_table_schema)); } } } return ret; } // update relevant inner table if both ddl_operator and trans are not null int ObDDLService::update_prev_id_for_add_column(const ObTableSchema &origin_table_schema, ObTableSchema &new_table_schema, AlterColumnSchema &alter_column_schema, ObDDLOperator *ddl_operator, common::ObMySQLTransaction *trans) { int ret = OB_SUCCESS; const uint64_t tenant_id = origin_table_schema.get_tenant_id(); const bool is_first = alter_column_schema.is_first_; const bool is_after = (!alter_column_schema.get_prev_column_name().empty()); const bool is_before = (!alter_column_schema.get_next_column_name().empty()); const bool is_last = !(is_first || is_after || is_before); const bool update_inner_table = nullptr != ddl_operator && nullptr != trans; if (is_last) { // do nothing } else { ObString pos_column_name; const uint64_t alter_column_id = alter_column_schema.get_column_id(); if (is_first) { // this first means the first of no hidden/shdow column. ObColumnIterByPrevNextID iter(new_table_schema); const ObColumnSchemaV2 *head_col = NULL; const ObColumnSchemaV2 *col = NULL; bool is_first = false; while (OB_SUCC(ret) && OB_SUCC(iter.next(col))) { if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("The column is null", K(ret)); } else if (col->is_shadow_column() || col->is_hidden()) { // do nothing } else if (!is_first) { head_col = col; is_first = true; } } if (ret != OB_ITER_END) { LOG_WARN("Failed to iterate all table columns. iter quit. ", K(ret)); } else { ret = OB_SUCCESS; if (OB_ISNULL(head_col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("Failed to get first column", K(ret)); } else { alter_column_schema.set_next_column_name(head_col->get_column_name()); } } } if (OB_SUCC(ret)) { pos_column_name = (is_after ? alter_column_schema.get_prev_column_name() : alter_column_schema.get_next_column_name()); ObColumnSchemaV2 *pos_column_schema = new_table_schema.get_column_schema(pos_column_name); ObColumnSchemaV2 *update_column_schema = NULL; if (OB_ISNULL(pos_column_schema)) { ret = OB_ERR_BAD_FIELD_ERROR; LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, pos_column_name.length(), pos_column_name.ptr(), new_table_schema.get_table_name_str().length(), new_table_schema.get_table_name_str().ptr()); LOG_WARN("pos column is NULL", K(pos_column_name)); } else { if (is_after) { // add column after alter_column_schema.set_prev_column_id(pos_column_schema->get_column_id()); update_column_schema = new_table_schema.get_column_schema_by_prev_next_id(pos_column_schema->get_next_column_id()); if (OB_NOT_NULL(update_column_schema)) { update_column_schema->set_prev_column_id(alter_column_id); } } else { // add column before / first alter_column_schema.set_prev_column_id(pos_column_schema->get_prev_column_id()); update_column_schema = pos_column_schema; update_column_schema->set_prev_column_id(alter_column_id); } if (OB_SUCC(ret)) { if (OB_ISNULL(update_column_schema)) { // alter column is the last column } else if (update_inner_table) { if (OB_FAIL(ddl_operator->update_single_column( *trans, origin_table_schema, new_table_schema, *update_column_schema))) { LOG_WARN("Failed to update single column", K(ret), K(update_column_schema->get_column_name_str())); } } } } } } return ret; } int ObDDLService::alter_table_column(const ObTableSchema &origin_table_schema, const AlterTableSchema &alter_table_schema, const int64_t frozen_version, ObTableSchema &new_table_schema, obrpc::ObAlterTableArg &alter_table_arg, ObSchemaGetterGuard &schema_guard, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans, ObIArray *global_idx_schema_array/*=NULL*/) { int ret = OB_SUCCESS; const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_; const common::ObString *nls_formats = alter_table_arg.nls_formats_; common::ObIAllocator &allocator = alter_table_arg.allocator_; bool is_oracle_mode = false; LOG_INFO("check before alter table column", K(origin_table_schema), K(alter_table_schema), K(new_table_schema)); ObSchemaChecker schema_checker; if (OB_FAIL(schema_checker.init(schema_guard))) { LOG_WARN("failed to init schema guard", K(ret)); } else if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { RS_LOG(WARN, "failed to get oracle mode", K(ret)); } else if (OB_ISNULL(tz_info_wrap.get_time_zone_info()) || OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret)); } else if (OB_ISNULL(nls_formats)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid nls_formats", K(ret)); } else { AlterColumnSchema *alter_column_schema; ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end(); lib::Worker::CompatMode compat_mode = (is_oracle_mode ? lib::Worker::CompatMode::ORACLE : lib::Worker::CompatMode::MYSQL); lib::CompatModeGuard tmpCompatModeGuard(compat_mode); ObArray idx_schema_array; common::hash::ObHashSet update_column_name_set; ObSEArray gen_col_expr_arr; bool is_origin_table_has_lob_column = false; if (OB_FAIL(update_column_name_set.create(32))) { LOG_WARN("failed to create update column name set", K(ret)); } else if (OB_FAIL(generate_tmp_idx_schemas(new_table_schema, idx_schema_array, schema_guard))) { LOG_WARN("generate tmp idx schemas failed", K(ret)); } else { share::schema::ObTableSchema::const_column_iterator iter = origin_table_schema.column_begin(); share::schema::ObTableSchema::const_column_iterator end = origin_table_schema.column_end(); for (; OB_SUCC(ret) && iter != end; ++iter) { const share::schema::ObColumnSchemaV2 *column = *iter; if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid column schema", K(column)); } else if (column->is_generated_column()) { const common::ObObj* ObObjtmp = &column->get_cur_default_value(); if (OB_FAIL(gen_col_expr_arr.push_back(ObObjtmp->get_string()))) { ret = OB_ERR_UNEXPECTED; } } if (OB_SUCC(ret) && is_lob_v2(column->get_data_type())) { is_origin_table_has_lob_column = true; } } } // Extended type info is resolved in session collation type, then we convert it to // system collation in ObDDLResolver::fill_extended_type_info(). const ObCollationType cur_extended_type_info_collation = ObCharset::get_system_collation(); for(;OB_SUCC(ret) && it_begin != it_end; it_begin++) { if (OB_ISNULL(alter_column_schema = static_cast(*it_begin))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin is NULL", K(ret)); } else { const ObString &orig_column_name = alter_column_schema->get_origin_column_name(); //cnolumn that has been alter, change or modify const ObColumnSchemaV2 *orig_column_schema = NULL; switch (alter_column_schema->alter_type_) { case OB_DDL_ADD_COLUMN: { if (OB_FAIL(add_new_column_to_table_schema(origin_table_schema, alter_table_schema, tz_info_wrap, *nls_formats, alter_table_arg.sequence_ddl_arg_, alter_table_arg.allocator_, new_table_schema, *alter_column_schema, schema_guard, &ddl_operator, &trans))) { LOG_WARN("failed to add new column to table schema", K(ret)); } break; } case OB_DDL_CHANGE_COLUMN: { orig_column_schema = new_table_schema.get_column_schema(orig_column_name); ObColumnNameHashWrapper orig_column_key(orig_column_name); /* rename column, no need to check gen col dup */ ObSEArray empty_expr_arr; if (OB_FAIL(ObDDLResolver::check_default_value( alter_column_schema->get_cur_default_value(), tz_info_wrap, nls_formats, allocator, new_table_schema, *alter_column_schema, empty_expr_arr, alter_table_schema.get_sql_mode(), !alter_column_schema->is_generated_column(), /* allow_sequence */ &schema_checker))) { LOG_WARN("fail to check default value", KPC(alter_column_schema),K(ret)); } else if (OB_FAIL(pre_check_orig_column_schema(*alter_column_schema, origin_table_schema, update_column_name_set))) { RS_LOG(WARN, "failed to pre check orig column schema", K(ret)); } else if (orig_column_schema->has_generated_column_deps()) { if (OB_FAIL(update_generated_column_schema(*alter_column_schema, *orig_column_schema, origin_table_schema, tz_info_wrap, new_table_schema, &ddl_operator, &trans))) { RS_LOG(WARN, "failed to rebuild generated column schema", K(ret)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(validate_update_column_for_materialized_view( origin_table_schema, *orig_column_schema))) { LOG_WARN("fail to validate update column for materialized view", K(ret)); } if (OB_SUCC(ret)) { if (alter_column_schema->is_primary_key_) { if (new_table_schema.get_rowkey_column_num() > 0) { if (new_table_schema.is_heap_table()) { ret = OB_NOT_SUPPORTED; RS_LOG(WARN, "not support to add primary key!", K(ret)); } else { ret = OB_ERR_MULTIPLE_PRI_KEY; RS_LOG(WARN, "multiple primary key defined", K(ret)); } } } if (OB_SUCC(ret) && alter_column_schema->is_autoincrement_) { if (alter_column_schema->is_autoincrement()) { if (orig_column_schema->get_column_id() != new_table_schema.get_autoinc_column_id()) { // not supported now; from non-auto-increment column to auto-increment column ret = OB_NOT_SUPPORTED; RS_LOG(WARN, "from non-auto-increment column to auto-increment column", "alter_column_id", alter_column_schema->get_column_id(), "auto_inc_column_id", new_table_schema.get_autoinc_column_id(), K(ret)); } } } } if (OB_SUCC(ret) && orig_column_schema->get_column_name_str() != alter_column_schema->get_column_name_str()) { if (orig_column_schema->is_tbl_part_key_column() && OB_FAIL(modify_part_func_expr( orig_column_schema->get_column_name_str(), alter_column_schema->get_column_name_str(), new_table_schema, *tz_info_wrap.get_time_zone_info(), allocator))) { LOG_WARN("modify part func expr failed", K(ret)); } else if (OB_FAIL(modify_part_func_expr_for_global_index( *orig_column_schema, *alter_column_schema, new_table_schema, *tz_info_wrap.get_time_zone_info(), allocator, &ddl_operator, &trans, global_idx_schema_array))) { LOG_WARN("failed to modify part_func_expr for global_index", K(ret)); } else if (OB_FAIL(modify_constraint_check_expr( *orig_column_schema, *alter_column_schema, new_table_schema, *tz_info_wrap.get_time_zone_info(), allocator, &ddl_operator, &trans))) { LOG_WARN("failed to modify check_expr constraint", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(check_can_alter_column_type(*orig_column_schema, *alter_column_schema, origin_table_schema))) { LOG_WARN("fail to check can alter column type", K(ret)); } } if (OB_SUCC(ret)) { //copy attributes that can be change by alter table change ... ObColumnSchemaV2 new_column_schema = *orig_column_schema; if (OB_FAIL(fill_new_column_attributes(*alter_column_schema, new_column_schema))) { RS_LOG(WARN, "fail to fill new column attributes", K(ret)); } else if (OB_FAIL(resolve_timestamp_column(alter_column_schema, new_table_schema, new_column_schema, tz_info_wrap, nls_formats, allocator))) { RS_LOG(WARN, "fail to resolve timestamp column", K(ret)); } else if (OB_FAIL(new_table_schema.alter_column(new_column_schema, ObTableSchema::CHECK_MODE_ONLINE))) { RS_LOG(WARN, "failed to change column", K(ret)); } else if (OB_FAIL(check_new_column_for_index( idx_schema_array, new_column_schema))) { RS_LOG(WARN, "failed to check new column for index", K(ret)); } else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) { RS_LOG(WARN, "fail to check primary key cover partition column", K(ret)); } else if (OB_FAIL(ddl_operator.update_single_column( trans, origin_table_schema, new_table_schema, new_column_schema))) { RS_LOG(WARN, "failed to alter column", K(alter_column_schema), K(ret)); } else if (OB_FAIL(alter_table_update_index_and_view_column( new_table_schema, new_column_schema, ddl_operator, trans, global_idx_schema_array))) { RS_LOG(WARN, "failed to update index column", K(ret)); } else { if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) { ret = OB_HASH_EXIST; RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name)); } else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) { RS_LOG(WARN, "failed to add index_name to hash set.", K(orig_column_name), K(ret)); } } } break; } case OB_DDL_MODIFY_COLUMN: { LOG_DEBUG("check alter column schema", KPC(alter_column_schema)); orig_column_schema = new_table_schema.get_column_schema(orig_column_name); ObColumnNameHashWrapper orig_column_key(orig_column_name); if (alter_column_schema->is_generated_column() && OB_FAIL(ObDDLResolver::reformat_generated_column_expr(alter_column_schema->get_cur_default_value(), tz_info_wrap, nls_formats, allocator, new_table_schema, *alter_column_schema, alter_table_schema.get_sql_mode(), &schema_checker))) { LOG_WARN("fail to check default value", KPC(alter_column_schema), K(ret)); } else if (OB_FAIL(pre_check_orig_column_schema(*alter_column_schema, origin_table_schema, update_column_name_set))) { RS_LOG(WARN, "failed to pre check orig column schema", K(ret)); } else if (!alter_column_schema->is_generated_column() /* Not support modify to generate columns, so there is no need to check again here */ && OB_FAIL(ObDDLResolver::check_default_value(alter_column_schema->get_cur_default_value(), tz_info_wrap, nls_formats, allocator, new_table_schema, *alter_column_schema, gen_col_expr_arr, alter_table_schema.get_sql_mode(), !alter_column_schema->is_generated_column(), /* allow_sequence */ &schema_checker))) { LOG_WARN("fail to check default value", KPC(alter_column_schema), K(ret)); } else if (OB_FAIL(alter_sequence_in_alter_column(new_table_schema, *alter_column_schema, trans, schema_guard, alter_table_arg.sequence_ddl_arg_))) { LOG_WARN("alter table modeify identity column fail", K(alter_column_schema), K(ret)); } else if (alter_column_schema->is_primary_key_) { if (new_table_schema.get_rowkey_column_num() > 0) { if (new_table_schema.is_heap_table()) { ret = OB_NOT_SUPPORTED; RS_LOG(WARN, "not support to add primary key!", K(ret)); } else { ret = OB_ERR_MULTIPLE_PRI_KEY; RS_LOG(WARN, "multiple primary key defined", K(ret)); } } } if (OB_SUCC(ret) && alter_column_schema->is_autoincrement_) { if (alter_column_schema->is_autoincrement()) { if (orig_column_schema->get_column_id() != new_table_schema.get_autoinc_column_id()) { // not supported now; from non-auto-increment column to auto-increment column ret = OB_NOT_SUPPORTED; RS_LOG(WARN, "from non-auto-increment column to auto-increment column", K(ret)); } } } if (OB_SUCC(ret)) { if (OB_FAIL(validate_update_column_for_materialized_view(origin_table_schema, *orig_column_schema))) { LOG_WARN("fail to validate update column for materialized view", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(check_can_alter_column_type(*orig_column_schema, *alter_column_schema, origin_table_schema))) { LOG_WARN("fail to check can alter column type", K(ret)); } } if (OB_SUCC(ret)) { ObColumnSchemaV2 new_column_schema = *orig_column_schema; //copy attributes that can be change by alter table modify ... if (OB_FAIL(fill_new_column_attributes(*alter_column_schema, new_column_schema))) { RS_LOG(WARN, "fail to fill new column attributes", K(ret)); } else if (OB_FAIL(check_modify_column_when_upgrade(new_column_schema, *orig_column_schema))) { LOG_WARN("fail to check modify column when upgrade", K(ret), K(new_column_schema), K(*orig_column_schema)); } else if (OB_FAIL(resolve_timestamp_column(alter_column_schema, new_table_schema, new_column_schema, tz_info_wrap, nls_formats, allocator))) { RS_LOG(WARN, "fail to resolve timestamp column", K(ret)); } else if (OB_FAIL(new_table_schema.alter_column(new_column_schema, ObTableSchema::CHECK_MODE_ONLINE))) { RS_LOG(WARN, "failed to change column", K(ret)); } else if (OB_FAIL(check_new_column_for_index( idx_schema_array, new_column_schema))) { RS_LOG(WARN, "failed to check new column for index", K(ret)); } else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) { RS_LOG(WARN, "fail to check primary key cover partition column", K(ret)); } else if (OB_FAIL(ddl_operator.update_single_column( trans, origin_table_schema, new_table_schema, new_column_schema))) { RS_LOG(WARN, "failed to alter column", K(alter_column_schema), K(ret)); } else if (OB_FAIL(alter_table_update_index_and_view_column(new_table_schema, new_column_schema, ddl_operator, trans))) { RS_LOG(WARN, "failed to update index column", K(ret)); } else { if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) { ret = OB_HASH_EXIST; RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name)); } else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) { RS_LOG(WARN, "failed to add index_name to hash set.", K(orig_column_name), K(ret)); } } } break; } case OB_DDL_ALTER_COLUMN: { orig_column_schema = new_table_schema.get_column_schema(orig_column_name); ObColumnNameHashWrapper orig_column_key(orig_column_name); if (OB_FAIL(pre_check_orig_column_schema(*alter_column_schema, origin_table_schema, update_column_name_set))) { RS_LOG(WARN, "failed to pre check orig column schema", K(ret)); } else if (OB_FAIL(validate_update_column_for_materialized_view( origin_table_schema, *orig_column_schema))) { LOG_WARN("fail to validate update column for materialized view", K(ret)); } //column that has been modified, can't not modify again if (OB_SUCC(ret)) { ObColumnSchemaV2 new_column_schema = *orig_column_schema; // orig_column_schema is passed in resolve_timestamp_column to verify whether the now() is dropped. if (OB_FAIL(resolve_timestamp_column(alter_column_schema, new_table_schema, new_column_schema, tz_info_wrap, nls_formats, allocator))) { RS_LOG(WARN, "fail to resolve timestamp column", K(ret)); } else { ObObj default_value; if (alter_column_schema->is_drop_default_) { default_value.set_null(); new_column_schema.del_column_flag(DEFAULT_EXPR_V2_COLUMN_FLAG); if (OB_FAIL(new_column_schema.set_cur_default_value(default_value))) { RS_LOG(WARN, "failed to set current default value"); } } else { default_value = alter_column_schema->get_cur_default_value(); if (!default_value.is_null() && ob_is_text_tc(new_column_schema.get_data_type())) { ret = OB_INVALID_DEFAULT; LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(), new_column_schema.get_column_name_str().ptr()); RS_LOG(WARN, "BLOB, TEXT column can't have a default value!", K(default_value), K(ret)); } else if (ob_is_json_tc(new_column_schema.get_data_type())) { // cannot alter json column to any default value // text column also cannot be alter to null in mysql ret = OB_ERR_BLOB_CANT_HAVE_DEFAULT; LOG_USER_ERROR(OB_ERR_BLOB_CANT_HAVE_DEFAULT, new_column_schema.get_column_name_str().length(), new_column_schema.get_column_name_str().ptr()); RS_LOG(WARN, "JSON column can't have a default value!", K(default_value), K(ret)); } else if (!new_column_schema.is_nullable() && default_value.is_null()) { ret = OB_INVALID_DEFAULT; LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(), new_column_schema.get_column_name_str().ptr()); RS_LOG(WARN, "not null column with default value null!", K(ret)); } else if (OB_FAIL(ObDDLResolver::check_default_value(default_value, tz_info_wrap, nls_formats, allocator, new_table_schema, new_column_schema, gen_col_expr_arr, alter_table_schema.get_sql_mode(), !alter_column_schema->is_generated_column(), /* allow_sequence */ &schema_checker))) { LOG_WARN("fail to check default value", K(new_column_schema),K(ret)); } else if (OB_FAIL(new_column_schema.set_cur_default_value(default_value))) { RS_LOG(WARN, "failed to set current default value"); } } } if (OB_SUCC(ret)) { if (OB_FAIL(new_table_schema.alter_column(new_column_schema, ObTableSchema::CHECK_MODE_ONLINE))) { RS_LOG(WARN, "failed to change column", K(ret)); } else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) { RS_LOG(WARN, "failed to check primary key cover partition column", K(ret)); } else if (OB_FAIL(ddl_operator.update_single_column(trans, origin_table_schema, new_table_schema, new_column_schema))) { RS_LOG(WARN, "failed to alter column", K(alter_column_schema), K(ret)); } else if (OB_FAIL(alter_table_update_index_and_view_column(new_table_schema, new_column_schema, ddl_operator, trans))) { RS_LOG(WARN, "failed to update index column", K(ret)); } else { if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) { ret = OB_HASH_EXIST; RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name)); } else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) { RS_LOG(WARN, "failed to add index_name to hash set.", K(orig_column_name), K(ret)); } } } } break; } case OB_DDL_DROP_COLUMN: { orig_column_schema = origin_table_schema.get_column_schema(orig_column_name); ObColumnNameHashWrapper orig_column_key(orig_column_name); if (OB_FAIL(check_can_drop_column(orig_column_name, orig_column_schema, new_table_schema, new_table_schema.get_column_count()))) { LOG_WARN("column cannot be dropped", K(ret), KPC(orig_column_schema)); } else if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) { //column that has been modified, can't not modify again ret = OB_ERR_BAD_FIELD_ERROR; LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(), origin_table_schema.get_table_name_str().length(), origin_table_schema.get_table_name_str().ptr()); RS_LOG(WARN, "column that has been altered, can't not update again"); } else if (OB_FAIL(validate_update_column_for_materialized_view(origin_table_schema, *orig_column_schema))) { LOG_WARN("fail to validate update column for materialized view", K(ret)); } else if (OB_FAIL(ddl_operator.drop_sequence_in_drop_column(*orig_column_schema, trans, schema_guard))) { RS_LOG(WARN, "alter table drop identity column fail", K(ret)); } else if (OB_FAIL(ddl_operator.alter_table_drop_aux_column( new_table_schema, *orig_column_schema, trans, USER_INDEX))) { RS_LOG(WARN, "failed to drop index column", K(ret)); } else if (OB_FAIL(ddl_operator.alter_table_drop_aux_column( new_table_schema, *orig_column_schema, trans, AUX_VERTIAL_PARTITION_TABLE))) { RS_LOG(WARN, "failed to drop aux vp column", K(ret)); } else if (OB_FAIL(ddl_operator.update_prev_id_for_delete_column(origin_table_schema, new_table_schema, *orig_column_schema, trans))) { LOG_WARN("failed to update column previous id for delele column", K(ret)); } else if (OB_FAIL(ddl_operator.delete_single_column(trans, new_table_schema, orig_column_name))) { RS_LOG(WARN, "failed to delete column", K(alter_column_schema), K(ret)); } break; } default: { ret = OB_INVALID_ARGUMENT; RS_LOG(WARN, "unhandled operator type!", K_(alter_column_schema->alter_type)); break; } } } } if (OB_SUCC(ret) && !is_origin_table_has_lob_column) { if (OB_FAIL(create_aux_lob_table_if_need( new_table_schema, schema_guard, ddl_operator, trans))) { LOG_WARN("fail to create_aux_lob_table_if_need", K(ret), K(new_table_schema)); } } } return ret; } int ObDDLService::create_aux_lob_table_if_need(ObTableSchema &data_table_schema, ObSchemaGetterGuard &schema_guard, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans) { int ret = OB_SUCCESS; ObArray aux_table_schemas; const uint64_t tenant_id = data_table_schema.get_tenant_id(); bool need_sync_schema_version = false; int64_t frozen_scn = 0; if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) { LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id)); } else if (OB_FAIL(build_aux_lob_table_schema_if_need(data_table_schema, aux_table_schemas))) { LOG_WARN("fail to build_aux_lob_table_schema_if_need", K(ret), K(data_table_schema)); } else if (aux_table_schemas.count() == 0) { // no need create aux lob table, do nothing } else if (is_sys_table(data_table_schema.get_table_id())) { // whatever enable_sys_table_ddl is, sys table do not allow create lob aux tables by alter columns char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE]; ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg),"%s", "system table add or modify column"); LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg); } else { ObTableCreator table_creator( tenant_id, frozen_scn, *lst_operator_, trans); ObNewTableTabletAllocator new_table_tablet_allocator( tenant_id, schema_guard, sql_proxy_); const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version(); if (OB_INVALID_VERSION == last_schema_version) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid schema version" , K(last_schema_version)); } else if (OB_FAIL(table_creator.init())) { LOG_WARN("fail to init table creator", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.init())) { LOG_WARN("fail to init new table tablet allocator", KR(ret)); } ObSEArray schemas; for (int64_t i = 0; OB_SUCC(ret) && i < aux_table_schemas.count(); i++) { share::schema::ObTableSchema &table_schema = aux_table_schemas.at(i); if (OB_FAIL(ddl_operator.create_table(table_schema, trans, NULL, need_sync_schema_version))) { LOG_WARN("failed to create table schema", K(ret)); } else if (OB_FAIL(schemas.push_back(&table_schema))) { LOG_WARN("failed to push_back table schema", K(ret), K(table_schema)); } if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.insert_ori_schema_version( trans, tenant_id, table_schema.get_table_id(), last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(table_schema), K(last_schema_version)); } } } if (OB_SUCC(ret)) { common::ObArray ls_id_array; if(OB_FAIL(new_table_tablet_allocator.prepare_like(data_table_schema))) { LOG_WARN("fail to prepare like", KR(ret), K(data_table_schema)); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_local_aux_tables_arg( schemas, &data_table_schema, ls_id_array))) { LOG_WARN("create table partitions failed", KR(ret), K(last_schema_version)); } else if (OB_FAIL(table_creator.execute())) { LOG_WARN("fail to execute crate tablet", KR(ret)); } } // finishing is always invoked for new table tablet allocator int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) { LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret)); } FLOG_INFO("finish create aux lob table for data table", K(ret), K(data_table_schema)); } return ret; } int ObDDLService::alter_table_foreign_keys(const share::schema::ObTableSchema &orig_table_schema, share::schema::ObTableSchema &inc_table_schema, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans) { int ret = OB_SUCCESS; if (OB_FAIL(ddl_operator.add_table_foreign_keys( orig_table_schema, inc_table_schema, trans))) { LOG_WARN("failed to add table foreign_keys", K(ret)); } return ret; } int ObDDLService::drop_not_null_cst_in_column_flag( const ObTableSchema &orig_table_schema, const AlterTableSchema &alter_table_schema, ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; int64_t col_cnt = new_table_schema.get_column_count(); for (ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin(); OB_SUCC(ret) && iter != alter_table_schema.constraint_end(); iter ++) { if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) { const uint64_t column_id = *((*iter)->cst_col_begin()); ObColumnSchemaV2 *column = new_table_schema.get_column_schema(column_id); if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid column", K(ret)); } else if (column->has_not_null_constraint()) { column->drop_not_null_cst(); if (OB_FAIL(ddl_operator.update_single_column( trans, orig_table_schema, new_table_schema, *column))) { LOG_WARN("failed to update single column", K(ret)); } } } } return ret; } int ObDDLService::alter_not_null_cst_in_column_flag( const ObTableSchema &orig_table_schema, const AlterTableSchema &alter_table_schema, ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; for (ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin(); OB_SUCC(ret) && iter != alter_table_schema.constraint_end(); iter ++) { if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) { const uint64_t column_id = *((*iter)->cst_col_begin()); ObColumnSchemaV2 *column = new_table_schema.get_column_schema(column_id); if (OB_ISNULL(column)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid column", K(ret)); } else { int64_t column_flags = column->get_column_flags(); column->add_or_del_column_flag(NOT_NULL_ENABLE_FLAG, (*iter)->get_enable_flag()); column->add_or_del_column_flag(NOT_NULL_RELY_FLAG, (*iter)->get_rely_flag()); column->add_or_del_column_flag(NOT_NULL_VALIDATE_FLAG, (*iter)->is_validated()); if (column_flags != column->get_column_flags() && OB_FAIL(ddl_operator.update_single_column( trans, orig_table_schema, new_table_schema, *column))) { LOG_WARN("failed to update single column", K(ret)); } } } } return ret; } int ObDDLService::alter_table_constraints(const ObAlterTableArg::AlterConstraintType op_type, share::schema::ObSchemaGetterGuard &schema_guard, const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema, ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; if (OB_FAIL(check_can_alter_table_constraints(op_type, schema_guard, orig_table_schema, inc_table_schema))) { LOG_WARN("fail to check can alter constraints", K(ret), K(op_type), K(inc_table_schema)); } if (OB_SUCC(ret)) { if (obrpc::ObAlterTableArg::ADD_CONSTRAINT == op_type) { if (OB_FAIL(ddl_operator.add_table_constraints(inc_table_schema, new_table_schema, trans))) { LOG_WARN("failed to add table constraints", K(ret)); } } else if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == op_type) { if (OB_FAIL(drop_not_null_cst_in_column_flag(orig_table_schema, inc_table_schema, new_table_schema, ddl_operator, trans))) { LOG_WARN("failed to drop not null cst in column flag", K(ret)); } else if (OB_FAIL(ddl_operator.drop_table_constraints(orig_table_schema, inc_table_schema, new_table_schema, trans))) { LOG_WARN("failed to drop table constraints", K(ret)); } } else if (obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE == op_type) { if (OB_FAIL(alter_not_null_cst_in_column_flag(orig_table_schema, inc_table_schema, new_table_schema, ddl_operator, trans))) { LOG_WARN("failed to drop not null cst in column flag", K(ret)); } else if (OB_FAIL(ddl_operator.modify_check_constraints_state( orig_table_schema, inc_table_schema, new_table_schema, trans))) { LOG_WARN("failed to drop table constraints", K(ret)); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected op type", K(op_type), K(ret)); } } return ret; } int ObDDLService::check_can_alter_table_constraints( const ObAlterTableArg::AlterConstraintType op_type, share::schema::ObSchemaGetterGuard &schema_guard, const ObTableSchema &orig_table_schema, const AlterTableSchema &inc_table_schema) { int ret = OB_SUCCESS; bool is_oracle_mode = false; if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check if tenant mode is oracle mode", K(ret)); } else { for (ObTableSchema::const_constraint_iterator iter = inc_table_schema.constraint_begin(); OB_SUCC(ret) && iter != inc_table_schema.constraint_end(); iter ++) { const ObString &cst_name = (*iter)->get_constraint_name_str(); if (ObAlterTableArg::AlterConstraintType::ADD_CONSTRAINT == op_type) { bool is_check_constraint_name_exist = true; if (OB_FAIL(check_constraint_name_is_exist(schema_guard, orig_table_schema, cst_name, false, is_check_constraint_name_exist))) { LOG_WARN("fail to check check constraint name is exist or not", K(ret), K(cst_name)); } else if (is_check_constraint_name_exist) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; LOG_WARN("check constraint name is duplicate", K(ret), K(cst_name)); if (!is_oracle_mode) { LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst_name.length(), cst_name.ptr()); } } } for (ObTableSchema::const_constraint_iterator iter_r = iter + 1; OB_SUCC(ret) && iter_r != inc_table_schema.constraint_end(); iter_r ++) { if (is_oracle_mode && (*iter_r)->get_constraint_name_str() == cst_name) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; LOG_WARN("duplicate constraint name", K(ret), K(cst_name)); } else if (!is_oracle_mode && 0 == cst_name.case_compare((*iter_r)->get_constraint_name_str())) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst_name.length(), cst_name.ptr()); LOG_WARN("duplicate constraint name", K(ret), K(cst_name)); } } if (OB_SUCC(ret)) { ObConstraint* const* res = nullptr; if (is_oracle_mode) { res = std::find_if(orig_table_schema.constraint_begin(), orig_table_schema.constraint_end(), [&cst_name](const ObConstraint *cst) { return cst_name == cst->get_constraint_name_str(); }); } else { res = std::find_if(orig_table_schema.constraint_begin(), orig_table_schema.constraint_end(), [&cst_name](const ObConstraint* cst) { return 0 == cst_name.case_compare(cst->get_constraint_name_str()); }); } if (orig_table_schema.constraint_end() == res) { if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == op_type || obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE == op_type) { ret = OB_ERR_CONTRAINT_NOT_FOUND; LOG_WARN("constraint not exist", K(ret), K(cst_name)); } } else { if (obrpc::ObAlterTableArg::ADD_CONSTRAINT == op_type) { ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE; if (!is_oracle_mode) { LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst_name.length(), cst_name.ptr()); } LOG_WARN("constraint duplicate", K(ret), K(cst_name)); } else if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == op_type || obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE == op_type) { // drop or modify state if ((*iter)->get_constraint_id() != OB_INVALID_ID) { if ((*iter)->get_constraint_id() != (*res)->get_constraint_id()) { ret = OB_ERR_CONTRAINT_NOT_FOUND; LOG_WARN("constraint not exist", K(ret), K((*iter)->get_constraint_id()), K((*res)->get_constraint_id())); } } if (OB_FAIL(ret)) { } else if (CONSTRAINT_TYPE_PRIMARY_KEY == (*res)->get_constraint_type()) { ret = OB_NOT_SUPPORTED; LOG_WARN("alter table drop/modify pk not supported now", K(ret), K((*res)->get_constraint_type())); LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter table drop primary key"); } else { const_cast(*iter)->set_constraint_id((*res)->get_constraint_id()); const_cast(*iter)->set_constraint_type((*res)->get_constraint_type()); } } } } } } return ret; } int ObDDLService::get_partition_by_subpart_name(const ObTableSchema &orig_table_schema, const ObSubPartition &subpart_name, const ObPartition *&part, const ObSubPartition *&subpart) { int ret = OB_SUCCESS; if (part != NULL) { ObSubPartIterator sub_iter(orig_table_schema, *part, CHECK_PARTITION_MODE_NORMAL); while (OB_SUCC(ret) && OB_SUCC(sub_iter.next(subpart))) { if (OB_ISNULL(subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null subpartition", K(ret)); } else if (ObCharset::case_insensitive_equal(subpart_name.get_part_name(), subpart->get_part_name())) { break; } } } else { ObPartIterator iter(orig_table_schema, CHECK_PARTITION_MODE_NORMAL); while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) { if (OB_ISNULL(part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", KR(ret)); } else { ObSubPartIterator sub_iter(orig_table_schema, *part, CHECK_PARTITION_MODE_NORMAL); subpart = NULL; while (OB_SUCC(ret) && OB_SUCC(sub_iter.next(subpart))) { if (OB_ISNULL(subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null subpartition", K(ret)); } else if (ObCharset::case_insensitive_equal(subpart_name.get_part_name(), subpart->get_part_name())) { break; } } if (OB_ITER_END == ret) { ret = OB_SUCCESS; } else { break; } } } } return ret; } int ObDDLService::gen_inc_table_schema_for_add_part( const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema) { int ret = OB_SUCCESS; // TODO: Add partition in interval partitioned table should reorganize part_idx, which will be supported in the later version. int64_t max_part_idx = OB_INVALID_INDEX; if (orig_table_schema.is_interval_part()) { ret = OB_NOT_SUPPORTED; LOG_WARN("add interval part will support later", KR(ret)); } else if (OB_FAIL(orig_table_schema.get_max_part_idx(max_part_idx))) { LOG_WARN("fail to get max part idx", KR(ret), K(orig_table_schema)); } else { const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num(); ObPartition **inc_part_array = inc_table_schema.get_part_array(); for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) { ObPartition *inc_part = inc_part_array[i]; if (OB_ISNULL(inc_part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc part is null", KR(ret), K(i)); } else { // part_id will be generated by generate_object_id_for_partition_schema() inc_part->set_part_idx(max_part_idx + i + 1); if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level()) { for (int64_t j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) { ObSubPartition *subpart = inc_part->get_subpart_array()[j]; if (OB_ISNULL(subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("subpart is null", KR(ret), K(j)); } else if (subpart->get_sub_part_idx() >= 0 && subpart->get_sub_part_idx() != j) { ret = OB_ERR_UNEXPECTED; LOG_WARN("subpart_idx should be invalid", KR(ret), KPC(subpart), K(j)); } else { // subpart_id will be generated by generate_object_id_for_partition_schema() // For some situations(not hash like), subpart_idx from ddl_resolver is invalid. subpart->set_sub_part_idx(j); } } // end for iterate inc subpart } } } // end for iterate inc part } return ret; } int ObDDLService::gen_inc_table_schema_for_add_subpart( const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema) { int ret = OB_SUCCESS; const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num(); ObPartition **inc_part_array = inc_table_schema.get_part_array(); for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) { ObPartition *inc_part = inc_part_array[i]; ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL; ObPartIterator iter(orig_table_schema, check_partition_mode); const ObPartition *part = NULL; while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) { if (OB_ISNULL(part) || OB_ISNULL(inc_part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", KP(part), KP(inc_part), KR(ret)); } else if (ObCharset::case_insensitive_equal(inc_part->get_part_name(), part->get_part_name())) { inc_part->set_part_id(part->get_part_id()); int64_t max_subpart_idx = OB_INVALID_INDEX; if (OB_FAIL(part->get_max_sub_part_idx(max_subpart_idx))) { LOG_WARN("fail to get max sub part idx", KR(ret), KPC(part)); } else { for (int64_t j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) { ObSubPartition *subpart = inc_part->get_subpart_array()[j]; if (OB_ISNULL(subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("subpart is null", KR(ret), K(j)); } else { // subpart_id will be generated by generate_object_id_for_partition_schema() subpart->set_sub_part_idx(max_subpart_idx + j + 1); } } // end for iterate inc subpart } break; } } if (OB_ITER_END == ret) { ret = OB_PARTITION_NOT_EXIST; LOG_WARN("part should exists", KR(ret)); } else if (OB_FAIL(ret)) { LOG_WARN("iter part failed", KR(ret)); } } // end for iterate inc part return ret; } int ObDDLService::gen_inc_table_schema_for_drop_part( const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema) { int ret = OB_SUCCESS; const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num(); ObPartition **inc_part_array = inc_table_schema.get_part_array(); for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) { ObPartition *inc_part = inc_part_array[i]; ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL; ObPartIterator iter(orig_table_schema, check_partition_mode); const ObPartition *part = NULL; while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) { if (OB_ISNULL(part) || OB_ISNULL(inc_part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", KP(part), KP(inc_part), KR(ret)); } else if (ObCharset::case_insensitive_equal(inc_part->get_part_name(), part->get_part_name())) { if (OB_FAIL(inc_part->assign(*part))) { LOG_WARN("failed to assign partition", KR(ret), K(part), K(inc_part)); } break; } } if (OB_ITER_END == ret) { ret = OB_PARTITION_NOT_EXIST; LOG_WARN("part should exists", KR(ret)); } else if (OB_SUCC(ret) && orig_table_schema.is_interval_part() && inc_part->get_high_bound_val() == orig_table_schema.get_transition_point()) { // deleting the partition whose high level is transition point not support yet. ret = OB_NOT_SUPPORTED; LOG_WARN("delete the max range part of interval part table", KR(ret), K(orig_table_schema), K(inc_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "delete the max range part of interval part table"); } } return ret; } int ObDDLService::gen_inc_table_schema_for_drop_subpart(const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema) { int ret = OB_SUCCESS; ObPartition *inc_part = NULL; if (1 != inc_table_schema.get_partition_num()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("drop subparts not in a part", KR(ret)); } else if (OB_ISNULL(inc_part = inc_table_schema.get_part_array()[0])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc_part is null", K(ret)); } else { const ObPartition *part = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < inc_part->get_subpartition_num(); i++) { ObSubPartition *inc_subpart = inc_part->get_subpart_array()[i]; if (OB_ISNULL(inc_subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc_subpart is null", K(ret)); } else if (i == 0) { const ObSubPartition *subpart = NULL; if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, part, subpart))) { if (OB_ITER_END == ret) { ret = OB_PARTITION_NOT_EXIST; LOG_USER_ERROR(OB_PARTITION_NOT_EXIST); LOG_WARN("part should exists", K(ret)); } else { LOG_WARN("fail to get partition by subpart name", K(ret)); } } else if (OB_ISNULL(part) || OB_ISNULL(subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null part/subpart", K(ret)); } else if (part->get_sub_part_num() <= inc_part->get_subpartition_num()) { ret = OB_ERR_DROP_LAST_PARTITION; LOG_WARN("cannot drop all subpartitions", K(part), "partitions current", part->get_sub_part_num(), "partitions to be dropped", inc_part->get_subpartition_num(), K(ret)); LOG_USER_ERROR(OB_ERR_DROP_LAST_PARTITION); } else if (OB_FAIL(inc_subpart->assign(*subpart))) { LOG_WARN("failed to assign partition", K(ret), K(part), K(inc_part)); } else { inc_part->set_part_id(part->get_part_id()); } } else if (OB_ISNULL(part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", KR(ret)); } else { const ObSubPartition *subpart = NULL; if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, part, subpart))) { if (OB_ITER_END == ret) { part = NULL; if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, part, subpart))) { if (OB_ITER_END == ret) { ret = OB_PARTITION_NOT_EXIST; LOG_USER_ERROR(OB_PARTITION_NOT_EXIST); LOG_WARN("part should exists", K(ret)); } else { LOG_WARN("fail to get partition by subpart name", K(ret)); } } else if (OB_ISNULL(part) || OB_ISNULL(subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null part/subpart", K(ret)); } else { ret = OB_INVALID_PARTITION; LOG_USER_ERROR(OB_INVALID_PARTITION); LOG_WARN("subparts for deleted in different part", K(ret)); } } else { LOG_WARN("fail to get partition by subpart name", K(ret)); } } else if (OB_FAIL(inc_subpart->assign(*subpart))) { LOG_WARN("failed to assign partition", K(ret), K(part), K(inc_part)); } } } } return ret; } int ObDDLService::gen_inc_table_schema_for_trun_part( const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema, AlterTableSchema &del_table_schema) { int ret = OB_SUCCESS; const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num(); ObPartition **inc_part_array = inc_table_schema.get_part_array(); const int64_t del_part_num = del_table_schema.get_part_option().get_part_num(); ObPartition **del_part_array = del_table_schema.get_part_array(); if (OB_ISNULL(inc_part_array) || OB_ISNULL(del_part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("partition array is null", KR(ret), K(inc_part_array), K(del_part_array), K(inc_table_schema)); } else if (inc_part_num != del_part_num) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc is not equal to del", KR(ret), K(del_table_schema), K(inc_table_schema)); } for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) { ObPartition *inc_part = inc_part_array[i]; ObPartition *del_part = del_part_array[i]; if (OB_ISNULL(inc_part) || OB_ISNULL(del_part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("partition is null", KR(ret), K(i), K(inc_part_num)); } else { ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL; ObPartIterator iter(orig_table_schema, check_partition_mode); const ObPartition *part = NULL; while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) { if (OB_ISNULL(part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(part), KR(ret)); } else if (ObCharset::case_insensitive_equal(inc_part->get_part_name(), part->get_part_name())) { if (OB_FAIL(inc_part->assign(*part))) { LOG_WARN("failed to assign partition", KR(ret), K(part), K(inc_part)); } else if (OB_FAIL(del_part->assign(*part))) { LOG_WARN("failed to assign partition", KR(ret), K(part), K(del_part)); } break; } } if (OB_ITER_END == ret) { ret = OB_PARTITION_NOT_EXIST; LOG_WARN("part should exists", KR(ret), KPC(inc_part)); } } } return ret; } // sql puts the subpartitions of multiple partitions that may exist // into one partition and transfers it // This function generates the necessary partitions // and puts all the subpartitions in the right position int ObDDLService::gen_inc_table_schema_for_trun_subpart( const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema, AlterTableSchema &del_table_schema) { int ret = OB_SUCCESS; ObPartition *inc_part_p = NULL; ObPartition inc_part_set; if (OB_ISNULL(inc_table_schema.get_part_array())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_array is null", KR(ret)); } else if (OB_ISNULL(inc_part_p = inc_table_schema.get_part_array()[0])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc_part_p is null", KR(ret)); } else if (OB_FAIL(inc_part_set.assign(*inc_part_p))) { LOG_WARN("failed to assign partition", KR(ret), K(inc_part_set), KPC(inc_part_p)); } else { inc_table_schema.reset_partition_array(); for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_set.get_subpartition_num(); i++) { const ObPartition *inc_part = NULL; const ObSubPartition *orig_subpart = NULL; const ObPartition *orig_part = NULL; ObSubPartition *inc_subpart = inc_part_set.get_subpart_array()[i]; if (OB_ISNULL(inc_subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc_subpart is null", KR(ret)); } else if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, orig_part, orig_subpart))) { if (OB_ITER_END == ret) { ret = OB_PARTITION_NOT_EXIST; LOG_USER_ERROR(OB_PARTITION_NOT_EXIST); LOG_WARN("part should exists", KR(ret), KPC(inc_subpart)); } else { LOG_WARN("fail to get partition by subpart name", KR(ret)); } } else if (OB_ISNULL(orig_part) || OB_ISNULL(orig_subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null part/subpart", KR(ret)); } else if (0 != i && OB_FAIL(get_part_by_part_id( inc_table_schema, orig_part->get_part_id(), inc_part))) { LOG_WARN("fail to get partition", KR(ret), K(orig_part->get_part_id())); } else if (OB_ISNULL(inc_part)) { ObPartition new_part; new_part.set_part_id(orig_part->get_part_id()); if (OB_FAIL(new_part.add_partition(*orig_subpart))){ LOG_WARN("failed to add partition", K(orig_subpart), KR(ret)); } else if (OB_FAIL(inc_table_schema.add_partition(new_part))) { LOG_WARN("failed to add partition", KR(ret)); } } else if (OB_FAIL(const_cast(inc_part)->add_partition(*orig_subpart))) { LOG_WARN("failed to add partition", K(orig_subpart), KR(ret)); } } if (OB_SUCC(ret)) { ObPartition **part_array = inc_table_schema.get_part_array(); const int64_t part_num = inc_table_schema.get_partition_num(); inc_table_schema.set_part_num(part_num); if (OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_array is NULL", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < part_num; i++) { if (OB_ISNULL(part_array[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_array[i] is NULL", K(ret), K(i)); } else { part_array[i]->set_sub_part_num(part_array[i]->get_subpartition_num()); } } } } if (FAILEDx(del_table_schema.assign_partition_schema(inc_table_schema))) { LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret)); } return ret; } const char* ObDDLService::ddl_type_str(const ObDDLType ddl_type) { const char *str = ""; if (DDL_CREATE_INDEX == ddl_type) { str = "create index"; } else if (DDL_MODIFY_COLUMN == ddl_type) { str = "alter column"; } else if (DDL_CHECK_CONSTRAINT == ddl_type) { str = "add or modify check constraint"; } else if (DDL_FOREIGN_KEY_CONSTRAINT == ddl_type) { str = "alter foreign key constraint"; } else if (DDL_ADD_PRIMARY_KEY == ddl_type) { str = "add primary key"; } else if (DDL_DROP_PRIMARY_KEY == ddl_type) { str = "drop primary key"; } else if (DDL_ALTER_PRIMARY_KEY == ddl_type) { str = "alter primary key"; } else if (DDL_ALTER_PARTITION_BY == ddl_type) { str = "alter partition by"; } else if (DDL_DROP_COLUMN == ddl_type) { str = "drop column"; } else if (DDL_ADD_NOT_NULL_COLUMN == ddl_type) { str = "add not null column"; } else if (DDL_ADD_COLUMN_OFFLINE == ddl_type) { str = "add column offline"; } else if (DDL_ADD_COLUMN_ONLINE == ddl_type) { str = "add column online"; } else if (DDL_COLUMN_REDEFINITION == ddl_type) { str = "column redefinition"; } else if (DDL_TABLE_REDEFINITION == ddl_type) { str = "table redefinition"; } else if (DDL_MODIFY_AUTO_INCREMENT == ddl_type) { str = "modify auto_increment"; } else if (DDL_CONVERT_TO_CHARACTER == ddl_type) { str = "convert to character"; } else if (DDL_CHANGE_COLUMN_NAME == ddl_type) { str = "change column name"; } return str; } int ObDDLService::get_part_by_part_id( const ObPartitionSchema &partition_schema, const int64_t part_id, const ObPartition *&part) { int ret = OB_SUCCESS; part = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < partition_schema.get_partition_num(); i++) { const ObPartition *it = partition_schema.get_part_array()[i]; if (OB_NOT_NULL(it) && it->get_part_id() == part_id) { part = it; break; } } return ret; } int ObDDLService::update_global_index(ObAlterTableArg &arg, const uint64_t tenant_id, const ObTableSchema &orig_table_schema, ObDDLOperator &ddl_operator, const int64_t frozen_version, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; ObSEArray simple_index_infos; ObSchemaGetterGuard schema_guard; if (obrpc::ObAlterTableArg::DROP_PARTITION == arg.alter_part_type_ || obrpc::ObAlterTableArg::DROP_SUB_PARTITION == arg.alter_part_type_ || obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == arg.alter_part_type_ || obrpc::ObAlterTableArg::TRUNCATE_PARTITION == arg.alter_part_type_) { if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("valid schema service", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("fail to get tenant schema guard", K(ret), K(tenant_id)); } else if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get_index_tid_array failed", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema( tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", simple_index_infos.at(i).table_id_, K(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (index_table_schema->is_unavailable_index()) { ret = OB_NOT_SUPPORTED; LOG_WARN("drop/truncate partition before finishing index build not support", KR(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop/truncate partition before finishing index build "); } else if (!index_table_schema->can_read_index()) { // If the index is not available, the partition operation will not do any intervention } else if (index_table_schema->is_global_index_table()) { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2276) { ret = OB_NOT_SUPPORTED; LOG_WARN("drop/truncate partition with global indexes not support", KR(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop/truncate partition with global indexes "); } else if (!arg.is_update_global_indexes_) { if (OB_FAIL(ddl_operator.update_index_status( tenant_id, index_table_schema->get_data_table_id(), index_table_schema->get_table_id(), INDEX_STATUS_UNUSABLE, orig_table_schema.get_in_offline_ddl_white_list(), trans))) { LOG_WARN("update_index_status failed", K(index_table_schema->get_data_table_id())); } } else { ObTableSchema new_table_schema; if (OB_FAIL(new_table_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(rebuild_index_in_trans(schema_guard, new_table_schema, frozen_version, NULL, &trans))) { LOG_WARN("ddl_service_ rebuild_index failed", KR(ret)); } else { ObSArray &index_arg_list = arg.index_arg_list_; void *tmp_ptr = NULL; obrpc::ObCreateIndexArg *create_index_arg = NULL; if (NULL == (tmp_ptr = (ObCreateIndexArg*)arg.allocator_.alloc(sizeof(ObCreateIndexArg)))) { ret = OB_ALLOCATE_MEMORY_FAILED; SHARE_LOG(ERROR, "failed to alloc memory!", KR(ret)); } else { create_index_arg = new (tmp_ptr)ObCreateIndexArg(); create_index_arg->index_action_type_ = ObIndexArg::REBUILD_INDEX; create_index_arg->index_type_ = new_table_schema.get_index_type(); if (OB_FAIL(create_index_arg->index_schema_.assign(new_table_schema))) { LOG_WARN("fail to assign index schema", KR(ret), K(new_table_schema)); } else if (OB_FAIL(index_arg_list.push_back(create_index_arg))) { LOG_WARN("push back to index_arg_list failed", KR(ret), K(create_index_arg)); } } } } } } } } return ret; } int ObDDLService::fill_interval_info_for_set_interval(const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema, AlterTableSchema &inc_table_schema) { int ret = OB_SUCCESS; ObPartition **part_array = orig_table_schema.get_part_array(); int64_t part_num = orig_table_schema.get_partition_num(); if (OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to part_array is null", K(orig_table_schema), K(inc_table_schema), KR(ret)); } else if (part_num < 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part num is less 1", K(orig_table_schema), K(inc_table_schema), KR(ret)); } else if (OB_ISNULL(part_array[part_num - 1])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("the last part is null", K(orig_table_schema), K(inc_table_schema), KR(ret)); } else if (OB_FAIL(new_table_schema.set_transition_point(part_array[part_num - 1]->get_high_bound_val()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to set_transition_point", K(orig_table_schema), K(inc_table_schema),KR(ret)); } else if (OB_FAIL(new_table_schema.set_interval_range(inc_table_schema.get_interval_range()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to new_table_schema.set_interval_range", K(orig_table_schema), K(inc_table_schema),KR(ret)); } else { new_table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_INTERVAL); } return ret; } int ObDDLService::fill_interval_info_for_offline(const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema) { int ret = OB_SUCCESS; if (new_table_schema.is_interval_part()) { ObPartition **part_array = orig_table_schema.get_part_array(); int64_t part_num = orig_table_schema.get_partition_num(); if (OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to part_array is null", K(orig_table_schema), KR(ret)); } else if (part_num < 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part num is less 1", K(orig_table_schema), KR(ret)); } else if (OB_ISNULL(part_array[part_num - 1])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("the last part is null", K(orig_table_schema), KR(ret)); } else if (OB_FAIL(new_table_schema.set_transition_point(part_array[part_num - 1]->get_high_bound_val()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to set_transition_point", K(orig_table_schema), KR(ret)); } else if (OB_FAIL(new_table_schema.set_interval_range(orig_table_schema.get_interval_range()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to new_table_schema.set_interval_range", K(orig_table_schema), KR(ret)); } else { new_table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_INTERVAL); } } return ret; } //For truncate part/subpart, inc_table_schema and del_table_schema should be different in the later process. //For other situations, del_table_schema is useless and equal to inc_table_schema. int ObDDLService::generate_tables_array(const ObAlterTableArg::AlterPartitionType op_type, ObIArray &orig_table_schemas, ObIArray &new_table_schemas, ObIArray &inc_table_schemas, ObIArray &del_table_schemas, const ObTableSchema &orig_table_schema, ObTableSchema &new_table_schema, AlterTableSchema &inc_table_schema, ObSchemaGetterGuard &schema_guard, ObArenaAllocator &allocator) { int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObSEArray aux_table_ids; ObSEArray simple_index_infos; AlterTableSchema tmp_inc_table_schema; bool modify_sub_part_template_flags = false; if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level() && orig_table_schema.sub_part_template_def_valid() && (obrpc::ObAlterTableArg::ADD_PARTITION == op_type || obrpc::ObAlterTableArg::ADD_SUB_PARTITION == op_type || obrpc::ObAlterTableArg::DROP_SUB_PARTITION == op_type)) { // sub_part_template_def_valid() is only used for schema printer. // To simplify relate logic, we consider that add partition/subpartition or add subpartition // make cause partitions different. modify_sub_part_template_flags = true; new_table_schema.unset_sub_part_template_def_valid(); } if (!orig_table_schema.has_tablet() || orig_table_schema.is_index_local_storage() || orig_table_schema.is_aux_lob_table()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("table_schema must be data table or global indexes", KR(ret), K(orig_table_schema)); } else if (OB_FAIL(orig_table_schemas.push_back(&orig_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), K(orig_table_schema)); } else if (OB_FAIL(new_table_schemas.push_back(&new_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), K(new_table_schema)); } else if (OB_FAIL(tmp_inc_table_schema.assign(new_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), K(new_table_schema)); } else if (OB_FAIL(tmp_inc_table_schema.assign_partition_schema(inc_table_schema))) { LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret)); } else if (OB_FAIL(tmp_inc_table_schema.set_transition_point(inc_table_schema.get_transition_point()))) { LOG_WARN("fail to set transition point", K(ret)); } else if (OB_FAIL(tmp_inc_table_schema.set_interval_range(inc_table_schema.get_interval_range()))) { LOG_WARN("fail to set interval range", K(ret)); } else if (OB_FAIL(inc_table_schema.assign(tmp_inc_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), K(new_table_schema)); } else if (OB_FAIL(inc_table_schemas.push_back(&inc_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), K(inc_table_schema)); } else if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get_simple_index_infos failed", KR(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { if (OB_FAIL(aux_table_ids.push_back(simple_index_infos.at(i).table_id_))) { LOG_WARN("fail to push back index table id", KR(ret)); } } if (OB_SUCC(ret)) { uint64_t mtid = orig_table_schema.get_aux_lob_meta_tid(); uint64_t ptid = orig_table_schema.get_aux_lob_piece_tid(); if (!((mtid != OB_INVALID_ID && ptid != OB_INVALID_ID) || (mtid == OB_INVALID_ID && ptid == OB_INVALID_ID))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("Expect meta tid and piece tid both valid or both invalid", KR(ret), K(mtid), K(ptid)); } else if (OB_INVALID_ID != mtid && OB_FAIL(aux_table_ids.push_back(mtid))) { LOG_WARN("fail to push back lob meta tid", KR(ret), K(mtid)); } else if (OB_INVALID_ID != ptid && OB_FAIL(aux_table_ids.push_back(ptid))) { LOG_WARN("fail to push back lob piece tid", KR(ret), K(ptid)); } } for (int64_t i = 0; OB_SUCC(ret) && i < aux_table_ids.count(); ++i) { const ObTableSchema *orig_aux_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, aux_table_ids.at(i), orig_aux_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), K(aux_table_ids.at(i)), KR(ret)); } else if (OB_ISNULL(orig_aux_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", KR(ret)); } else if (orig_aux_table_schema->is_index_local_storage() || orig_aux_table_schema->is_aux_lob_table()) { ObTableSchema *new_aux_table_schema = NULL; AlterTableSchema *inc_aux_table_schema = NULL; void *new_schema_ptr = allocator.alloc(sizeof(ObTableSchema)); void *inc_schema_ptr = allocator.alloc(sizeof(AlterTableSchema)); if (OB_ISNULL(new_schema_ptr) || OB_ISNULL(inc_schema_ptr)) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail alloc memory", KR(ret), KP(new_schema_ptr), KP(inc_schema_ptr)); } else { new_aux_table_schema = new (new_schema_ptr)ObTableSchema(&allocator); inc_aux_table_schema = new (inc_schema_ptr)AlterTableSchema(&allocator); } if (OB_FAIL(ret)) { } else if (OB_FAIL(new_aux_table_schema->assign(*orig_aux_table_schema))) { LOG_WARN("fail to assign schema", KR(ret), KPC(orig_aux_table_schema)); } else if (modify_sub_part_template_flags && FALSE_IT(new_aux_table_schema->unset_sub_part_template_def_valid())) { } else if (OB_FAIL(inc_aux_table_schema->assign(*new_aux_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), KPC(new_aux_table_schema)); } else if (OB_FAIL(inc_aux_table_schema->assign_partition_schema(inc_table_schema))) { LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret)); } else if (OB_FAIL(inc_aux_table_schema->set_transition_point(inc_table_schema.get_transition_point()))) { LOG_WARN("fail to set transition point", K(ret)); } else if (OB_FAIL(inc_aux_table_schema->set_interval_range(inc_table_schema.get_interval_range()))) { LOG_WARN("fail to set interval range", K(ret)); } else if (OB_FAIL(new_table_schemas.push_back(new_aux_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), K(new_aux_table_schema)); } else if (OB_FAIL(orig_table_schemas.push_back(orig_aux_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), K(orig_aux_table_schema)); } else if (OB_FAIL(inc_table_schemas.push_back(inc_aux_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), KPC(inc_aux_table_schema)); } } } if (OB_FAIL(ret)) { } else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == op_type || obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == op_type) { // truncate part/subpart may cause del_table_schema and inc_table_schema for (int i = 0; OB_SUCC(ret) && i < inc_table_schemas.count(); i++) { AlterTableSchema *del_table_schema = NULL; void *del_schema_ptr = allocator.alloc(sizeof(AlterTableSchema)); if (OB_ISNULL(del_schema_ptr)) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail alloc memory", KR(ret)); } else if (FALSE_IT(del_table_schema = new (del_schema_ptr)AlterTableSchema(&allocator))) { } else if (OB_FAIL(del_table_schema->assign(*inc_table_schemas.at(i)))) { LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret)); } else if (OB_FAIL(del_table_schemas.push_back(del_table_schema))) { LOG_WARN("failed to push back table_schema", KR(ret), KPC(del_table_schema)); } } } // for other situations, del_table_schemas is useless and equal to inc_table_schemas else if (OB_FAIL(del_table_schemas.assign(inc_table_schemas))) { LOG_WARN("failed to assign inc_table_schemas", KR(ret), K(inc_table_schemas)); } return ret; } int ObDDLService::reset_interval_info_for_interval_to_range(ObTableSchema &new_table_schema) { int ret = OB_SUCCESS; ObRowkey null_row_key; null_row_key.reset(); if (OB_FAIL(new_table_schema.set_transition_point(null_row_key))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to set_transition_point", K(new_table_schema), KR(ret)); } else if (OB_FAIL(new_table_schema.set_interval_range(null_row_key))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to new_table_schema.set_interval_range", K(new_table_schema), KR(ret)); } else { new_table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_RANGE_COLUMNS); } return ret; } int ObDDLService::alter_tables_partitions(const obrpc::ObAlterTableArg &alter_table_arg, ObIArray &orig_table_schemas, ObIArray &new_table_schemas, ObIArray &inc_table_schemas, ObIArray &del_table_schemas, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; if (orig_table_schemas.count() != new_table_schemas.count()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("orig_table_schemas.count() new_table_schemas.count() is not equal", KR(ret), K(orig_table_schemas), K(new_table_schemas)); } else if (orig_table_schemas.count() != inc_table_schemas.count()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("orig_table_schemas.count() inc_table_schemas.count() is not equal", KR(ret), K(orig_table_schemas), K(inc_table_schemas)); } else if (orig_table_schemas.count() != del_table_schemas.count()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("orig_table_schemas.count() inc_table_schemas.count() is not equal", KR(ret), K(orig_table_schemas), K(del_table_schemas)); } for (int64_t i = 0; OB_SUCC(ret) && i < new_table_schemas.count(); ++i) { // todo fill AlterTableSchema for splitting partition with inc_table_schema if (OB_ISNULL(new_table_schemas.at(i)) || OB_ISNULL(inc_table_schemas.at(i)) || OB_ISNULL(orig_table_schemas.at(i)) || OB_ISNULL(del_table_schemas.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema ptr is NULL", KR(ret), K(new_table_schemas.at(i)), K(inc_table_schemas.at(i)), K(orig_table_schemas.at(i)), K(del_table_schemas.at(i))); } else if (OB_FAIL(alter_table_partitions(alter_table_arg, *orig_table_schemas.at(i), *inc_table_schemas.at(i), *del_table_schemas.at(i), *new_table_schemas.at(i), ddl_operator, trans))) { LOG_WARN("alter table partitions failed", KR(ret), K(i), KPC(new_table_schemas.at(i)), KPC(inc_table_schemas.at(i))); } } return ret; } int ObDDLService::alter_table_partitions(const obrpc::ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, AlterTableSchema &inc_table_schema, AlterTableSchema &del_table_schema, ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { DEBUG_SYNC(BEFORE_ALTER_TABLE_PARTITION); int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); const int64_t table_id = orig_table_schema.get_table_id(); const obrpc::ObAlterTableArg::AlterPartitionType &op_type = alter_table_arg.alter_part_type_; int64_t schema_version = OB_INVALID_VERSION; if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("valid schema service", K(ret)); } else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == op_type || obrpc::ObAlterTableArg::REORGANIZE_PARTITION == op_type || obrpc::ObAlterTableArg::SPLIT_PARTITION == op_type) { ret = OB_NOT_SUPPORTED; LOG_WARN("split table partitions is not supported", KR(ret), K(orig_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "split table partitions is"); } else if (obrpc::ObAlterTableArg::ADD_PARTITION == op_type) { if (OB_FAIL(gen_inc_table_schema_for_add_part(orig_table_schema, inc_table_schema))) { LOG_WARN("fail to gen inc table schema for add part", KR(ret), K(orig_table_schema), K(inc_table_schema)); } else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema)); } else if (OB_FAIL(generate_tablet_id(inc_table_schema))) { LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret)); } else if (OB_FAIL(ddl_operator.add_table_partitions(orig_table_schema, inc_table_schema, new_table_schema, trans))) { LOG_WARN("failed to add table partitions", KR(ret)); } } else if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == op_type) { if (OB_FAIL(gen_inc_table_schema_for_add_subpart(orig_table_schema, inc_table_schema))) { LOG_WARN("fail to gen inc table schema for add subpart", KR(ret), K(orig_table_schema), K(inc_table_schema)); } else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema, true))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema)); } else if (OB_FAIL(generate_tablet_id(inc_table_schema))) { LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret)); } else if (OB_FAIL(ddl_operator.add_table_subpartitions(orig_table_schema, inc_table_schema, new_table_schema, trans))) { LOG_WARN("failed to add table partitions", KR(ret)); } } else if (obrpc::ObAlterTableArg::DROP_PARTITION == op_type) { if (OB_FAIL(gen_inc_table_schema_for_drop_part(orig_table_schema, inc_table_schema))) { LOG_WARN("fail to gen inc table schema for drop part", KR(ret), K(orig_table_schema), K(inc_table_schema)); } else if (OB_FAIL(lock_partitions(trans, inc_table_schema))) { LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(inc_table_schema)); // for ddl retry task, upper layer only focus on `OB_TRY_LOCK_ROW_CONFLICT`, and then retry it. const bool is_ddl_scheduled_task = alter_table_arg.task_id_ > 0 ? true : false; ret = is_ddl_scheduled_task && ObDDLUtil::is_table_lock_retry_ret_code(ret) ? OB_TRY_LOCK_ROW_CONFLICT : ret; } else if (OB_FAIL(ddl_operator.drop_table_partitions(orig_table_schema, inc_table_schema, new_table_schema, trans))) { LOG_WARN("failed to drop table partitions", KR(ret)); } } else if (obrpc::ObAlterTableArg::DROP_SUB_PARTITION == op_type) { if (inc_table_schema.get_partition_num() != 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("drop subparts not in a part", KR(ret)); } else if (OB_FAIL(gen_inc_table_schema_for_drop_subpart(orig_table_schema, inc_table_schema))) { LOG_WARN("fail to gen inc table for drop subpart", KR(ret), K(orig_table_schema), K(inc_table_schema)); } else if (OB_FAIL(lock_partitions(trans, inc_table_schema))) { LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(inc_table_schema)); // for ddl retry task, upper layer only focus on `OB_TRY_LOCK_ROW_CONFLICT`, and then retry it. const bool is_ddl_scheduled_task = alter_table_arg.task_id_ > 0 ? true : false; ret = is_ddl_scheduled_task && ObDDLUtil::is_table_lock_retry_ret_code(ret) ? OB_TRY_LOCK_ROW_CONFLICT : ret; } else if (OB_FAIL(ddl_operator.drop_table_subpartitions(orig_table_schema, inc_table_schema, new_table_schema, trans))) { LOG_WARN("failed to drop table partitions", KR(ret)); } } else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == op_type) { if (OB_FAIL(gen_inc_table_schema_for_trun_part( orig_table_schema, inc_table_schema, del_table_schema))) { LOG_WARN("fail to generate inc table schema", KR(ret), K(orig_table_schema)); } else if (OB_FAIL(lock_partitions(trans, del_table_schema))) { LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(del_table_schema)); // for ddl retry task, upper layer only focus on `OB_TRY_LOCK_ROW_CONFLICT`, and then retry it. const bool is_ddl_scheduled_task = alter_table_arg.task_id_ > 0 ? true : false; ret = is_ddl_scheduled_task && ObDDLUtil::is_table_lock_retry_ret_code(ret) ? OB_TRY_LOCK_ROW_CONFLICT : ret; } else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema)); } else if (OB_FAIL(generate_tablet_id(inc_table_schema))) { LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret)); } else if (OB_FAIL(ddl_operator.truncate_table_partitions(orig_table_schema, inc_table_schema, del_table_schema, trans))) { LOG_WARN("failed to truncate partitions", KR(ret)); } } else if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == op_type) { if (OB_FAIL(gen_inc_table_schema_for_trun_subpart( orig_table_schema, inc_table_schema, del_table_schema))) { LOG_WARN("fail to generate inc table schema", KR(ret), K(orig_table_schema)); } else if (OB_FAIL(lock_partitions(trans, del_table_schema))) { LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(del_table_schema)); // for ddl retry task, upper layer only focus on `OB_TRY_LOCK_ROW_CONFLICT`, and then retry it. const bool is_ddl_scheduled_task = alter_table_arg.task_id_ > 0 ? true : false; ret = is_ddl_scheduled_task && ObDDLUtil::is_table_lock_retry_ret_code(ret) ? OB_TRY_LOCK_ROW_CONFLICT : ret; } else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema, true))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema)); } else if (OB_FAIL(generate_tablet_id(inc_table_schema))) { LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret)); } else if (OB_FAIL(ddl_operator.truncate_table_subpartitions(orig_table_schema, inc_table_schema, del_table_schema, trans))) { LOG_WARN("failed to drop table partitions", KR(ret)); } } else if (obrpc::ObAlterTableArg::SET_INTERVAL == op_type) { // interval part table: modify interval // range part table: switch interval part table if (OB_FAIL(fill_interval_info_for_set_interval(orig_table_schema, new_table_schema, inc_table_schema))) { LOG_WARN("failed to fill interval info for set interval", KR(ret), K(orig_table_schema), K(inc_table_schema)); } } else if (obrpc::ObAlterTableArg::INTERVAL_TO_RANGE == op_type) { if (OB_FAIL(reset_interval_info_for_interval_to_range(new_table_schema))) { LOG_WARN("failed to reset interval info for interval to range", KR(ret), K(orig_table_schema), K(inc_table_schema)); } } return ret; } int ObDDLService::check_enable_sys_table_ddl(const ObTableSchema &table_schema, const ObSchemaOperationType operation_type) { int ret = OB_SUCCESS; if (is_inner_table(table_schema.get_table_id())) { const bool enable_sys_table_ddl = common::ObServerConfig::get_instance().enable_sys_table_ddl; char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE]; if (!enable_sys_table_ddl) { ret = OB_OP_NOT_ALLOW; switch(operation_type) { case OB_DDL_CREATE_TABLE: { (void)snprintf(err_msg, sizeof(err_msg), "%s", "create system table"); break; } case OB_DDL_ALTER_TABLE: { (void)snprintf(err_msg, sizeof(err_msg), "%s", "alter system table"); break; } case OB_DDL_TABLE_RENAME: { (void)snprintf(err_msg, sizeof(err_msg), "%s", "rename system table"); break; } case OB_DDL_DROP_TABLE: { if (table_schema.is_view_table()) { (void)snprintf(err_msg, sizeof(err_msg), "%s", "drop system view"); } else { (void)snprintf(err_msg, sizeof(err_msg), "%s", "drop system table"); } break; } case OB_DDL_TRUNCATE_TABLE_CREATE: { (void)snprintf(err_msg, sizeof(err_msg), "%s", "truncate system table"); break; } default : { (void)snprintf(err_msg, sizeof(err_msg), "%s", "ddl on system table"); } } } else if (OB_DDL_DROP_TABLE == operation_type && table_schema.has_partition()) { ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s", "drop partitioned system table"); } else if (OB_DDL_DROP_COLUMN == operation_type && is_sys_table(table_schema.get_table_id())) { ret = OB_OP_NOT_ALLOW; (void)snprintf(err_msg, sizeof(err_msg), "%s", "system table drop column"); } if (OB_FAIL(ret)) { LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg); } } return ret; } // FIXME: this function should move to observer int ObDDLService::alter_table_sess_active_time_in_trans(obrpc::ObAlterTableArg &alter_table_arg, const int64_t frozen_version, obrpc::ObAlterTableRes &res) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; alter_table_schema.set_sess_active_time(ObTimeUtility::current_time()); ObArray tenant_ids; ObDDLType &ddl_type = res.ddl_type_; if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; } else if (OB_FAIL(alter_table_schema.alter_option_bitset_.add_member(obrpc::ObAlterTableArg::SESSION_ACTIVE_TIME))) { LOG_WARN("failed to add member SESSION_ACTIVE_TIME for alter table schema", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_ids(tenant_ids))) { LOG_WARN("fail to get schema guard", K(ret)); } else { ObSchemaGetterGuard schema_guard; ObSEArray table_schemas; bool found = false; FOREACH_CNT_X(tenant_id, tenant_ids, OB_SUCC(ret) && !found) { if (OB_FAIL(schema_service_->get_tenant_schema_guard(*tenant_id, schema_guard))) { LOG_WARN("fail to get tenant schema guard", KR(ret), "tenant_id", *tenant_id); } else if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(*tenant_id, table_schemas))) { LOG_WARN("fail to get table schema", K(ret), "tenant_id", *tenant_id); } else { const ObDatabaseSchema *database_schema = NULL; for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); i++) { const ObSimpleTableSchemaV2 *table_schema = table_schemas.at(i); if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("got invalid schema", K(ret), K(i)); } else if (table_schema->is_tmp_table() && alter_table_arg.session_id_ == table_schema->get_session_id()) { // session_id must belong to one tenant found = true; database_schema = NULL; if (OB_FAIL(schema_guard.get_database_schema(*tenant_id, table_schema->get_database_id(), database_schema))) { LOG_WARN("failed to get database schema", K(ret), "tenant_id", *tenant_id); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database schema is null", K(ret)); } else if (database_schema->is_in_recyclebin() || table_schema->is_in_recyclebin()) { LOG_INFO("skip table schema in recyclebin", K(*table_schema)); } else { alter_table_schema.set_origin_database_name(database_schema->get_database_name()); alter_table_schema.set_origin_table_name(table_schema->get_table_name()); alter_table_schema.set_tenant_id(table_schema->get_tenant_id()); if (OB_FAIL(check_is_offline_ddl(alter_table_arg, res.ddl_type_))) { LOG_WARN("failed to to check is offline ddl", K(ret)); } else { // offline ddl cannot appear at the same time with other ddl types if (is_long_running_ddl(res.ddl_type_)) { if (OB_FAIL(do_offline_ddl_in_trans(alter_table_arg, frozen_version, res))) { LOG_WARN("failed to do offline ddl in trans", K(ret), K(alter_table_arg));; } } else { if (OB_FAIL(alter_table_in_trans(alter_table_arg, frozen_version, res))) { LOG_WARN("refresh sess active time of temporary table failed", K(alter_table_arg), K(ret)); } else { LOG_INFO("a temporary table just refreshed sess active time", K(alter_table_arg)); } } } } } } // end for } } // end FOREACH_CNT_X } return ret; } int ObDDLService::update_tables_attribute(ObIArray &new_table_schemas, ObDDLOperator &ddl_operator, common::ObMySQLTransaction &trans, const ObSchemaOperationType operation_type, const ObString &ddl_stmt_str) { int ret = OB_SUCCESS; int64_t schema_count = new_table_schemas.count(); if (schema_count < 1) { ret = OB_INVALID_ARGUMENT; LOG_WARN("local index should exist", K(new_table_schemas), KR(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < schema_count; ++i) { if (OB_ISNULL(new_table_schemas.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema ptr is NULL", KR(ret)); } else if (OB_FAIL(ddl_operator.update_table_attribute(*new_table_schemas.at(i), trans, operation_type, 0 == i ? &ddl_stmt_str : NULL))) { LOG_WARN("failed to update data table schema version and max used column is!", KR(ret), KPC(new_table_schemas.at(i))); } } return ret; } //fix me :Check whether the newly added index column covers the partition column --by rongxuan.lc // It can be repaired after the featrue that add index in alter_table statement int ObDDLService::alter_table_in_trans(obrpc::ObAlterTableArg &alter_table_arg, const int64_t frozen_version, obrpc::ObAlterTableRes &res) { int ret = OB_SUCCESS; const ObDDLType ddl_type = res.ddl_type_; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; HEAP_VAR(ObArray, ddl_tasks) { HEAP_VAR(ObTableSchema, new_table_schema) { HEAP_VAR(ObAlterTableArg, const_alter_table_arg) { HEAP_VAR(ObMockFKParentTableSchema, mock_fk_parent_table_schema) { ObSchemaGetterGuard schema_guard; const ObTableSchema *orig_table_schema = NULL; const ObTenantSchema *tenant_schema = NULL; uint64_t tenant_id = alter_table_schema.get_tenant_id(); schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; ObArenaAllocator allocator; ObArray mock_fk_parent_table_schema_array; bool need_modify_notnull_validate = false; bool is_add_not_null_col = false; ObString empty_stmt; const bool need_deep_copy_arg = alter_table_arg.alter_constraint_type_ == ObAlterTableArg::ADD_CONSTRAINT || alter_table_arg.alter_constraint_type_ == ObAlterTableArg::ALTER_CONSTRAINT_STATE || alter_table_arg.foreign_key_arg_list_.count() > 0; // because the sql optimizer is using validate flag to generate optimized plan, // if add or modify a foregin key to validate state, it should take effect after checking whether the data is satisfied with the foreigin key if (need_deep_copy_arg && OB_FAIL(ObConstraintTask::deep_copy_table_arg(allocator, alter_table_arg, const_alter_table_arg))) { LOG_WARN("deep copy table arg failed", K(ret)); } else if (alter_table_arg.foreign_key_checks_ && 1 == alter_table_arg.foreign_key_arg_list_.count() && alter_table_arg.foreign_key_arg_list_.at(0).need_validate_data_) { if ((!alter_table_arg.foreign_key_arg_list_.at(0).is_modify_fk_state_ && alter_table_arg.foreign_key_arg_list_.at(0).validate_flag_) || (alter_table_arg.foreign_key_arg_list_.at(0).is_modify_validate_flag_ && alter_table_arg.foreign_key_arg_list_.at(0).validate_flag_)) { alter_table_arg.ddl_stmt_str_ = empty_stmt; alter_table_arg.foreign_key_arg_list_.at(0).validate_flag_ = CST_FK_NO_VALIDATE; } } else if (OB_FAIL(need_modify_not_null_constraint_validate( alter_table_arg, is_add_not_null_col, need_modify_notnull_validate))) { LOG_WARN("check need modify not null constraint validate failed", K(ret)); } else if (need_modify_notnull_validate) { alter_table_arg.ddl_stmt_str_ = empty_stmt; ObConstraint *cst = *alter_table_arg.alter_table_schema_.constraint_begin_for_non_const_iter(); cst->set_validate_flag(CST_FK_NO_VALIDATE); cst->set_is_modify_validate_flag(false); const uint64_t col_id = *(cst->cst_col_begin()); ObColumnSchemaV2 *col_schema = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.alter_table_schema_.get_column_count(); i++) { if (alter_table_arg.alter_table_schema_.get_column_schema_by_idx(i)->get_column_id() == col_id) { col_schema = alter_table_arg.alter_table_schema_.get_column_schema_by_idx(i); } } if (OB_ISNULL(col_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column schema not found", K(ret), K(alter_table_arg)); } else { col_schema->del_column_flag(NOT_NULL_VALIDATE_FLAG); } } else if (is_add_not_null_col) { for (int64_t i = 0; i < alter_table_arg.alter_table_schema_.get_column_count() && OB_SUCC(ret); i++) { ObColumnSchemaV2 *col_schema = NULL; if (OB_ISNULL(col_schema = alter_table_arg.alter_table_schema_.get_column_schema_by_idx(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column schema is null", K(ret)); } else if (OB_DDL_ADD_COLUMN == static_cast(col_schema)->alter_type_) { col_schema->set_is_hidden(true); } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(get_and_check_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema))) { LOG_WARN("fail to get and check table schema", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id)); } else if (OB_UNLIKELY(NULL == tenant_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant schema is null", K(ret), KP(tenant_schema), K(tenant_id)); } else if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { bool need_update_index_table = false; AlterLocalityOp alter_locality_op = ALTER_LOCALITY_OP_INVALID; if (alter_table_arg.is_alter_columns_ || (alter_table_arg.is_alter_options_ && alter_table_arg.need_progressive_merge())) { if (alter_table_arg.alter_table_schema_.alter_option_bitset_. has_member(ObAlterTableArg::ENCRYPTION) && alter_table_arg.alter_table_schema_.is_equal_encryption(*orig_table_schema)) { // If the values before and after changing the encryption algorithm in the table are the same, // the merge is not marked } else { alter_table_arg.is_alter_options_ = true; alter_table_arg.alter_table_schema_.set_progressive_merge_round(orig_table_schema->get_progressive_merge_round() + 1); if (OB_FAIL(alter_table_arg.alter_table_schema_.alter_option_bitset_.add_member(ObAlterTableArg::PROGRESSIVE_MERGE_ROUND))) { LOG_WARN("fail to add member progressive merge round", K(ret)); } } } if (OB_SUCC(ret)) { // check foreign key info and check constraint // and put foreign_key_info into table schema if (OB_FAIL(deal_with_cst_for_alter_table(schema_guard, tenant_id, orig_table_schema, alter_table_arg, mock_fk_parent_table_schema))) { LOG_WARN("deal with cst for alter table failed", K(ret)); } else if (MOCK_FK_PARENT_TABLE_OP_INVALID != mock_fk_parent_table_schema.get_operation_type() && OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) { // scence : alter table add fk referenced a parent table which is not exist LOG_WARN("push back to mock_fk_parent_table_schema_array failed", K(ret), K(mock_fk_parent_table_schema)); } else if (OB_FAIL(set_new_table_options(alter_table_arg, alter_table_schema, *tenant_schema, new_table_schema, *orig_table_schema, schema_guard, need_update_index_table, alter_locality_op))) { LOG_WARN("failed to set new table options", K(ret), K(new_table_schema), K(*orig_table_schema), K(ret)); } } ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(ret)) { //do nothing } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObArray global_idx_schema_array; //table columns if (OB_SUCC(ret) && alter_table_arg.is_alter_columns_) { if (OB_FAIL(check_can_alter_column(tenant_id, alter_table_schema, *orig_table_schema))) { LOG_WARN("fail to can alter column", K(ret), K(alter_table_arg)); } else if (OB_FAIL(alter_table_column(*orig_table_schema, alter_table_schema, frozen_version, new_table_schema, alter_table_arg, schema_guard, ddl_operator, trans, &global_idx_schema_array))) { LOG_WARN("failed to alter table column!", K(*orig_table_schema), K(new_table_schema), K(ret)); } } //table options // if there is no auto-increment column, ignore table option auto_increment if (0 == new_table_schema.get_autoinc_column_id()) { new_table_schema.set_auto_increment(1); } if (OB_SUCC(ret) && OB_FAIL(ddl_operator.alter_table_options( schema_guard, new_table_schema, *orig_table_schema, need_update_index_table, trans, &global_idx_schema_array))) { ObString origin_table_name = alter_table_schema.get_origin_table_name(); LOG_WARN("failed to alter table options,", K(origin_table_name), K(ret)); } // table foreign key if (OB_SUCC(ret) && !alter_table_arg.alter_table_schema_.get_foreign_key_infos().empty()) { if (OB_FAIL(alter_table_foreign_keys( *orig_table_schema, alter_table_schema, ddl_operator, trans))) { LOG_WARN("alter table foreign keys failed", K(ret)); } } if (alter_table_arg.has_rename_action()) { OZ (rebuild_trigger_package(schema_guard, new_table_schema, ddl_operator, trans), new_table_schema.get_table_name_str()); } //table indexs if (OB_SUCC(ret) && alter_table_arg.is_alter_indexs_) { if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) { LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id), K(orig_table_schema->get_table_id())); } else if (OB_FAIL(alter_table_index(alter_table_arg, *orig_table_schema, new_table_schema, schema_guard, frozen_version, ddl_operator, trans, alter_table_arg.allocator_, res, ddl_tasks))) { LOG_WARN("failed to alter table index!", K(ret)); } } // table constraints if (OB_SUCC(ret) && alter_table_arg.alter_constraint_type_ != obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION) { if (OB_FAIL(alter_table_constraints( alter_table_arg.alter_constraint_type_, schema_guard, *orig_table_schema, alter_table_schema, new_table_schema, ddl_operator, trans))) { LOG_WARN("alter table constraints failed", K(ret)); } } //table partitions // the first element is data_table_schema // the others element are local_index_schemas ObArray orig_table_schemas; ObArray new_table_schemas; // 1. truncate part/subpart: del_table_schemas will be deleted and inc_table_schemas will be added // 2. others: inc_table_schemas and del_table_schemas is pointed same partitions ObArray inc_table_schemas; ObArray del_table_schemas; ObArenaAllocator allocator("ModifyPart"); //table partitons ObArray new_partition_ids; if (OB_SUCC(ret) && alter_table_arg.is_alter_partitions_) { if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) { LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id), K(orig_table_schema->get_table_id())); } else if (OB_FAIL(update_global_index(alter_table_arg, tenant_id, *orig_table_schema, ddl_operator, frozen_version, trans))) { LOG_WARN("update_global_index failed", K(ret)); } else if (OB_FAIL(generate_tables_array(alter_table_arg.alter_part_type_, orig_table_schemas, new_table_schemas, inc_table_schemas, del_table_schemas, *orig_table_schema, new_table_schema, alter_table_schema, schema_guard, allocator))) { } else if (OB_FAIL(alter_tables_partitions(alter_table_arg, orig_table_schemas, new_table_schemas, inc_table_schemas, del_table_schemas, ddl_operator, trans))) { LOG_WARN("alter table partitions failed", K(ret)); } else if (orig_table_schemas.count() != new_table_schemas.count() || inc_table_schemas.count() != orig_table_schemas.count() || del_table_schemas.count() != orig_table_schemas.count() || inc_table_schemas.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("array count is unexpected" , K(orig_table_schemas), K(new_table_schemas), K(inc_table_schemas), K(del_table_schemas), KR(ret)); } else if (alter_table_arg.task_id_ > 0 && OB_FAIL(ObDDLRetryTask::update_task_status_succ(trans, tenant_id, alter_table_arg.task_id_))) { LOG_WARN("update ddl task status failed", K(ret)); } } if (OB_SUCC(ret)) { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_PARTITIONED_TABLE; } else if (obrpc::ObAlterTableArg::SPLIT_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::REORGANIZE_PARTITION == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_SPLIT_PARTITION; } else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_TRUNCATE_PARTITION; } else if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_ADD_SUB_PARTITION; } else if (obrpc::ObAlterTableArg::DROP_SUB_PARTITION == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_DROP_SUB_PARTITION; } else if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_TRUNCATE_SUB_PARTITION; } else if (obrpc::ObAlterTableArg::SET_INTERVAL == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_SET_INTERVAL; } else if (obrpc::ObAlterTableArg::INTERVAL_TO_RANGE == alter_table_arg.alter_part_type_) { operation_type = OB_DDL_INTERVAL_TO_RANGE; } if (!alter_table_arg.is_alter_partitions_) { if (OB_FAIL(ddl_operator.update_table_attribute( new_table_schema, trans, operation_type, &alter_table_arg.ddl_stmt_str_))) { LOG_WARN("failed to update tablets attribute", K(ret), K(new_table_schema)); } } else { if (OB_FAIL(update_tables_attribute( new_table_schemas, ddl_operator, trans, operation_type, alter_table_arg.ddl_stmt_str_))) { LOG_WARN("failed to update tablets attribute", K(ret), K(new_table_schema)); } } if (OB_FAIL(ret)) { } else if (alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::SESSION_ID) && 0 == new_table_schema.get_session_id() && !new_table_schema.is_tmp_table() && OB_FAIL(ddl_operator.delete_temp_table_info(trans, new_table_schema))) { LOG_WARN("failed to delete temp table info", K(ret)); } } // scence : alter table rename to a mock fk parent table existed, will replace mock fk parent table with real table if (OB_SUCC(ret) && alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::TABLE_NAME)) { const ObMockFKParentTableSchema *ori_mock_fk_parent_table_ptr = NULL; if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name( orig_table_schema->get_tenant_id(), new_table_schema.get_database_id(), new_table_schema.get_table_name_str(), ori_mock_fk_parent_table_ptr))) { LOG_WARN("get_mock_fk_parent_table_schema_with_name failed", K(ret), K(orig_table_schema->get_tenant_id()), K(new_table_schema.get_database_id()), K(new_table_schema.get_table_name_str())); } else if (OB_NOT_NULL(ori_mock_fk_parent_table_ptr)) { ObArray simple_index_infos; ObArray unique_index_schemas; if (OB_FAIL(orig_table_schema->get_simple_index_infos(simple_index_infos))) { SERVER_LOG(WARN, "get simple_index_infos without delay_deleted_tid failed", K(ret)); } else { const ObTableSchema *index_schema = NULL; for (int64_t j = 0; OB_SUCC(ret) && j < simple_index_infos.count(); ++j) { if (OB_FAIL(schema_guard.get_table_schema(alter_table_schema.get_tenant_id(), simple_index_infos.at(j).table_id_, index_schema))) { LOG_WARN("fail to get to_table_schema schema", K(ret)); } else if (OB_ISNULL(index_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index_schema is null", K(ret)); } else if (index_schema->is_unique_index() && OB_FAIL(unique_index_schemas.push_back(index_schema))) { LOG_WARN("fail to push_back index_schema to unique_index_schemas", K(ret)); } } if (OB_SUCC(ret)) { // alter table rename to mock fk parent table name with other actions is not supported // so we can use orig_table_schema to check and gen mock_fk_parent_table_for_replacing if (OB_FAIL(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table( schema_guard, ori_mock_fk_parent_table_ptr->get_mock_fk_parent_table_id(), *orig_table_schema, unique_index_schemas, mock_fk_parent_table_schema))) { LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table", K(ret)); } else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) { LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema)); } } } } } if (FAILEDx(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to deal_with_mock_fk_parent_tables", K(ret), K(mock_fk_parent_table_schema_array.count())); } int64_t last_schema_version = OB_INVALID_VERSION; if (OB_SUCC(ret)) { last_schema_version = ddl_operator.get_last_operation_schema_version(); if (OB_INVALID_VERSION == last_schema_version) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid schema version" , K(last_schema_version)); } } for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.index_arg_list_.size(); ++i) { const ObIndexArg *index_arg = alter_table_arg.index_arg_list_.at(i); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("index arg should not be null", K(ret)); } else if (index_arg->index_action_type_ == ObIndexArg::ADD_INDEX) { const ObCreateIndexArg *create_index_arg = static_cast(index_arg); const ObTableSchema &index_schema = create_index_arg->index_schema_; if (OB_FAIL(ddl_operator.insert_ori_schema_version( trans, tenant_id, index_schema.get_table_id(), last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(tenant_id)); } } } if (OB_SUCC(ret)) { common::ObArray inc_table_schema_ptrs; common::ObArray del_table_schema_ptrs; for (int i = 0; i < inc_table_schemas.count() && OB_SUCC(ret); i++) { const ObTableSchema *tmp_table_schema = inc_table_schemas.at(i); if (OB_ISNULL(tmp_table_schema) || OB_ISNULL(new_table_schemas.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schemas is NULL", KR(ret), K(i), K(tmp_table_schema)); } else if (FALSE_IT(inc_table_schemas.at(i)->set_schema_version(new_table_schemas.at(i)->get_schema_version()))) { } else if (OB_FAIL(inc_table_schema_ptrs.push_back(tmp_table_schema))) { LOG_WARN("fail to push back", KR(ret), KPC(tmp_table_schema)); } } for (int i = 0; i < del_table_schemas.count() && OB_SUCC(ret); i++) { const ObTableSchema *tmp_table_schema = del_table_schemas.at(i); if (OB_ISNULL(tmp_table_schema) || OB_ISNULL(new_table_schemas.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schemas is NULL", KR(ret), K(i), K(tmp_table_schema)); } else if (FALSE_IT(del_table_schemas.at(i)->set_schema_version(new_table_schemas.at(i)->get_schema_version()))) { } else if (OB_FAIL(del_table_schema_ptrs.push_back(tmp_table_schema))) { LOG_WARN("fail to push back", KR(ret), KPC(tmp_table_schema)); } } if (OB_FAIL(ret)) { } else if (obrpc::ObAlterTableArg::DROP_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::DROP_SUB_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) { int64_t new_schema_version = OB_INVALID_VERSION; if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id)); } else { ObTabletDrop tablet_drop(tenant_id, *GCTX.lst_operator_, trans, new_schema_version); if (OB_FAIL(tablet_drop.init())) { LOG_WARN("fail to init tablet drop", KR(ret), K(del_table_schema_ptrs)); } else if (OB_FAIL(tablet_drop.add_drop_tablets_of_table_arg(del_table_schema_ptrs))) { LOG_WARN("failed to add drop tablets", KR(ret), K(del_table_schema_ptrs)); } else if (OB_FAIL(tablet_drop.execute())) { LOG_WARN("failed to execute", KR(ret), K(del_table_schema_ptrs)); } } } if (OB_FAIL(ret)) { } else if (obrpc::ObAlterTableArg::ADD_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) { int64_t frozen_scn = 0; if (OB_ISNULL(GCTX.root_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("root service is null", KR(ret)); } else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) { LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id)); } else { ObTableCreator table_creator(tenant_id, frozen_scn, *lst_operator_, trans); common::ObArray ls_id_array; const ObTableSchema *tmp_table_schema = inc_table_schema_ptrs.at(0); ObNewTableTabletAllocator new_table_tablet_allocator(tenant_id, schema_guard, sql_proxy_); if (OB_ISNULL(tmp_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schem is null", KR(ret), K(inc_table_schema_ptrs)); } else if (OB_FAIL(table_creator.init())) { LOG_WARN("fail to init table creator", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.init())) { LOG_WARN("fail to init new table tablet allocator", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.prepare(*tmp_table_schema))) { LOG_WARN("failed to prepare tablet allocator", KR(ret), KPC(tmp_table_schema)); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg( inc_table_schema_ptrs, ls_id_array))) { LOG_WARN("create table partitions failed", KR(ret), K(alter_table_schema), K(inc_table_schema_ptrs)); } else if (OB_FAIL(table_creator.execute())) { LOG_WARN("execute create partition failed", KR(ret)); } } } } if (OB_FAIL(ret)) { } else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::REORGANIZE_PARTITION == alter_table_arg.alter_part_type_ || obrpc::ObAlterTableArg::SPLIT_PARTITION == alter_table_arg.alter_part_type_) { ret = OB_NOT_SUPPORTED; LOG_WARN("split partition is not supported", KR(ret), K(alter_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "split partition is"); } if (OB_SUCC(ret) && alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::INCREMENT_MODE) && 0 != orig_table_schema->get_autoinc_column_id()) { ObAutoincrementService &autoinc_service = ObAutoincrementService::get_instance(); if (OB_FAIL(autoinc_service.clear_autoinc_cache_all(tenant_id, orig_table_schema->get_table_id(), orig_table_schema->get_autoinc_column_id(), orig_table_schema->is_order_auto_increment_mode()))) { LOG_WARN("fail to clear autoinc cache", K(ret)); } } if (OB_FAIL(ret)) { } else if (DDL_CREATE_INDEX == ddl_type || DDL_NORMAL_TYPE == ddl_type) { ObIndexBuilder index_builder(*this); const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; int tmp_ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) { ObIndexArg *index_arg = const_cast(index_arg_list.at(i)); ObDDLTaskRecord task_record; if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("index arg should not be null", K(ret)); } else if (ObIndexArg::ADD_INDEX == index_arg->index_action_type_ || ObIndexArg::REBUILD_INDEX == index_arg->index_action_type_) { ObCreateIndexArg *create_index_arg = static_cast(index_arg); ObTableSchema &index_schema = create_index_arg->index_schema_; if (INDEX_TYPE_PRIMARY == create_index_arg->index_type_) { // do nothing } else if (OB_FAIL(index_builder.submit_build_index_task(trans, *create_index_arg, &new_table_schema, &index_schema, alter_table_arg.parallelism_, alter_table_arg.allocator_, task_record))) { LOG_WARN("fail to submit build index task", K(ret), "type", create_index_arg->index_type_); } else if (OB_FAIL(ddl_tasks.push_back(task_record))) { LOG_WARN("fail to push ddl task", K(ret), K(task_record)); } else { res.task_id_ = task_record.task_id_; obrpc::ObAlterTableResArg arg(TABLE_SCHEMA, create_index_arg->index_schema_.get_table_id(), create_index_arg->index_schema_.get_schema_version()); if (OB_FAIL(res.res_arg_array_.push_back(arg))) { LOG_WARN("push back to res_arg_array failed", K(ret), K(arg)); } } } } } if (OB_FAIL(ret)) { } else if (alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::ADD_CONSTRAINT || alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE) { ObTableSchema::const_constraint_iterator iter = alter_table_arg.alter_table_schema_.constraint_begin(); /* * observer need return while create index, create constraint or modify constraint. * in the resolver phase, it ensures that no other actions will happen at the same time while alter table. * check constraint need return constriant_id_ and schema_version_. other constraint return schema_version_. * the schema version is data table after finish alter table. */ const bool need_check = need_check_constraint_validity(const_alter_table_arg); res.constriant_id_ = (*iter)->get_constraint_id(); res.schema_version_ = new_table_schema.get_schema_version(); if (need_check) { bool need_modify_notnull_validate = false; bool is_add_not_null_col = false; if (OB_FAIL(need_modify_not_null_constraint_validate( const_alter_table_arg, is_add_not_null_col, need_modify_notnull_validate))) { } else { ObDDLTaskRecord task_record; ObCreateDDLTaskParam param(new_table_schema.get_tenant_id(), is_add_not_null_col ? ObDDLType::DDL_ADD_NOT_NULL_COLUMN : ObDDLType::DDL_CHECK_CONSTRAINT, &new_table_schema, nullptr, (*iter)->get_constraint_id(), new_table_schema.get_schema_version(), 0/*parallelsim*/, &alter_table_arg.allocator_, &const_alter_table_arg); if (OB_FAIL(GCTX.root_service_->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) { LOG_WARN("submit constraint task failed", K(ret)); } else if (OB_FAIL(ddl_tasks.push_back(task_record))) { LOG_WARN("fail to push ddl task", K(ret), K(task_record)); } else { res.task_id_ = task_record.task_id_; } } } } else if (const_alter_table_arg.foreign_key_arg_list_.count() > 0) { if (1 != const_alter_table_arg.foreign_key_arg_list_.count()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, can not add more than one foreign key at the same time", K(ret)); } else { const ObCreateForeignKeyArg &foreign_key_arg = const_alter_table_arg.foreign_key_arg_list_.at(0); int64_t fk_id = OB_INVALID_ID; res.schema_version_ = new_table_schema.get_schema_version(); if (foreign_key_arg.need_validate_data_ && ((!foreign_key_arg.is_modify_fk_state_ && foreign_key_arg.validate_flag_) || (foreign_key_arg.is_modify_validate_flag_ && foreign_key_arg.validate_flag_))) { const ObIArray &fk_infos = alter_table_schema.get_foreign_key_infos(); const int64_t fk_cnt = fk_infos.count(); ObDDLTaskRecord task_record; for (int64_t i = 0; OB_SUCC(ret) && i < fk_infos.count(); ++i) { if (0 == foreign_key_arg.foreign_key_name_.compare(fk_infos.at(i).foreign_key_name_)) { fk_id = fk_infos.at(i).foreign_key_id_; break; } } if (OB_FAIL(ret)) { } else if (OB_INVALID_ID == fk_id) { ret = OB_ERR_UNEXPECTED; LOG_WARN("can not find foreign key", K(ret)); } else { ObCreateDDLTaskParam param(new_table_schema.get_tenant_id(), ObDDLType::DDL_FOREIGN_KEY_CONSTRAINT, &new_table_schema, nullptr, fk_id, new_table_schema.get_schema_version(), 0/*parallelism*/, &alter_table_arg.allocator_, &const_alter_table_arg); if (OB_FAIL(GCTX.root_service_->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) { LOG_WARN("submit constraint task", K(ret)); } else if (OB_FAIL(ddl_tasks.push_back(task_record))) { LOG_WARN("fail to push ddl task", K(ret), K(task_record)); } else { res.task_id_ = task_record.task_id_; } } } } } const bool is_commit = OB_SUCC(ret); if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(is_commit))) { LOG_WARN("trans end failed", K(is_commit), K(temp_ret)); ret = is_commit ? temp_ret : ret; } } if (OB_SUCC(ret)) { int tmp_ret = OB_SUCCESS; if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } else { for (int64_t i = 0; OB_SUCCESS == tmp_ret && i < ddl_tasks.count(); i++) { ObDDLTaskRecord &task_record = ddl_tasks.at(i); if (OB_TMP_FAIL(GCTX.root_service_->get_ddl_scheduler().schedule_ddl_task(task_record))) { LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record)); } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(delete_auto_increment_attribute(orig_table_schema, new_table_schema, alter_table_schema))) { LOG_WARN("fail to delete auto-incr attribute", K(ret), KPC(orig_table_schema), K(alter_table_schema)); } } } } } } } } } return ret; } int ObDDLService::check_is_offline_ddl(ObAlterTableArg &alter_table_arg, ObDDLType &ddl_type) { int ret = OB_SUCCESS; ddl_type = ObDDLType::DDL_INVALID; bool is_oracle_mode = false; ObSchemaGetterGuard schema_guard; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; uint64_t tenant_id = alter_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret)); } else { char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0}; const ObTableSchema *orig_table_schema = NULL; if (OB_FAIL(get_and_check_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema))) { LOG_WARN("fail to get and check table schema", K(ret)); } else if (OB_FAIL(orig_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", KR(ret), KPC(orig_table_schema)); } if (OB_SUCC(ret) && alter_table_arg.is_alter_columns_ && OB_FAIL(check_alter_table_column(alter_table_arg, *orig_table_schema, schema_guard, is_oracle_mode, ddl_type))) { LOG_WARN("fail to check alter table column", K(ret)); } if (OB_SUCC(ret) && alter_table_arg.is_alter_indexs_ && OB_FAIL(check_alter_table_index(alter_table_arg, ddl_type))) { LOG_WARN("fail to check alter table index", K(ret)); } if (OB_SUCC(ret) && alter_table_arg.is_alter_partitions_ && OB_FAIL(check_alter_table_partition(alter_table_arg, *orig_table_schema, is_oracle_mode, ddl_type))) { LOG_WARN("fail to check alter table partition", K(ret)); } if (OB_SUCC(ret) && alter_table_arg.alter_constraint_type_!= obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION && OB_FAIL(check_alter_table_constraint(alter_table_arg, *orig_table_schema, ddl_type))) { LOG_WARN("fail to check alter table constraint", K(ret), K(alter_table_arg), K(ddl_type)); } if (OB_SUCC(ret) && alter_table_arg.is_convert_to_character_ && OB_FAIL(check_convert_to_character(alter_table_arg, *orig_table_schema, ddl_type))) { LOG_WARN("fail to check convert to character", K(ret)); } if (OB_SUCC(ret) && alter_table_arg.foreign_key_arg_list_.count() > 0 && ddl_type == ObDDLType::DDL_INVALID) { ddl_type = ObDDLType::DDL_NORMAL_TYPE; } if (OB_SUCC(ret) && is_long_running_ddl(ddl_type) && (alter_table_arg.is_alter_options_ || !alter_table_arg.alter_table_schema_.get_foreign_key_infos().empty())) { if (alter_table_arg.is_alter_options_) {// alter options (void)snprintf(err_msg, sizeof(err_msg), "%s and alter options in single statment", ddl_type_str(ddl_type)); } else if (!alter_table_arg.alter_table_schema_.get_foreign_key_infos().empty()) {// alter foreign key (void)snprintf(err_msg, sizeof(err_msg), "%s and alter foreign key in single statment", ddl_type_str(ddl_type)); } ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); } if (OB_SUCC(ret) && is_long_running_ddl(ddl_type)) { if (orig_table_schema->is_primary_vp_table()) { ret = OB_NOT_SUPPORTED; (void)snprintf(err_msg, sizeof(err_msg), "%s with primary vp table", ddl_type_str(ddl_type)); LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg); } // offline ddl is allowed on table with trigger(enable/disable). } if (OB_SUCC(ret) && (is_long_running_ddl(ddl_type) || alter_table_arg.is_alter_indexs_ || alter_table_arg.alter_constraint_type_!= obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION)) { if (OB_FAIL(check_fk_related_table_ddl(*orig_table_schema))) { LOG_WARN("check whether the foreign key related table is executing ddl failed", K(ret)); } } if (OB_SUCC(ret) && is_double_table_long_running_ddl(ddl_type)) { bool is_building_index = false; bool is_adding_constraint = false; uint64_t table_id = alter_table_arg.alter_table_schema_.get_table_id(); if (OB_FAIL(check_is_building_index(schema_guard, tenant_id, table_id, is_building_index))) { LOG_WARN("failed to call check_is_building_index", K(ret)); } else if (OB_FAIL(check_is_adding_constraint(table_id, is_adding_constraint))) { LOG_WARN("failed to call check_is_adding_constraint", K(ret)); } else if (is_building_index) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "The DDL cannot be run concurrently with creating index."); } else if (is_adding_constraint) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "The DDL cannot be run concurrently with adding constraint."); } } } return ret; } int ObDDLService::check_is_building_index(ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, const uint64_t table_id, bool &is_building_index) { int ret = OB_SUCCESS; is_building_index = false; // 1. get table schema const ObTableSchema *orig_table = nullptr; ObSEArray index_infos; if (OB_UNLIKELY(tenant_id == OB_INVALID_ID || table_id == OB_INVALID_ID)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(tenant_id), K(table_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, orig_table))) { LOG_WARN("get table schema failed", K(ret)); } else if (OB_ISNULL(orig_table)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid table id", "table_id", table_id); } else if (OB_FAIL(orig_table->get_simple_index_infos(index_infos, false/*with_mv*/))) { LOG_WARN("get simple_index_infos failed", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < index_infos.count(); i++) { // 2. get all index schemas const ObTableSchema *index_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, index_infos.at(i).table_id_, index_schema))) { LOG_WARN("get index schema failed", K(ret), K(index_infos.at(i).table_id_)); } else if (OB_ISNULL(index_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid index table id", "index_table_id", index_infos.at(i).table_id_); } else if (index_schema->is_unavailable_index()) { // 3. check if index is still constructing is_building_index = true; break; } } } return ret; } // check if is adding check constraint, foreign key, not null constraint int ObDDLService::check_is_adding_constraint(const uint64_t table_id, bool &is_building) { ObArenaAllocator allocator(lib::ObLabel("DdlTasRecord")); return ObDDLTaskRecordOperator::check_is_adding_constraint(sql_proxy_, allocator, table_id, is_building); } // check whether the foreign key related table is executing offline ddl, creating index, and executin constrtaint task. // And ddl should be refused if the foreign key related table is executing above ddl. int ObDDLService::check_fk_related_table_ddl( const share::schema::ObTableSchema &data_table_schema) { int ret = OB_SUCCESS; const uint64_t tenant_id = data_table_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(ret)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); } else { const ObIArray &foreign_key_infos = data_table_schema.get_foreign_key_infos(); for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); const uint64_t related_table_id = data_table_schema.get_table_id() == foreign_key_info.parent_table_id_ ? foreign_key_info.child_table_id_ : foreign_key_info.parent_table_id_; bool has_long_running_ddl = false; const ObTableSchema *related_schema = nullptr; if (foreign_key_info.is_parent_table_mock_ || data_table_schema.get_table_id() == related_table_id) { // mock table and self reference foreign key table, no need to check. } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, related_table_id, related_schema))) { LOG_WARN("get schema failed", K(ret), K(tenant_id), K(related_table_id)); } else if (OB_ISNULL(related_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error, related schema is nullptr", K(ret), K(related_table_id), K(foreign_key_info)); } else if (!related_schema->check_can_do_ddl()) { ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "execute ddl while foreign key related table is executing offline ddl"); } else if (OB_FAIL(ObDDLTaskRecordOperator::check_has_long_running_ddl(sql_proxy_, tenant_id, related_table_id, has_long_running_ddl))) { LOG_WARN("check has long running ddl failed", K(ret), K(tenant_id), K(related_table_id)); } else if (has_long_running_ddl) { ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "execute ddl while there are some long running ddl on foreign key related table"); } } } return ret; } int ObDDLService::check_can_bind_tablets(const share::ObDDLType ddl_type, bool &bind_tablets) { int ret = OB_SUCCESS; switch (ddl_type) { case DDL_ADD_COLUMN_OFFLINE: case DDL_DROP_COLUMN: case DDL_COLUMN_REDEFINITION: { bind_tablets = true; break; } default: { bind_tablets = false; } } return ret; } // check if the ddl contains primary key operation. int ObDDLService::check_ddl_with_primary_key_operation( const obrpc::ObAlterTableArg &alter_table_arg, bool &with_primary_key_operation) { int ret = OB_SUCCESS; with_primary_key_operation = false; const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; for (int64_t i = 0; OB_SUCC(ret) && !with_primary_key_operation && i < index_arg_list.size(); ++i) { ObIndexArg *index_arg = const_cast(index_arg_list.at(i)); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("index arg should not be null", K(ret)); } else { const ObIndexArg::IndexActionType type = index_arg->index_action_type_; with_primary_key_operation = ObIndexArg::DROP_PRIMARY_KEY == type || ObIndexArg::ADD_PRIMARY_KEY == type || ObIndexArg::ALTER_PRIMARY_KEY == type; } } return ret; } int ObDDLService::do_offline_ddl_in_trans(obrpc::ObAlterTableArg &alter_table_arg, const int64_t frozen_version, obrpc::ObAlterTableRes &res) { int ret = OB_SUCCESS; bool bind_tablets = false; ObSchemaGetterGuard schema_guard; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; uint64_t tenant_id = alter_table_schema.get_tenant_id(); const ObDDLType ddl_type = res.ddl_type_; ObRootService *root_service = GCTX.root_service_; bool need_redistribute_column_id = false; if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "Unsupported ddl operation"); } else if (OB_UNLIKELY(DDL_INVALID == ddl_type)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("unexpected ddl type", K(ret), K(ddl_type), K(alter_table_arg)); } else if (OB_ISNULL(root_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, root service must not be nullptr", K(ret)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(check_can_bind_tablets(ddl_type, bind_tablets))) { LOG_WARN("failed to check can bind tablets", K(ret), K(ddl_type)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObTableSchema new_table_schema; const ObTableSchema *orig_table_schema = NULL; if (OB_FAIL(get_and_check_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema))) { LOG_WARN("fail to get and check table schema", K(ret)); } else if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLTaskRecord task_record; int64_t refreshed_schema_version = 0; bool with_primary_key_operation = false; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } // TODO yiren, refactor it, create user hidden table after alter index/column/part/cst... if (OB_FAIL(ret)) { } else if (OB_FAIL(check_ddl_with_primary_key_operation(alter_table_arg, with_primary_key_operation))) { LOG_WARN("check ddl with primary key operation failed", K(ret)); } else if (with_primary_key_operation) { if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) { LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id), KPC(orig_table_schema)); } else if (OB_FAIL(alter_table_primary_key(alter_table_arg, *orig_table_schema, new_table_schema, schema_guard, frozen_version, ddl_operator, trans, alter_table_arg.allocator_))) { LOG_WARN("failed to alter table primary key", K(ret)); } } if (OB_SUCC(ret) && ObDDLType::DDL_TABLE_REDEFINITION == ddl_type) { bool need_modify_notnull_validate = false; bool is_add_not_null_col = false; if (OB_FAIL(need_modify_not_null_constraint_validate( alter_table_arg, is_add_not_null_col, need_modify_notnull_validate))) { LOG_WARN("check need modify not null constraint validate failed", K(ret)); } else if (need_modify_notnull_validate) { ObConstraint *cst = *alter_table_schema.constraint_begin_for_non_const_iter(); const uint64_t col_id = *(cst->cst_col_begin()); ObColumnSchemaV2 *col_schema = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_column_count(); i++) { if (alter_table_schema.get_column_schema_by_idx(i)->get_column_id() == col_id) { col_schema = alter_table_schema.get_column_schema_by_idx(i); } } if (OB_ISNULL(col_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column schema not found", K(ret), K(alter_table_arg)); } else { col_schema->del_column_flag(NOT_NULL_VALIDATE_FLAG); } } } if (OB_SUCC(ret) && alter_table_arg.is_alter_columns_) { if (ObDDLType::DDL_MODIFY_AUTO_INCREMENT == ddl_type) { new_table_schema.set_in_offline_ddl_white_list(true); new_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL); } else if (OB_FAIL(check_can_alter_column(tenant_id, alter_table_schema, *orig_table_schema))) { LOG_WARN("fail to can alter column", K(ret), K(alter_table_arg)); } else if (OB_FAIL(gen_alter_column_new_table_schema_offline( *orig_table_schema, alter_table_schema, new_table_schema, alter_table_arg, schema_guard, need_redistribute_column_id))) { LOG_WARN("failed to alter table column!", K(*orig_table_schema), K(new_table_schema), K(ret)); } else if (OB_FAIL(create_user_hidden_table(*orig_table_schema, new_table_schema, &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, frozen_version, ddl_operator, trans, alter_table_arg.allocator_))) { LOG_WARN("fail to create user hidden table", K(ret)); } } if (OB_SUCC(ret) && alter_table_arg.is_alter_partitions_) { if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) { LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id), K(orig_table_schema->get_table_id())); } else if (OB_FAIL(alter_table_partition_by(alter_table_arg, *orig_table_schema, new_table_schema, schema_guard, frozen_version, ddl_operator, trans))) { LOG_WARN("failed to alter table partition by", K(ret)); } } // table constraints if (OB_SUCC(ret) && alter_table_arg.alter_constraint_type_ != obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION) { if (ObDDLType::DDL_TABLE_REDEFINITION == ddl_type || ObDDLType::DDL_MODIFY_COLUMN == ddl_type) { if (OB_FAIL(alter_table_constraints( alter_table_arg.alter_constraint_type_, schema_guard, *orig_table_schema, alter_table_schema, new_table_schema, ddl_operator, trans))) { LOG_WARN("alter table constraints failed", K(ret)); } } } if (OB_SUCC(ret) && alter_table_arg.is_convert_to_character_) { if (OB_FAIL(convert_to_character(alter_table_arg, *orig_table_schema, new_table_schema, schema_guard, frozen_version, ddl_operator, trans))) { LOG_WARN("failed to convert to character", K(ret)); } } if (OB_SUCC(ret) && need_redistribute_column_id) { if (OB_FAIL(redistribute_column_ids(new_table_schema))) { LOG_WARN("failed to redistribute column ids", K(ret)); } else { // do nothing } } if (OB_SUCC(ret)) { if (OB_FAIL(new_table_schema.sort_column_array_by_column_id())) { LOG_WARN("failed to sort column", K(ret), K(new_table_schema)); } } if (OB_SUCC(ret)) { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema, trans, operation_type))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } else { alter_table_arg.alter_table_schema_.set_schema_version(new_table_schema.get_schema_version()); if (ObDDLType::DDL_MODIFY_AUTO_INCREMENT != ddl_type) { alter_table_arg.alter_table_schema_.set_association_table_id(new_table_schema.get_table_id()); } } } // submit async build index task if (OB_FAIL(ret)) { } else if (is_double_table_long_running_ddl(ddl_type)) { ObCreateDDLTaskParam param(tenant_id, ddl_type, orig_table_schema, &new_table_schema, 0/*object_id*/, new_table_schema.get_schema_version(), alter_table_arg.parallelism_, &alter_table_arg.allocator_, &alter_table_arg); if (orig_table_schema->is_tmp_table()) { ret = OB_OP_NOT_ALLOW; char err_msg[OB_MAX_ERROR_MSG_LEN] = {0}; (void)snprintf(err_msg, sizeof(err_msg), "%s on temporary table is", ddl_type_str(ddl_type)); LOG_WARN("double table long running ddl on temporary table is disallowed", K(ret), K(ddl_type)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg); } else if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) { LOG_WARN("submit ddl task failed", K(ret)); } else { res.task_id_ = task_record.task_id_; } } else if (is_simple_table_long_running_ddl(ddl_type)) { ObCreateDDLTaskParam param(tenant_id, ddl_type, &new_table_schema, nullptr, 0/*object_id*/, new_table_schema.get_schema_version(), alter_table_arg.parallelism_, &alter_table_arg.allocator_, &alter_table_arg); if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) { LOG_WARN("submit ddl task failed", K(ret)); } else { res.task_id_ = task_record.task_id_; } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to identify long running ddl type", K(ret), K(ddl_type)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { int tmp_ret = OB_SUCCESS; if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } else if (OB_TMP_FAIL(root_service->get_ddl_scheduler().schedule_ddl_task(task_record))) { LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record)); } } } } return ret; } int ObDDLService::get_and_check_table_schema( const obrpc::ObAlterTableArg &alter_table_arg, ObSchemaGetterGuard &schema_guard, const AlterTableSchema &alter_table_schema, const ObTableSchema *&orig_table_schema, bool is_offline_ddl/*false*/) { int ret = OB_SUCCESS; uint64_t tenant_id = alter_table_schema.get_tenant_id(); schema_guard.set_session_id(alter_table_arg.session_id_); const ObString &origin_database_name = alter_table_schema.get_origin_database_name(); const ObString &origin_table_name = alter_table_schema.get_origin_table_name(); if (origin_database_name.empty() || origin_table_name.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("database name or table name is null", K(alter_table_schema), K(origin_database_name), K(origin_table_name), K(ret)); } else { bool is_index = false; bool is_db_in_recyclebin = false; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, origin_database_name, origin_table_name, is_index, orig_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(origin_database_name), K(origin_table_name)); } else if (NULL == orig_table_schema) { ret = OB_TABLE_NOT_EXIST; LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(origin_database_name), to_cstring(origin_table_name)); } else if (OB_FAIL(schema_guard.check_database_in_recyclebin( tenant_id, orig_table_schema->get_database_id(), is_db_in_recyclebin))) { LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), K(*orig_table_schema)); } else if (!alter_table_arg.is_inner_ && is_db_in_recyclebin) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not alter table in recyclebin", K(ret), K(alter_table_arg)); } else if (!alter_table_arg.skip_sys_table_check_ && OB_FAIL(check_enable_sys_table_ddl(*orig_table_schema, OB_DDL_ALTER_TABLE))) { LOG_WARN("ddl is not allowed on system table", K(ret)); } else if (!alter_table_arg.is_inner_ && (orig_table_schema->is_in_recyclebin() || is_db_in_recyclebin)) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not alter table in recyclebin", K(ret), K(alter_table_arg), K(is_db_in_recyclebin)); } else if (!orig_table_schema->is_user_table() && !orig_table_schema->is_sys_table() && !orig_table_schema->is_tmp_table()) { ret = OB_ERR_WRONG_OBJECT; LOG_USER_ERROR(OB_ERR_WRONG_OBJECT, to_cstring(origin_database_name), to_cstring(origin_table_name), "BASE TABLE"); // NOTE: if_offline_ddl is true only in the execution phase of offline ddl, skip check_can_do_ddl check } else if (!is_offline_ddl && !orig_table_schema->check_can_do_ddl()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("offline ddl is being executed, other ddl operations are not allowed", K(ret), K(is_offline_ddl), K(*orig_table_schema)); } } return ret; } /* * FIXME: * Before minor merge sstable sparse format support, in order to relieve the dependence on gc_snapshot_timestamp, * first implement a temporary solution to avoid the operation of modifying columns in the process of * merging the standalone cluster. * 2.2.x, * to primary cluster, Since RS determines the major version first, * and then determine the schema_version used for the merge by ddl, it can be judged * whether the merge has started according to whether the __all_zone version numbers are consistent * to standalone, It is impossible to make the DDL of the primary cluster and the merge of standalone library * to be mutually exclusive, so this solution is only to reduce the probability of bad cases. * When the merge checksum is inconsistent in the standalone cluster, it can be corrected by drop replica * >= 3.3, allow drop column when doing major freeze */ int ObDDLService::check_can_alter_column( const int64_t tenant_id, const AlterTableSchema &alter_table_schema, const ObTableSchema &orig_table_schema) { int ret = OB_SUCCESS; const int64_t table_id = orig_table_schema.get_table_id(); DEBUG_SYNC(BEFORE_ALTER_TABLE_COLUMN); if (THIS_WORKER.is_timeout_ts_valid() && THIS_WORKER.is_timeout()) { ret = OB_TIMEOUT; LOG_WARN("already timeout", KR(ret)); } else if (OB_FAIL(check_restore_point_allow(tenant_id, orig_table_schema))) { LOG_WARN("restore point check fail, cannot alter column", K(ret), K(tenant_id), K(table_id)); } else { ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin(); ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end(); AlterColumnSchema *alter_column_schema = NULL; bool need_drop_column = false; for(; OB_SUCC(ret) && !need_drop_column && it_begin != it_end; it_begin++) { if (OB_ISNULL(*it_begin)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin is NULL", K(ret)); } else { alter_column_schema = static_cast(*it_begin); if (OB_DDL_DROP_COLUMN == alter_column_schema->alter_type_) { need_drop_column = true; if (OB_FAIL(check_enable_sys_table_ddl(orig_table_schema, OB_DDL_DROP_COLUMN))) { LOG_WARN("fail to check enable sys table ddl", KR(ret), K(orig_table_schema)); } } } } } return ret; } // If there is a restore point on the table, it is not allowed to create indexes, add or delete columns. int ObDDLService::check_restore_point_allow(const int64_t tenant_id, const ObTableSchema &table_schema) { int ret = OB_SUCCESS; bool is_exist = false; ObArray tablet_ids; if (!is_inner_table(table_schema.get_table_id()) && OB_FAIL(get_snapshot_mgr().check_restore_point( get_sql_proxy(), tenant_id, table_schema.get_table_id(), is_exist))) { LOG_WARN("failed to check restore point", K(ret), K(tenant_id)); } if (OB_SUCC(ret) && is_exist) { ret = OB_OP_NOT_ALLOW; LOG_WARN("restore point exist, cannot alter ", K(ret), K(tenant_id), K(table_schema.get_table_id())); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "restore point exist, create index/alter"); } return ret; } int ObDDLService::check_all_server_frozen_version( const int64_t frozen_version) { int ret = OB_SUCCESS; if (OB_ISNULL(rpc_proxy_) || OB_ISNULL(server_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", K(ret), KP_(rpc_proxy), KP_(server_mgr)); } else { ObCheckFrozenVersionProxy check_frozen_version_proxy(*rpc_proxy_, &obrpc::ObSrvRpcProxy::check_frozen_version); ObZone zone; ObArray server_statuses; ObCheckFrozenVersionArg arg; arg.frozen_version_ = frozen_version; if (OB_FAIL(server_mgr_->get_server_statuses(zone, server_statuses))) { LOG_WARN("fail to get server statuses", K(ret)); } else if (server_statuses.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid server cnt", K(ret)); } // check server alive for (int64_t i = 0; OB_SUCC(ret) && i < server_statuses.count(); i++) { if (!server_statuses[i].is_alive()) { ret = OB_SERVER_NOT_ALIVE; LOG_WARN("server not alive", K(ret), "server", server_statuses[i]); } } if (OB_SUCC(ret)) { // send async rpc for (int64_t i = 0; OB_SUCC(ret) && i < server_statuses.count(); i++) { const int64_t rpc_timeout_us = THIS_WORKER.get_timeout_remain(); const ObAddr &addr = server_statuses[i].server_; if (OB_FAIL(check_frozen_version_proxy.call(addr, rpc_timeout_us, arg))) { LOG_WARN("fail to check frozen version", K(ret), K(addr), K(rpc_timeout_us)); } } int tmp_ret = OB_SUCCESS; // all server should success; if (OB_SUCCESS != (tmp_ret = check_frozen_version_proxy.wait())) { LOG_WARN("fail to execute rpc", K(tmp_ret)); } ret = OB_SUCC(ret) ? tmp_ret : ret; } } return ret; } // This code will be used for partition operations of table and tablegroup // 1. for table, parameter is_drop_truncate_and_alter_index parameter avoids the drop/truncate partition // of table with global index and the index create in the same alter table statement // 2. for tablegroup, avoid drop/truncate partition in tablegroup with global index // (After the tablegroup supports drop/truncate partitions of table with global indexes, // the behavior will be unified.) int ObDDLService::check_index_valid_for_alter_partition( const share::schema::ObTableSchema &orig_table_schema, share::schema::ObSchemaGetterGuard &schema_guard, const bool is_drop_truncate_and_alter_index, const bool is_split) { int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObSEArray simple_index_infos; if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get_index_tid_array failed", KR(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema( tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", "table id", simple_index_infos.at(i).table_id_, KR(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", KR(ret)); } else if (index_table_schema->is_global_index_table()) { if (is_drop_truncate_and_alter_index) { ret = OB_NOT_SUPPORTED; LOG_WARN("alter index and drop/truncate tables with global index not support", KR(ret), "index_tid", simple_index_infos.at(i).table_id_, K(index_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter index and drop/truncate tables with global index"); } } else if (index_table_schema->is_global_local_index_table()) { if (is_split) { ret = OB_NOT_SUPPORTED; LOG_WARN("can not convert a non-partitioned table with non-partitioned global index to a partitioned table", KR(ret), "index_tid", simple_index_infos.at(i).table_id_, K(index_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "converting a non-partitioned table with non-partitioned global index to a partitioned table"); } } } } return ret; } int ObDDLService::check_alter_set_interval(const share::schema::ObTableSchema &orig_table_schema, const obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; ObPartitionFuncType part_func_type = orig_table_schema.get_part_option().get_part_func_type(); const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; ObPartition **part_array = orig_table_schema.get_part_array(); int64_t part_num = orig_table_schema.get_partition_num(); if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level() && !orig_table_schema.has_sub_part_template_def()) { ret = OB_NOT_SUPPORTED; LOG_WARN("interval part of composited-partitioned table not support", K(ret), K(orig_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval part of composited-partitioned table"); } else if (1 != orig_table_schema.get_partition_key_column_num()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("more than one partition key not support", K(ret), K(orig_table_schema)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "more than one partition key"); } else if (OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to part_array is null", K(orig_table_schema), K(alter_table_schema), KR(ret)); } else if (PARTITION_FUNC_TYPE_INTERVAL == part_func_type) { ret = OB_NOT_SUPPORTED; LOG_WARN("change interval range in inteval table not support yet", KR(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "change interval range in inteval table"); } else if (PARTITION_FUNC_TYPE_INTERVAL != part_func_type && PARTITION_FUNC_TYPE_RANGE_COLUMNS != part_func_type) { ret = OB_ERR_UNEXPECTED; LOG_WARN("type is unexpected when set interval", K(orig_table_schema), K(alter_table_schema), KR(ret)); } else if (part_num < 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part num is less 1", K(orig_table_schema), K(alter_table_schema), KR(ret)); } else if (OB_ISNULL(part_array[part_num - 1])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("the last part is null", K(orig_table_schema), K(alter_table_schema), KR(ret)); } else if (OB_FAIL(ObPartitionUtils::check_interval_partition_table(part_array[part_num - 1]->get_high_bound_val(), alter_table_schema.get_interval_range()))) { LOG_WARN("fail to check_interval_partition_table", KR(ret)); } return ret; } // Check various conditions of partition management // after 2.0 do not allow separate additions, deletions, and splitting of tables in tablegroup. int ObDDLService::check_alter_partitions(const ObTableSchema &orig_table_schema, obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; const obrpc::ObAlterTableArg::AlterPartitionType alter_part_type = alter_table_arg.alter_part_type_; const uint64_t tablegroup_id = orig_table_schema.get_tablegroup_id(); const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; bool is_drop_or_truncate = false; bool is_split = false; bool is_oracle_mode = false; bool has_local_index = false; if (GCONF.in_upgrade_mode()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("in upgrade, can not do partition maintenance", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "partition maintenance during upgrade"); } else if (OB_INVALID_ID != tablegroup_id && !is_sys_tablegroup_id(tablegroup_id) && obrpc::ObAlterTableArg::TRUNCATE_PARTITION != alter_part_type) { ret = OB_OP_NOT_ALLOW; LOG_WARN("add/drop table partition in 2.0 tablegroup not allowed", K(ret), K(tablegroup_id)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "add/drop table partition in 2.0 tablegroup"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } else if (orig_table_schema.is_interval_part()) { if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level() && !orig_table_schema.has_sub_part_template_def()) { ret = OB_NOT_SUPPORTED; LOG_WARN("interval part of composited-partitioned table not support", K(ret), K(orig_table_schema)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval part of composited-partitioned table"); } } else if (OB_FAIL(orig_table_schema.check_has_local_index(schema_guard, has_local_index))) { LOG_WARN("fail to check_has_local_index", K(ret), K(has_local_index)); } else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_part_type && obrpc::ObAlterTableArg::SPLIT_PARTITION == alter_part_type && obrpc::ObAlterTableArg::REPARTITION_TABLE == alter_part_type && obrpc::ObAlterTableArg::PARTITIONED_PARTITION == alter_part_type) { ret = OB_OP_NOT_ALLOW; LOG_WARN("split partition in 4.0 not allowed", K(ret), K(tablegroup_id)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "split partition in 4.0"); } if (OB_FAIL(ret)) { } else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_part_type) { bool is_truncate = true; if (OB_FAIL(check_alter_drop_partitions(orig_table_schema, alter_table_arg, is_truncate))) { LOG_WARN("failed to check truncate partitons", K(ret), K(orig_table_schema), K(alter_table_arg)); } is_drop_or_truncate = true; } else if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_part_type) { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2276) { ret = OB_NOT_SUPPORTED; LOG_WARN("truncate subpartition in update not support", KR(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate subpartition in update "); } else if (OB_FAIL(check_alter_drop_subpartitions(orig_table_schema, alter_table_arg))) { LOG_WARN("failed to check drop partition", KR(ret), K(orig_table_schema), K(alter_table_arg)); } is_drop_or_truncate = true; } else if (obrpc::ObAlterTableArg::DROP_PARTITION == alter_part_type) { bool is_truncate = false; if (OB_FAIL(check_alter_drop_partitions(orig_table_schema, alter_table_arg, is_truncate))) { LOG_WARN("failed to check drop partition", K(ret), K(orig_table_schema), K(alter_table_arg)); } is_drop_or_truncate = true; } else if (obrpc::ObAlterTableArg::DROP_SUB_PARTITION == alter_part_type) { if (OB_FAIL(check_alter_drop_subpartitions(orig_table_schema, alter_table_arg))) { LOG_WARN("failed to check drop partition", K(ret), K(orig_table_schema), K(alter_table_arg)); } is_drop_or_truncate = true; } else if (obrpc::ObAlterTableArg::ADD_PARTITION == alter_part_type) { if (OB_FAIL(check_alter_add_partitions(orig_table_schema, alter_table_arg))) { LOG_WARN("failed to check add paritions", K(ret), K(orig_table_schema), K(alter_table_arg)); } } else if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_part_type) { if (OB_FAIL(check_alter_add_subpartitions(orig_table_schema, alter_table_arg))) { LOG_WARN("failed to check add paritions", K(ret), K(orig_table_schema), K(alter_table_arg)); } } else if (is_oracle_mode && obrpc::ObAlterTableArg::SET_INTERVAL == alter_part_type) { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_WARN("interval partition less than 4.0 not support", K(ret), K(GET_MIN_CLUSTER_VERSION())); LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval partition less than 4.0"); } else if (OB_FAIL(check_alter_set_interval(orig_table_schema, alter_table_arg))) { LOG_WARN("failed to check set interval", K(ret), K(orig_table_schema), K(alter_table_arg)); } } else if (is_oracle_mode && obrpc::ObAlterTableArg::INTERVAL_TO_RANGE == alter_part_type) { if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_0_0_0) { ret = OB_NOT_SUPPORTED; LOG_WARN("interval partition less than 4.0 not support", K(ret), K(GET_MIN_CLUSTER_VERSION())); LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval partition less than 4.0"); } else if (PARTITION_FUNC_TYPE_INTERVAL != orig_table_schema.get_part_option().get_part_func_type()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("type is unexpected when interval to range", K(orig_table_schema), K(alter_table_arg), KR(ret)); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected alter partition type", K(ret), K(alter_part_type)); } if (OB_SUCC(ret)) { if (OB_FAIL(check_index_valid_for_alter_partition(orig_table_schema, schema_guard, is_drop_or_truncate && alter_table_arg.index_arg_list_.size() != 0, is_split))) { LOG_WARN("failed to check index valid", K(ret), K(is_split), K(is_drop_or_truncate), K(orig_table_schema)); } } return ret; } int ObDDLService::check_table_pk(const share::schema::ObTableSchema &orig_table_schema) { int ret = OB_SUCCESS; // Check whether table is the parent table of the foreign key constraint // check whether it is partition table if (orig_table_schema.is_parent_table()) { // check whether it is self-referential if (orig_table_schema.is_child_table()) { const ObIArray &foreign_key_infos = orig_table_schema.get_foreign_key_infos(); FOREACH_CNT_X(foreign_key_info, foreign_key_infos, OB_SUCC(ret)) { if (orig_table_schema.get_table_id() != foreign_key_info->child_table_id_) { // If it is not self-referential, there are constraints ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK; LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK); } } } else { // If it is not self-referential, there are constraints const ObIArray &foreign_key_infos = orig_table_schema.get_foreign_key_infos(); FOREACH_CNT_X(foreign_key_info, foreign_key_infos, OB_SUCC(ret)) { if (orig_table_schema.get_table_id() == foreign_key_info->parent_table_id_) { ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK; LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK); } } } } return ret; } // drop or truncate partition //1. is partition table //2. Cannot drop all partitions, but truncate does not have this restriction //3. Ensure that all operating partitions exist1 //4. Currently this partition is not splitting //5. The partition type can only be list or range int ObDDLService::check_alter_drop_partitions(const share::schema::ObTableSchema &orig_table_schema, const obrpc::ObAlterTableArg &alter_table_arg, const bool is_truncate) { int ret = OB_SUCCESS; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const int64_t part_num = alter_table_schema.get_part_option().get_part_num(); ObPartition **part_array = alter_table_schema.get_part_array(); const ObPartitionLevel part_level = orig_table_schema.get_part_level(); const ObPartitionOption &part_option = orig_table_schema.get_part_option(); if (OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_array is null", K(ret), K(part_array)); } else if (PARTITION_LEVEL_ZERO == part_level) { ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED; LOG_WARN("unsupport management on non-partition table", K(ret)); } else if (!part_option.is_range_part() && !part_option.is_list_part()) { ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION; LOG_WARN("drop partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg)); } else if (OB_FAIL(check_table_pk(orig_table_schema))) { LOG_WARN("cannot drop/truncate partition with foreign keys", K(ret), K(alter_table_arg)); } else if (is_truncate) { } else if (alter_table_schema.get_part_option().get_part_num() >= orig_table_schema.get_part_option().get_part_num()) { ret = OB_ERR_DROP_LAST_PARTITION; LOG_WARN("cannot drop all partitions", "partitions current", orig_table_schema.get_part_option().get_part_num(), "partitions to be dropped", alter_table_schema.get_part_option().get_part_num(), K(ret)); LOG_USER_ERROR(OB_ERR_DROP_LAST_PARTITION); } if (OB_SUCC(ret)) { for (int64_t i = 0; OB_SUCC(ret) && i < part_num; ++i) { ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL; ObPartIterator iter(orig_table_schema, check_partition_mode); const ObPartition *part = NULL; while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) { if (OB_ISNULL(part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(part), K(ret)); } else if (ObCharset::case_insensitive_equal(part->get_part_name(), part_array[i]->get_part_name())) { break; } } if (OB_FAIL(ret)) { if (OB_ITER_END != ret) { LOG_WARN("iter failed", K(ret)); } else { ret = OB_ERR_DROP_PARTITION_NON_EXISTENT; LOG_WARN("partition to be dropped not exist", K(ret), "partition name", part_array[i]->get_part_name()); LOG_USER_ERROR(OB_ERR_DROP_PARTITION_NON_EXISTENT); } } } } return ret; } int ObDDLService::check_alter_drop_subpartitions(const share::schema::ObTableSchema &orig_table_schema, const obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; const ObPartitionLevel part_level = orig_table_schema.get_part_level(); const ObPartitionOption &subpart_option = orig_table_schema.get_sub_part_option(); if (PARTITION_LEVEL_ZERO == part_level || PARTITION_LEVEL_ONE == part_level) { ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED; LOG_WARN("unsupport management on non-partition table", K(ret)); } else if (!subpart_option.is_range_part() && !subpart_option.is_list_part()) { ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION; LOG_WARN("drop partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg)); } else if (OB_FAIL(check_table_pk(orig_table_schema))) { LOG_WARN("cannot drop/truncate partition with foreign keys", K(ret), K(alter_table_arg)); } return ret; } // filter out the partition which is same to orig_table_schema in alter_table_arg int ObDDLService::filter_out_duplicate_interval_part(const share::schema::ObTableSchema &orig_table_schema, share::schema::ObTableSchema &alter_table_schema) { int ret = OB_SUCCESS; int64_t j = 0; int64_t inc_num = 0; const ObRowkey *rowkey_orig= NULL; const int64_t inc_part_num = alter_table_schema.get_part_option().get_part_num(); const int64_t orig_part_num = orig_table_schema.get_part_option().get_part_num(); ObPartition **inc_part_array = alter_table_schema.get_part_array(); ObPartition **orig_part_array = orig_table_schema.get_part_array(); if (!orig_table_schema.is_interval_part()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("orig_table_schema is not interval part", K(ret), K(orig_table_schema), K(alter_table_schema)); } else if (OB_ISNULL(inc_part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } else if (OB_ISNULL(orig_part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("orig_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } else if (orig_table_schema.get_interval_range() != alter_table_schema.get_interval_range() || orig_table_schema.get_transition_point() != alter_table_schema.get_transition_point()) { ret = OB_ERR_INTERVAL_PARTITION_ERROR; LOG_WARN("interval_range or transition_point is changed", KR(ret), K(orig_table_schema), K(alter_table_schema)); } for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) { ObString empty_str; const ObRowkey *rowkey_cur = NULL; if (OB_ISNULL(inc_part_array[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } // interval part name is generated in rs, so reset. else if (OB_FAIL(inc_part_array[i]->set_part_name(empty_str))) { LOG_WARN("fail to set_part_name", KR(ret), K(orig_table_schema), K(alter_table_schema)); } else if (NULL == (rowkey_cur = &inc_part_array[i]->get_high_bound_val())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } while (OB_SUCC(ret) && j < orig_part_num) { if (OB_ISNULL(orig_part_array[j])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } else if (NULL == (rowkey_orig = &orig_part_array[j]->get_high_bound_val())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } else if (*rowkey_orig < *rowkey_cur) { j++; } else { break; } } if (OB_FAIL(ret)) { } else if (*rowkey_orig != *rowkey_cur) { if (inc_num != i) { inc_part_array[inc_num] = inc_part_array[i]; } inc_num++; } } if (OB_FAIL(ret)) { } else if (0 == inc_num) { LOG_INFO("all interval part for add is exist", K(alter_table_schema), K(orig_table_schema)); ret = OB_ERR_INTERVAL_PARTITION_EXIST; } else if (inc_num != inc_part_num) { alter_table_schema.set_part_num(inc_num); alter_table_schema.set_partition_num(inc_num); } return ret; } // add partition //1. ensure it is partition table //2. The number of new partitions cannot exceed the limit of the number of partitions //3. The partition name of the newly added partition cannot conflict with the previous partition //4. Partition type, currently only supports list or range partition //5. The value of the newly added partition is consistent with the previous one int ObDDLService::check_alter_add_partitions(const share::schema::ObTableSchema &orig_table_schema, obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const int64_t inc_part_num = alter_table_schema.get_part_option().get_part_num(); const int64_t orig_part_num = orig_table_schema.get_part_option().get_part_num(); ObPartition **inc_part_array = alter_table_schema.get_part_array(); ObPartition **orig_part_array = orig_table_schema.get_part_array(); const ObPartitionLevel part_level = orig_table_schema.get_part_level(); const ObPartitionOption &part_option = orig_table_schema.get_part_option(); bool is_oracle_mode = false; if (OB_ISNULL(inc_part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("inc_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } else if (OB_ISNULL(orig_part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("orig_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema)); } else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } else if (PARTITION_LEVEL_ZERO == part_level) { ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED; LOG_WARN("unsupport management on non-partition table", K(ret)); } else if (!part_option.is_range_part() && !part_option.is_list_part()) { ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION; LOG_WARN("add partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg)); } else if ((is_oracle_mode && OB_MAX_PARTITION_NUM_ORACLE < orig_table_schema.get_all_part_num() + inc_part_num) || (!is_oracle_mode && OB_MAX_PARTITION_NUM_MYSQL < orig_table_schema.get_all_part_num() + inc_part_num)) { ret = OB_TOO_MANY_PARTITIONS_ERROR; LOG_WARN("too many partitions", K(ret), "partition cnt current", orig_table_schema.get_all_part_num(), "partition cnt to be added", inc_part_num); } if (!orig_table_schema.is_interval_part()) { for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) { ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL; ObPartIterator iter(orig_table_schema, check_partition_mode); const ObPartition *part = NULL; ObPartition *inc_part = inc_part_array[i]; while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) { if (OB_ISNULL(part) || OB_ISNULL(inc_part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(part), K(ret)); } else if (ObCharset::case_insensitive_equal(part->get_part_name(), inc_part->get_part_name())) { ret = OB_ERR_SAME_NAME_PARTITION; LOG_WARN("duplicate partition name", K(ret), K(inc_part_array[i]->get_part_name())); LOG_USER_ERROR(OB_ERR_SAME_NAME_PARTITION, inc_part_array[i]->get_part_name().length(), inc_part_array[i]->get_part_name().ptr()); } else if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level()) { if (0 == inc_part->get_subpartition_num() || OB_ISNULL(inc_part->get_subpart_array())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("subpart_array is NULL", K(part), K(ret), K(i)); } else { for (int j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) { ObSubPartition *subpart = NULL; ObSubPartition *inc_subpart = inc_part->get_subpart_array()[j]; int64_t k = 0, subpart_num = part->get_subpartition_num(); for (k = 0; OB_SUCC(ret) && k < subpart_num; k++) { subpart = part->get_subpart_array()[k]; if (OB_ISNULL(subpart) || OB_ISNULL(inc_subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(part), K(inc_subpart), K(ret)); } else if (ObCharset::case_insensitive_equal(subpart->get_part_name(), inc_subpart->get_part_name())) { ret = OB_ERR_SAME_NAME_SUBPARTITION; LOG_WARN("duplicate subpartition name", K(ret), K(subpart->get_part_name())); LOG_USER_ERROR(OB_ERR_SAME_NAME_SUBPARTITION, subpart->get_part_name().length(), subpart->get_part_name().ptr()); } } } } } } if (OB_ITER_END == ret) { ret = OB_SUCCESS; } }// end for } // check the part of inc part is increased if (OB_FAIL(ret)) { } else if (orig_table_schema.is_range_part()) { const ObRowkey *rowkey_last = NULL; if (orig_table_schema.is_interval_part()) { rowkey_last = &orig_table_schema.get_transition_point(); } else if (OB_ISNULL(orig_part_array[orig_part_num - 1])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", K(ret), K(orig_part_array), K(alter_table_schema)); } else { rowkey_last = &orig_part_array[orig_part_num - 1]->get_high_bound_val(); } if (OB_FAIL(ret)) { } else if (OB_ISNULL(rowkey_last)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", K(ret), K(orig_part_array), K(alter_table_schema)); } for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) { const ObRowkey *rowkey_cur = &inc_part_array[i]->get_high_bound_val(); if (*rowkey_cur <= *rowkey_last) { ret = OB_ERR_ADD_PART_BOUN_NOT_INC; LOG_WARN("range values should increasing", K(ret), K(rowkey_cur), K(rowkey_last)); LOG_USER_ERROR(OB_ERR_ADD_PART_BOUN_NOT_INC); } else { rowkey_last = rowkey_cur; } } if (OB_SUCC(ret) && orig_table_schema.is_interval_part() && filter_out_duplicate_interval_part(orig_table_schema, alter_table_schema)) { LOG_WARN("fail to filter out duplicate interval part", KR(ret), K(orig_table_schema), K(alter_table_schema)); } } else if (orig_table_schema.is_list_part()) { if (OB_FAIL(check_add_list_partition(orig_table_schema, alter_table_schema))) { LOG_WARN("failed to check add list partition", K(ret), K(orig_table_schema), K(alter_table_schema)); } } return ret; } int ObDDLService::check_alter_add_subpartitions(const share::schema::ObTableSchema &orig_table_schema, const obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; int64_t part_num = 0; ObArray orig_parts; // get count of all partitions for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_partition_num(); i++) { if (OB_ISNULL(alter_table_schema.get_part_array()[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_array[i] is null", K(ret), K(i)); } else { part_num += alter_table_schema.get_part_array()[i]->get_subpartition_num(); } } ObPartition **part_array = alter_table_schema.get_part_array(); const ObPartitionLevel part_level = orig_table_schema.get_part_level(); const ObPartitionOption &subpart_option = orig_table_schema.get_sub_part_option(); bool is_oracle_mode = false; if (OB_SUCC(ret) && OB_ISNULL(part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part_array is null", K(ret), K(part_array)); } else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } else if (PARTITION_LEVEL_ZERO == part_level) { ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED; LOG_WARN("unsupport management on non-partition table", K(ret)); } else if (!subpart_option.is_range_part() && !subpart_option.is_list_part()) { ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION; LOG_WARN("add partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg)); } else if ((is_oracle_mode && OB_MAX_PARTITION_NUM_ORACLE < orig_table_schema.get_all_part_num() + part_num) || (!is_oracle_mode && OB_MAX_PARTITION_NUM_MYSQL < orig_table_schema.get_all_part_num() + part_num)) { ret = OB_TOO_MANY_PARTITIONS_ERROR; LOG_WARN("too many partitions", K(ret), "partition cnt current", orig_table_schema.get_all_part_num(), "partition cnt to be added", part_num); } // To add a subpartition separately, verify whether the partition exists // and whether the subpartition does not exist for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_partition_num(); ++i) { ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL; ObPartIterator iter(orig_table_schema, check_partition_mode); const ObPartition *part = NULL; const ObPartition *inc_part = part_array[i]; while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) { if (OB_ISNULL(part) || OB_ISNULL(inc_part)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(part), K(ret)); } else if (ObCharset::case_insensitive_equal(part->get_part_name(), inc_part->get_part_name())) { if (0 == inc_part->get_subpartition_num() || OB_ISNULL(inc_part->get_subpart_array())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("nontemplate subpart_array is NULL", K(part), K(ret), K(i)); } else if (OB_FAIL(orig_parts.push_back(part))) { LOG_WARN("fail to push back orig parts", KR(ret), KPC(part)); } else { for (int j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) { ObSubPartition *subpart = NULL; ObSubPartition *inc_subpart = inc_part->get_subpart_array()[j]; int64_t k = 0, subpart_num = part->get_subpartition_num(); for (k = 0; OB_SUCC(ret) && k < subpart_num; k++) { subpart = part->get_subpart_array()[k]; if (OB_ISNULL(subpart) || OB_ISNULL(inc_subpart)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(part), K(inc_subpart), K(ret)); } else if (ObCharset::case_insensitive_equal(subpart->get_part_name(), inc_subpart->get_part_name())) { ret = OB_ERR_SAME_NAME_SUBPARTITION; LOG_WARN("duplicate subpartition name", K(ret), K(subpart->get_part_name())); LOG_USER_ERROR(OB_ERR_SAME_NAME_SUBPARTITION, subpart->get_part_name().length(), subpart->get_part_name().ptr()); } } if (OB_SUCC(ret) && k >= subpart_num) { ret = OB_SUCCESS; } } } break; } } if (OB_ITER_END == ret) { ret = OB_PARTITION_NOT_EXIST; LOG_WARN("duplicate partition name", K(ret), K(inc_part->get_part_name())); LOG_USER_ERROR(OB_PARTITION_NOT_EXIST); } }// end for // check whether the value of list and range is valid for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_partition_num(); ++i) { ObPartition *inc_part = part_array[i]; int64_t inc_subpart_num = inc_part->get_subpartition_num(); ObSubPartition ** inc_subpart_array = inc_part->get_subpart_array(); if (OB_ISNULL(inc_part) || OB_ISNULL(inc_subpart_array) || 0 == inc_subpart_num) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(inc_part), K(inc_subpart_num), K(ret)); } else if (alter_table_schema.get_partition_num() != orig_parts.count()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("orig_parts count not equal inc__part count", K(orig_parts.count()), K(inc_subpart_num), K(ret)); } else if (orig_table_schema.is_range_subpart()) { const int64_t orig_subpart_num = orig_parts.at(i)->get_subpartition_num(); ObSubPartition **orig_subpart_array = orig_parts.at(i)->get_subpart_array(); if (OB_ISNULL(orig_subpart_array) || OB_ISNULL(orig_subpart_array[orig_subpart_num - 1])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(ret), K(orig_subpart_array), K(orig_subpart_array[orig_subpart_num - 1])); } else { const ObRowkey *rowkey_last = &(orig_subpart_array[orig_subpart_num - 1]->get_high_bound_val()); for (int64_t j = 0; OB_SUCC(ret) && j < inc_subpart_num; ++j) { const ObRowkey *rowkey_cur = &inc_subpart_array[j]->get_high_bound_val(); if (*rowkey_cur <= *rowkey_last) { ret = OB_ERR_ADD_PART_BOUN_NOT_INC; LOG_WARN("range values should increasing", K(ret), K(rowkey_cur), K(rowkey_last)); } else { rowkey_last = rowkey_cur; } } } } else if (orig_table_schema.is_list_subpart()) { if (OB_FAIL(check_add_list_subpartition(*orig_parts.at(i), *inc_part))) { LOG_WARN("failed to check add list partition", K(ret), K(orig_table_schema), K(alter_table_schema)); } } } return ret; } // Check whether the newly added partition has a duplicate value with the old partition int ObDDLService::check_add_list_partition(const share::schema::ObPartitionSchema &orig_part, const share::schema::ObPartitionSchema &new_part, const int64_t split_part_id) { int ret = OB_SUCCESS; int64_t part_num = new_part.get_partition_num(); ObPartition **part_array = new_part.get_part_array(); int64_t orig_part_num = orig_part.get_partition_num(); ObPartition **orig_part_array = orig_part.get_part_array(); common::ObRowkey row_key; if (OB_ISNULL(part_array) || OB_ISNULL(orig_part_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part array is null", K(ret), KP(part_array), KP(orig_part_array)); } else { common::hash::ObHashSet list_row_map; if (OB_FAIL(list_row_map.create(hash::cal_next_prime(part_num), ObModIds::OB_PARTITION_SPLIT, ObModIds::OB_PARTITION_SPLIT))) { LOG_WARN("failed to create list value", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < orig_part_num; ++i) { if (OB_ISNULL(orig_part_array[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", K(ret), K(i), K(orig_part)); } else if (split_part_id == orig_part_array[i]->get_part_id()) { // Ensure that the split partition, except for the split partition, // cannot conflict with other partitions continue; } else { const ObIArray* orig_list_value = &(orig_part_array[i]->get_list_row_values()); if (OB_ISNULL(orig_list_value)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("list row value is null", K(ret), K(orig_list_value)); } for (int j = 0; OB_SUCC(ret) && j < orig_list_value->count(); ++j) { const common::ObNewRow *new_row = &(orig_list_value->at(j)); if (1 == new_row->get_count() && new_row->get_cell(0).is_max_value()) { ret = OB_ERR_ADD_PARTITION_TO_DEFAULT_LIST; LOG_WARN("can add a table has default partition", K(ret), K(orig_part_array)); LOG_USER_ERROR(OB_ERR_ADD_PARTITION_TO_DEFAULT_LIST); } else { row_key.reset(); row_key.assign(new_row->cells_, new_row->get_count()); if (OB_FAIL(list_row_map.set_refactored(row_key))) { LOG_WARN("failed to insert hash map", K(ret), K(row_key)); } } } } }// end for for (int64_t i = 0; OB_SUCC(ret) && i < part_num; ++i) { if (OB_ISNULL(part_array[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", K(ret), K(i), K(new_part)); } else { const ObIArray* list_value = &(part_array[i]->get_list_row_values()); if (OB_ISNULL(list_value)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("list row value is null", K(ret), K(list_value)); } for (int j = 0; OB_SUCC(ret) && j < list_value->count(); ++j) { row_key.reset(); row_key.assign(list_value->at(j).cells_, list_value->at(j).get_count()); if (OB_HASH_NOT_EXIST != list_row_map.exist_refactored(row_key)) { ret = OB_ERR_MULTIPLE_DEF_CONST_IN_LIST_PART; LOG_WARN("add partition values duplicate to orig table", K(ret), K(row_key)); LOG_USER_ERROR(OB_ERR_MULTIPLE_DEF_CONST_IN_LIST_PART); } }//end for } }// end for } } return ret; } // Check whether the newly added partition has a duplicate value with the old partition int ObDDLService::check_add_list_subpartition(const ObPartition &orig_part, const ObPartition &new_part) { int ret = OB_SUCCESS; int64_t subpart_num = new_part.get_subpartition_num(); ObSubPartition **subpart_array = new_part.get_subpart_array(); int64_t orig_subpart_num = orig_part.get_subpartition_num(); ObSubPartition **orig_subpart_array = orig_part.get_subpart_array(); common::ObRowkey row_key; if (OB_ISNULL(subpart_array) || OB_ISNULL(orig_subpart_array)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part array is null", K(ret), KP(subpart_array), KP(orig_subpart_array)); } else { common::hash::ObHashSet list_row_map; if (OB_FAIL(list_row_map.create(hash::cal_next_prime(subpart_num), "ChkAddLstSPar", "ChkAddLstSPar"))) { LOG_WARN("failed to create list value", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < orig_subpart_num; ++i) { if (OB_ISNULL(orig_subpart_array[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", K(ret), K(i), K(orig_part)); } else { const ObIArray* orig_list_value = &(orig_subpart_array[i]->get_list_row_values()); if (OB_ISNULL(orig_list_value)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("list row value is null", K(ret), K(orig_list_value)); } for (int j = 0; OB_SUCC(ret) && j < orig_list_value->count(); ++j) { const common::ObNewRow *new_row = &(orig_list_value->at(j)); if (1 == new_row->get_count() && new_row->get_cell(0).is_max_value()) { ret = OB_ERR_ADD_PARTITION_TO_DEFAULT_LIST; LOG_WARN("can add a table has default partition", K(ret), K(orig_subpart_array)); LOG_USER_ERROR(OB_ERR_ADD_PARTITION_TO_DEFAULT_LIST); } else { row_key.reset(); row_key.assign(new_row->cells_, new_row->get_count()); if (OB_FAIL(list_row_map.set_refactored(row_key))) { LOG_WARN("failed to insert hash map", K(ret), K(row_key)); } } } } }// end for for (int64_t i = 0; OB_SUCC(ret) && i < subpart_num; ++i) { if (OB_ISNULL(subpart_array[i])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("part is null", K(ret), K(i), K(new_part)); } else { const ObIArray* list_value = &(subpart_array[i]->get_list_row_values()); if (OB_ISNULL(list_value)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("list row value is null", K(ret), K(list_value)); } for (int j = 0; OB_SUCC(ret) && j < list_value->count(); ++j) { row_key.reset(); row_key.assign(list_value->at(j).cells_, list_value->at(j).get_count()); if (OB_HASH_NOT_EXIST != list_row_map.exist_refactored(row_key)) { ret = OB_ERR_MULTIPLE_DEF_CONST_IN_LIST_PART; LOG_WARN("add partition values duplicate to orig table", K(ret), K(row_key)); LOG_USER_ERROR(OB_ERR_MULTIPLE_DEF_CONST_IN_LIST_PART); } }//end for } }// end for } } return ret; } int ObDDLService::alter_table(obrpc::ObAlterTableArg &alter_table_arg, const int64_t frozen_version, obrpc::ObAlterTableRes &res) { int ret = OB_SUCCESS; int64_t start_usec = 0; int64_t end_usec = 0; int64_t cost_usec = 0; start_usec = ObTimeUtility::current_time(); bool is_alter_sess_active_time = false; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); int64_t &task_id = res.task_id_; ObDDLType &ddl_type = res.ddl_type_; ddl_type = DDL_INVALID; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } ObSimpleTableSchemaV2 orig_table; if (OB_SUCC(ret)) { AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(alter_table_arg.session_id_); const ObTableSchema *orig_table_schema = NULL; is_alter_sess_active_time = alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::SESSION_ACTIVE_TIME); ObTZMapWrap tz_map_wrap; if (OB_FAIL(ret)) { } else if (OB_FAIL(OTTZ_MGR.get_tenant_tz(tenant_id, tz_map_wrap))) { LOG_WARN("get tenant timezone map failed", K(ret), K(tenant_id)); } else if (FALSE_IT(alter_table_arg.set_tz_info_map(tz_map_wrap.get_tz_map()))) { } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (false == is_alter_sess_active_time) { const ObString &origin_database_name = alter_table_schema.get_origin_database_name(); const ObString &origin_table_name = alter_table_schema.get_origin_table_name(); if (origin_database_name.empty() || origin_table_name.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("database name or table name is null", K(alter_table_schema), K(origin_database_name), K(origin_table_name), K(ret)); } else { if (OB_FAIL(schema_guard.get_table_schema(tenant_id, origin_database_name, origin_table_name, false, orig_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(origin_database_name), K(origin_table_name)); } else if (NULL == orig_table_schema) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(orig_table_schema), K(ret)); } else if (OB_FAIL(orig_table.assign(*orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } } } // check schema if (OB_SUCC(ret) && alter_table_arg.is_alter_partitions_ && obrpc::ObAlterTableArg::REPARTITION_TABLE != alter_table_arg.alter_part_type_) { if (OB_ISNULL(orig_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema is null", KR(ret), K(alter_table_arg)); } else { if (obrpc::ObAlterTableArg::DROP_PARTITION != alter_table_arg.alter_part_type_) { if ((alter_table_arg.alter_table_schema_.is_range_part() && !alter_table_arg.alter_table_schema_.is_interval_part()) || alter_table_arg.alter_table_schema_.is_list_part()) { if (OB_FAIL(fill_part_name(*orig_table_schema, alter_table_arg.alter_table_schema_))) { LOG_WARN("failed to fill part name", K(ret)); } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(check_alter_partitions(*orig_table_schema, alter_table_arg))) { LOG_WARN("check alter partitions failed", K(ret), K(orig_table_schema), K(alter_table_arg)); } } } //do alter table in transaction if (OB_SUCC(ret)) { if (is_alter_sess_active_time) { if (OB_FAIL(alter_table_sess_active_time_in_trans(alter_table_arg, frozen_version, res))) { LOG_WARN("alter_table_in_trans failed", K(frozen_version), K(ret)); } else { LOG_INFO("refresh session active time of temp tables succeed!", K(ret)); } } else if (OB_FAIL(check_is_offline_ddl(alter_table_arg, ddl_type))) { LOG_WARN("failed to check is offline ddl", K(ret)); } else { // offline ddl cannot appear at the same time with other ddl types if (is_long_running_ddl(ddl_type)) { if (OB_FAIL(do_offline_ddl_in_trans(alter_table_arg, frozen_version, res))) { LOG_WARN("failed to do offline ddl in trans", K(ret), K(alter_table_arg), K(ddl_type)); } } else { if (OB_FAIL(alter_table_in_trans(alter_table_arg, frozen_version, res))) { LOG_WARN("alter_table_in_trans failed", K(frozen_version), K(ret)); } } } end_usec = ObTimeUtility::current_time(); cost_usec = end_usec - start_usec; start_usec = end_usec; LOG_INFO("alter_table_in_trans cost: ", K(cost_usec), K(ddl_type)); } } // just for debug if (OB_SUCC(ret) && false == is_alter_sess_active_time) { int tmp_ret = OB_SUCCESS; const uint64_t table_id = (OB_INVALID_ID == orig_table.get_table_id()) ? alter_table_arg.alter_table_schema_.get_table_id() : orig_table.get_table_id(); const ObTableSchema *new_table_schema = NULL; HEAP_VAR(ObSchemaGetterGuard, new_schema_guard) { if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, new_schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(new_schema_guard.get_table_schema(tenant_id, table_id, new_table_schema))) { LOG_WARN("fail to get table schema", K(tmp_ret), K(tenant_id), K(table_id)); } else if (NULL == new_table_schema) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL ptr", K(tmp_ret)); } } // check const_column_iterator and ObColumnIterByPrevNextID if (OB_SUCC(ret)) { ObTableSchema::const_column_iterator iter = new_table_schema->column_begin(); ObColumnIterByPrevNextID iter2(*new_table_schema); const ObColumnSchemaV2 *col = NULL; while (OB_SUCC(ret) && OB_SUCC(iter2.next(col))) { if (new_table_schema->column_end() == iter) { ret = OB_ERR_UNEXPECTED; LOG_WARN("Inconsistent iterators: const_column_iterator is less than ObColumnIterByPrevNextID", K(ret)); } else { ++iter; } } if (ret != OB_ITER_END) { LOG_WARN("Failed to iterate all table columns. iter quit. ", K(ret)); } else if (iter != new_table_schema->column_end()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("Inconsistent iterators: const_column_iterator is bigger than ObColumnIterByPrevNextID", K(ret)); } else { ret = OB_SUCCESS; } } } return ret; } int ObDDLService::rename_table(const obrpc::ObRenameTableArg &rename_table_arg) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObSchemaGetterGuard schema_guard; const uint64_t tenant_id = rename_table_arg.tenant_id_; bool is_oracle_mode = false; bool sequence_exist = false; RenameOracleObjectType rename_oracle_obj_type = RENAME_TYPE_INVALID; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { ObSchemaService *schema_service = schema_service_->get_schema_service(); lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::MYSQL; if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_guard or scheam service is null", K(schema_service), K(ret)); } else if (OB_INVALID_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant_id is invalid", K(tenant_id), K(ret)); } else if (is_virtual_tenant_id(tenant_id) || OB_SYS_TENANT_ID == tenant_id) { compat_mode = lib::Worker::CompatMode::MYSQL; is_oracle_mode = false; } else if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) { LOG_WARN("failed to get compat mode", K(ret), K(tenant_id)); } else { if (lib::Worker::CompatMode::ORACLE == compat_mode) { is_oracle_mode = true; } else { is_oracle_mode = false; } if (is_oracle_mode && rename_table_arg.rename_table_items_.size() > 1) { ret = OB_ERR_ALTER_TABLE_RENAME_WITH_OPTION; LOG_WARN("alter table rename can't be combined with other operations in oracle mode", K(ret), K(rename_table_arg)); } } ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(ret)) { } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { //todo use array to replace hashmap and hashset @hualong //record table already be renamed in the schema mgr common::hash::ObPlacementHashSet delete_table_set; //record new table name set //table_item -> table_id common::hash::ObHashMap new_table_map; if (OB_FAIL(new_table_map.create(32, ObModIds::OB_HASH_BUCKET_RENAME_TABLE_MAP))) { LOG_WARN("failed to add create ObHashMap", K(ret)); } for (int32_t i = 0; OB_SUCC(ret) && i < rename_table_arg.rename_table_items_.size(); ++i) { const ObRenameTableItem &rename_item = rename_table_arg.rename_table_items_.at(i); bool in_new_table_set = false; //table has be rename before if (!rename_item.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("rename table item is invalid !", K(rename_item), K(ret)); } else { //check table duplicate ObTableItem to_table_item; to_table_item.table_name_ = rename_item.new_table_name_; to_table_item.database_name_ = rename_item.new_db_name_; uint64_t table_id = OB_INVALID_ID; const ObDatabaseSchema *database_schema = NULL; if (OB_FAIL(schema_guard.get_tenant_name_case_mode(tenant_id, to_table_item.mode_))) { LOG_WARN("failed to get tenant name case mode!", K(tenant_id), K(to_table_item), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, to_table_item.database_name_, database_schema))) { LOG_WARN("get_database_schema failed", K(ret), K(to_table_item)); } else if(OB_ISNULL(database_schema)) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, rename_item.new_db_name_.length(), rename_item.new_db_name_.ptr()); LOG_WARN("database schema is null", K(ret)); } else { const ObTableSchema *to_table_schema = NULL; const ObSynonymInfo *synonym_info = NULL; ObArray conflict_schema_types; if (ObString(OB_RECYCLEBIN_SCHEMA_NAME) == to_table_item.database_name_ || ObString(OB_PUBLIC_SCHEMA_NAME) == to_table_item.database_name_) { ret = OB_OP_NOT_ALLOW; LOG_WARN("rename table to __recyclebin database is not allowd", K(to_table_item)); } else if (OB_FAIL(schema_guard.check_oracle_object_exist(tenant_id, database_schema->get_database_id(), to_table_item.table_name_, OB_MAX_SCHEMA, INVALID_ROUTINE_TYPE, false, conflict_schema_types))) { LOG_WARN("fail to check oracle_object exist", K(ret), K(to_table_item.table_name_)); } else if (conflict_schema_types.count() > 0) { // Check whether the name of the new object after rename in oracle mode // is already occupied by other objects ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object in oralce mode", K(ret), K(to_table_item.table_name_)); } // check similar to rename table t1 to t3, t2 to t3 in mysql mode if (OB_FAIL(ret)) { } else if (OB_SUCCESS == new_table_map.get_refactored(to_table_item, table_id)) { //already had t1,t2 //rename table t1 to t3, t2 to t3(t3 exist!) ret = OB_ERR_TABLE_EXIST; LOG_USER_ERROR(OB_ERR_TABLE_EXIST, to_table_item.table_name_.length(), to_table_item.table_name_.ptr()); } else if (sequence_exist) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing sequence", K(ret), K(to_table_item)); } else if (OB_FAIL(schema_guard.get_synonym_info(tenant_id, database_schema->get_database_id(), to_table_item.table_name_, synonym_info))) { LOG_WARN("fail to check synonym exist", K(to_table_item), K(ret)); } else if (NULL != synonym_info) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing synonym", K(ret), K(to_table_item)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, to_table_item.database_name_, to_table_item.table_name_, false, to_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(to_table_item.database_name_), K(to_table_item.table_name_)); } else if (NULL != to_table_schema) { int hash_ret = OB_SUCCESS; hash_ret = delete_table_set.exist_refactored(to_table_item); if (OB_HASH_EXIST != hash_ret && OB_HASH_NOT_EXIST != hash_ret) { ret = hash_ret; LOG_WARN("delete_table_set check failed!", K(to_table_item), K(ret)); } else if (OB_HASH_NOT_EXIST == hash_ret) { //already had t1,t2 //rename table t2 to t1 (t1 exist!) if (!is_oracle_mode) { ret = OB_ERR_TABLE_EXIST; LOG_USER_ERROR(OB_ERR_TABLE_EXIST, to_table_item.table_name_.length(), to_table_item.table_name_.ptr()); LOG_WARN("table already exist!", K(to_table_item), K(ret)); } else { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(ret), K(to_table_item)); } } else if (OB_HASH_EXIST == hash_ret) { //already had t1,t2 //rename table t1 to t3, t2 to t1(success!) //[delete(t1), new(t3)] -> [delete(t1, t2), new(t3, t1)] LOG_INFO("rename to new table name", K(rename_item)); } } } const ObTableSchema *from_table_schema = NULL; const ObSynonymInfo *synonym_info = NULL; const ObSequenceSchema *sequence_schema = NULL; ObTableItem from_table_item; from_table_item.table_name_ = rename_item.origin_table_name_; from_table_item.database_name_ = rename_item.origin_db_name_; // Determine the object type in oracle mode if (is_oracle_mode && OB_SUCC(ret)) { if (OB_FAIL(check_rename_object_type(schema_guard, tenant_id, database_schema->get_database_id(), from_table_item.table_name_, from_table_schema, synonym_info, sequence_schema, rename_oracle_obj_type))) { LOG_WARN("fail to check rename object type!", K(ret), K(tenant_id), K(database_schema->get_database_id()), K(from_table_item.table_name_)); } } if (OB_SUCC(ret) && OB_FAIL(schema_guard.get_tenant_name_case_mode(tenant_id, from_table_item.mode_))) { LOG_WARN("failed to get tenant name case mode!", K(tenant_id), K(from_table_item), K(ret)); } if (OB_SUCC(ret)) { table_id = OB_INVALID_ID; if (OB_SUCCESS != new_table_map.get_refactored(from_table_item, table_id)) { if (OB_FAIL(schema_guard.get_table_schema(tenant_id, from_table_item.database_name_, from_table_item.table_name_, false, from_table_schema))) { LOG_WARN("fail to get table schema", K(ret)); } else if (NULL != from_table_schema) { bool is_db_in_recyclebin = false; bool has_mv = false; if (from_table_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("Can not perform operation in recyclebin", K(ret), K(from_table_item)); } else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id, from_table_schema->get_database_id(), is_db_in_recyclebin))) { LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), KPC(from_table_schema)); } else if (is_db_in_recyclebin) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not rename table in recyclebin", K(ret), K(from_table_item)); } else if (OB_FAIL(check_table_has_materialized_view(schema_guard, *from_table_schema, has_mv))) { LOG_WARN("fail to check table has materialized view", K(ret), K(*from_table_schema)); } else if (has_mv) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support rename table has materialized view", K(ret)); } else if (OB_HASH_EXIST == delete_table_set.exist_refactored(from_table_item)) { //already had t1,t2 //rename table t1 to table1, t1 to t3 (t1 not exist) ret = OB_FILE_NOT_EXIST; LOG_WARN("table not exist!", K(from_table_item), K(ret)); } else if (OB_FAIL(check_enable_sys_table_ddl(*from_table_schema, OB_DDL_TABLE_RENAME))) { LOG_WARN("rename table is not allowed on system table", K(ret)); } else { in_new_table_set = false; //has not been rename before LOG_WARN("orgin table exist! can do rename", K(rename_item), K(in_new_table_set)); } } else { //already had t1,t2 //rename table t3 to table3 (t3 not exist) if (!is_oracle_mode) { ret = OB_FILE_NOT_EXIST; LOG_WARN("table not exist!", K(rename_item), K(ret)); } else { LOG_INFO("oracle mode rename A to B", K(ret), K(rename_oracle_obj_type)); } } } else { //rename table t1 to t2, t2 to t3 //[new(t2), delete(t1)] -> [new(t3), delete(t1)] in_new_table_set = true; LOG_INFO("origin table exist in new table set", K(rename_item), K(in_new_table_set)); if (OB_INVALID_ID == table_id) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table id is invalid ", K(table_id), K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, from_table_schema))) { LOG_WARN("get_table_schema failed", K(ret), K(tenant_id), K(table_id)); } else if (OB_ISNULL(from_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema is null", K(table_id), K(ret)); } } } if (OB_SUCC(ret) && !is_oracle_mode && OB_FAIL(check_cst_name_dup_for_rename_table_mysql(schema_guard, from_table_schema, database_schema->get_database_id()))) { LOG_WARN("check cst_name dup for rename table in mysql mode failed", K(ret)); } if (OB_SUCC(ret)) { ObSqlString sql; if (!is_oracle_mode) { if (OB_FAIL(sql.append_fmt("RENAME TABLE %.*s.%.*s TO %.*s.%.*s", rename_item.origin_db_name_.length(), rename_item.origin_db_name_.ptr(), rename_item.origin_table_name_.length(), rename_item.origin_table_name_.ptr(), rename_item.new_db_name_.length(), rename_item.new_db_name_.ptr(), rename_item.new_table_name_.length(), rename_item.new_table_name_.ptr()))) { LOG_WARN("failed to append sql", K(ret)); } } else { // oracle mode if (OB_FAIL(sql.append_fmt("RENAME %.*s TO %.*s", rename_item.origin_table_name_.length(), rename_item.origin_table_name_.ptr(), rename_item.new_table_name_.length(), rename_item.new_table_name_.ptr()))) { LOG_WARN("failed to append sql", K(ret)); } } if (OB_SUCC(ret) && !is_oracle_mode) { ObString rename_sql = sql.string(); if (database_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not rename table in recyclebin", K(ret), K(to_table_item), K(tenant_id)); } else if (OB_FAIL(ddl_operator.rename_table(*from_table_schema, to_table_item.table_name_, database_schema->get_database_id(), trans, &rename_sql))) { LOG_WARN("failed to rename table!", K(rename_item), K(table_id), K(ret)); } else if (OB_FAIL(rebuild_trigger_package(schema_guard, tenant_id, from_table_schema->get_trigger_list(), database_schema->get_database_name_str(), to_table_item.table_name_, ddl_operator, trans))) { LOG_WARN("failed to rebuild trigger package", K(rename_item), K(ret)); } else { table_id = from_table_schema->get_table_id(); if (OB_FAIL(new_table_map.set_refactored(to_table_item, table_id))) { LOG_WARN("failed to insert into table map", K(to_table_item), K(table_id), K(ret)); } else { if (!in_new_table_set) { if (OB_FAIL(delete_table_set.set_refactored(from_table_item))) { LOG_WARN("failed to add to delete table set", K(from_table_item), K(ret)); } } else { //rename table t1 to t3, t3 to t4, t2 to t3; //should drop t3, when execute t3 to t4 if (OB_FAIL(new_table_map.erase_refactored(from_table_item))) { LOG_WARN("failed to delete from new table map!", K(from_table_item), K(ret)); } } } } } if (OB_SUCC(ret) && is_oracle_mode) { ObString rename_sql = sql.string(); if (database_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not rename table in recyclebin", K(ret), K(to_table_item), K(tenant_id)); } else { if (RENAME_TYPE_TABLE_OR_VIEW == rename_oracle_obj_type) { if (OB_FAIL(ddl_operator.rename_table(*from_table_schema, to_table_item.table_name_, database_schema->get_database_id(), trans, &rename_sql))) { LOG_WARN("failed to rename table!", K(ret), K(rename_item), K(table_id)); } else if (OB_FAIL(rebuild_trigger_package(schema_guard, tenant_id, from_table_schema->get_trigger_list(), database_schema->get_database_name_str(), to_table_item.table_name_, ddl_operator, trans))) { LOG_WARN("failed to rebuild trigger package", K(rename_item), K(ret)); } } else if (RENAME_TYPE_SYNONYM == rename_oracle_obj_type) { ObSynonymInfo tmp_synonym_info(*synonym_info); if (OB_FAIL(tmp_synonym_info.set_synonym_name(to_table_item.table_name_))) { LOG_WARN("failed to set new synonym name to synonym_info", K(ret), K(to_table_item.table_name_), KPC(synonym_info)); } else if (OB_FAIL(ddl_operator.replace_synonym(tmp_synonym_info, trans, &rename_sql))) { LOG_WARN("failed to rename synonym", K(ret), K(tmp_synonym_info)); } } else if (RENAME_TYPE_SEQUENCE == rename_oracle_obj_type) { ObSequenceDDLProxy ddl_operator(*schema_service_); ObSequenceSchema tmp_sequence_schema = *sequence_schema; if (OB_FAIL(tmp_sequence_schema.set_sequence_name(to_table_item.table_name_))) { LOG_WARN("failed to set new sequence name to sequence_schema", K(ret), K(to_table_item.table_name_), KPC(sequence_schema)); } else if (OB_FAIL(ddl_operator.rename_sequence(tmp_sequence_schema, trans, &rename_sql))) { LOG_WARN("failed to rename sequence", K(ret), K(tmp_sequence_schema)); } } } } } } } // end for if (OB_SUCC(ret) && !is_oracle_mode) { ObArray mock_fk_parent_table_schema_array; for (int64_t i = 0; OB_SUCC(ret) && i < rename_table_arg.rename_table_items_.size(); ++i) { const ObRenameTableItem &rename_item = rename_table_arg.rename_table_items_.at(i); uint64_t from_table_id = OB_INVALID_ID; const ObDatabaseSchema *to_database_schema = NULL; ObTableItem to_table_item; to_table_item.database_name_ = rename_item.new_db_name_; to_table_item.table_name_ = rename_item.new_table_name_; if (OB_FAIL(schema_guard.get_tenant_name_case_mode(tenant_id, to_table_item.mode_))) { LOG_WARN("failed to get tenant name case mode!", K(ret), K(tenant_id), K(to_table_item)); } else if (OB_FAIL(new_table_map.get_refactored(to_table_item, from_table_id))) { if (OB_HASH_NOT_EXIST == ret) { ret = OB_SUCCESS; // continue } else { LOG_WARN("get_refactored from new_table_map failed", K(ret), K(to_table_item), K(to_table_item.mode_), K(from_table_id)); } } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, to_table_item.database_name_, to_database_schema))) { LOG_WARN("get_database_schema failed", K(ret), K(to_table_item)); } else if (OB_ISNULL(to_database_schema)) { ret = OB_ERR_UNEXPECTED; SQL_RESV_LOG(WARN, "to_database_schema is null", K(ret), K(rename_item)); } else { const ObMockFKParentTableSchema *ori_mock_parent_table_schema_ptr = NULL; if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name( tenant_id, to_database_schema->get_database_id(), to_table_item.table_name_, ori_mock_parent_table_schema_ptr))) { SQL_RESV_LOG(WARN, "failed to check_mock_fk_parent_table_exist_with_name", K(ret)); } else if (OB_NOT_NULL(ori_mock_parent_table_schema_ptr)) { // will replace mock_parent_table with this new table const ObTableSchema *from_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(rename_table_arg.tenant_id_, from_table_id, from_table_schema))) { LOG_WARN("fail to get from_table_schema schema", K(ret)); } else if (OB_ISNULL(from_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("to_table_schema is null", K(ret)); } else { ObArray simple_index_infos; ObArray unique_index_schemas; if (OB_FAIL(from_table_schema->get_simple_index_infos(simple_index_infos))) { SERVER_LOG(WARN, "get simple_index_infos without delay_deleted_tid failed", K(ret)); } else { const ObTableSchema *index_schema = NULL; for (int64_t j = 0; OB_SUCC(ret) && j < simple_index_infos.count(); ++j) { if (OB_FAIL(schema_guard.get_table_schema(rename_table_arg.tenant_id_, simple_index_infos.at(j).table_id_, index_schema))) { LOG_WARN("fail to get to_table_schema schema", K(ret)); } else if (OB_ISNULL(index_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index_schema is null", K(ret)); } else if (index_schema->is_unique_index() && OB_FAIL(unique_index_schemas.push_back(index_schema))) { LOG_WARN("fail to push_back index_schema to unique_index_schemas", K(ret)); } } if (OB_SUCC(ret)) { ObMockFKParentTableSchema mock_fk_parent_table_schema; if (OB_FAIL(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table( schema_guard, ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id(), *from_table_schema, unique_index_schemas, mock_fk_parent_table_schema))) { LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table", K(ret)); } else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) { LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema)); } } } } } } } // end for if (FAILEDx(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) { LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(tenant_id)); } } } // trans.start if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } //refresh table schema if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("refresh_schema failed", K(ret)); } } } // get_schema_guard } // ddl_operator return ret; } int ObDDLService::truncate_table_in_trans(const obrpc::ObTruncateTableArg &arg, const ObTableSchema &orig_table_schema, ObIArray &table_schemas, const ObIArray &index_recycle_objs, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, const ObString *ddl_stmt_str, const ObString &database_name) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObSchemaService *schema_service = schema_service_->get_schema_service(); ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObSArray audit_schemas; ObArray orig_obj_privs_pointer_ora; ObArray orig_obj_privs_ora; const bool to_recyclebin = false; if (table_schemas.count() < 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schemas have no element", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service is null", K(ret)); } else { ObArray orig_audits; if (OB_FAIL(schema_guard.get_audit_schema_in_owner(tenant_id, AUDIT_TABLE, orig_table_schema.get_table_id(), orig_audits))) { LOG_WARN("fail to get_audit_schema_in_owner", K(tenant_id), K(orig_table_schema), K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < orig_audits.count(); ++i) { const ObSAuditSchema *audit_schema = orig_audits.at(i); if (OB_ISNULL(audit_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("audit_schema is NULL", K(ret)); } else { ObSAuditSchema new_audit_schema = *audit_schema; if (OB_FAIL(audit_schemas.push_back(new_audit_schema))) { LOG_WARN("failed to add audit_schema!", K(new_audit_schema), K(ret)); } } } } } // Save Oracle obj privs on table for later restore if (OB_SUCC(ret)) { if (OB_FAIL(schema_guard.get_obj_priv_with_obj_id(tenant_id, orig_table_schema.get_table_id(), static_cast(ObObjectType::TABLE), orig_obj_privs_pointer_ora, true /* reset flag */))) { LOG_WARN("get_obj_priv_with_obj_id failed", K(ret), K(tenant_id), K(orig_table_schema.get_table_id())); } else { for (int i = 0; OB_SUCC(ret) && i < orig_obj_privs_pointer_ora.count(); ++i) { if (OB_ISNULL(orig_obj_privs_pointer_ora.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("orig_obj_privs_pointer_ora contains NULL", K(ret), K(i)); } else { OZ (orig_obj_privs_ora.push_back(*(orig_obj_privs_pointer_ora.at(i)))); } } } } if (OB_SUCC(ret) && OB_FAIL(drop_aux_table_in_truncate( orig_table_schema, schema_guard, trans, ddl_operator, USER_INDEX, to_recyclebin))) { LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema)); } else if (OB_FAIL(drop_aux_table_in_truncate( orig_table_schema, schema_guard, trans, ddl_operator, AUX_VERTIAL_PARTITION_TABLE, to_recyclebin))) { LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema)); } else if (OB_FAIL(drop_aux_table_in_truncate( orig_table_schema, schema_guard, trans, ddl_operator, AUX_LOB_META, to_recyclebin))) { LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema)); } else if (OB_FAIL(drop_aux_table_in_truncate( orig_table_schema, schema_guard, trans, ddl_operator, AUX_LOB_PIECE, to_recyclebin))) { LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema)); } else { // Two conditions need to be met for placing in the recycle bin: // 1. to_recyclebin is marked // 2. it is not inner table if (OB_FAIL(drop_trigger_in_drop_table(trans, ddl_operator, schema_guard, orig_table_schema, to_recyclebin))) { LOG_WARN("drop trigger failed,", K(orig_table_schema), K(ret), K(to_recyclebin)); } if (OB_SUCC(ret)) { if (to_recyclebin && !is_inner_table(orig_table_schema.get_table_id())) { if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(orig_table_schema, schema_guard, trans, ddl_stmt_str, true))) { LOG_WARN("ddl_operator drop table to recyclebin failed,", K(orig_table_schema), K(ret)); } } else { if (OB_FAIL(ddl_operator.drop_table(orig_table_schema, trans, ddl_stmt_str, true/*is_truncate_table*/))) { LOG_WARN("ddl_operator drop_table failed", K(orig_table_schema), K(ret)); } } } bool is_truncate_table = false; bool recycle_cnt = 0; for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); i++) { if (0 == i) { is_truncate_table = true; //record create table with opertion type TRUNCATE_TABLE_CREATE } else { is_truncate_table = false; } ObTableSchema &tmp_schema = table_schemas.at(i); bool is_table_in_recyclebin = false; if (tmp_schema.is_in_recyclebin()) { is_table_in_recyclebin = true; if (!tmp_schema.is_index_table()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("normal table is in recyclebin", K(ret)); continue; } if (recycle_cnt >= index_recycle_objs.count()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index recycle objs indx is wrong", K(recycle_cnt), K(index_recycle_objs.count()), K(ret)); } else { const ObRecycleObject &recycle_obj = index_recycle_objs.at(recycle_cnt); if (OB_FAIL(reconstruct_table_schema_from_recyclebin(tmp_schema, recycle_obj, schema_guard))) { LOG_WARN("ddl operator reconstruct table schema from recyclebin failed", K(tmp_schema), K(ret)); } } if (OB_SUCC(ret)) { recycle_cnt++; ObString new_index_name; ObString new_index_table_name; if (OB_FAIL(ObTableSchema::get_index_name(allocator, orig_table_schema.get_table_id(), tmp_schema.get_table_name_str(), new_index_name))) { LOG_WARN("failed to build new index table name!", K(tmp_schema), K(ret)); continue; } else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator, tmp_schema.get_data_table_id(), new_index_name, new_index_table_name))) { LOG_WARN("failed to build new index table name!", K(tmp_schema.get_data_table_id())); continue; } else { tmp_schema.set_table_name(new_index_table_name); } const int VISIBLE = 0; const uint64_t DROPINDEX = 0; const uint64_t INVISIBLEBEFORE = 0; tmp_schema.set_drop_index(DROPINDEX); if (!tmp_schema.is_invisible_before()) { tmp_schema.set_index_visibility(VISIBLE); } tmp_schema.set_invisible_before(INVISIBLEBEFORE); } } if (OB_SUCC(ret) && (0 == i)) { // truncate table need update child table foreign key ID and foreign_key_id ObIArray &foreign_key_infos = tmp_schema.get_foreign_key_infos(); for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); foreign_key_info.foreign_key_id_ = OB_INVALID_ID; if (OB_FAIL(schema_service->fetch_new_constraint_id(tmp_schema.get_tenant_id(), foreign_key_info.foreign_key_id_))) { LOG_WARN("failed to fetch new foreign key id", K(ret), K(tmp_schema.get_tenant_id())); } else if (foreign_key_info.child_table_id_ == foreign_key_info.parent_table_id_) { // When it depends on itself, the parent table ID also needs to be updated foreign_key_info.parent_table_id_ = tmp_schema.get_table_id(); } if (OB_SUCC(ret)) { foreign_key_info.child_table_id_ = tmp_schema.get_table_id(); } } } if (OB_SUCC(ret)) { if (!is_table_in_recyclebin) { if (OB_FAIL(ddl_operator.create_sequence_in_create_table( tmp_schema, trans, schema_guard, NULL))) { LOG_WARN("failed to create sequence in create table", K(ret)); } else if (OB_FAIL(ddl_operator.create_table( tmp_schema, trans, 0 == i ? ddl_stmt_str : NULL, true, /*need_sync_schema_version*/ is_truncate_table))) { LOG_WARN("failed to create table schema, ", K(ret)); } else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, tmp_schema))) { LOG_WARN("failed to insert_temp_table_info!", K(ret)); } } else { if (OB_FAIL(ddl_operator.create_index_in_recyclebin( tmp_schema, schema_guard, trans, NULL))) { LOG_WARN("failed to create index schema", K(ret)); } } } // If table truncate is child table which has mock parent tables, will update the schema version of mock parent tables if (OB_SUCC(ret) && tmp_schema.get_foreign_key_real_count() > 0) { const ObIArray &foreign_key_infos = tmp_schema.get_foreign_key_infos(); ObArray mock_fk_parent_table_schema_array; for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_infos.count(); ++j) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(j); if (foreign_key_info.is_parent_table_mock_) { const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL; if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_id( tenant_id, foreign_key_info.parent_table_id_, mock_fk_parent_table_ptr))) { LOG_WARN("get_mock_fk_parent_table_schema_with_id failed", K(ret), K(tenant_id), K(foreign_key_info.parent_table_id_)); } else if (OB_ISNULL(mock_fk_parent_table_ptr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mock_fk_parent_table_ptr is null", K(ret)); } else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(*mock_fk_parent_table_ptr))) { LOG_WARN("push_back mock_fk_parent_table failed", K(ret)); } } } if (OB_SUCC(ret) && !mock_fk_parent_table_schema_array.empty()) { if (!mock_fk_parent_table_schema_array.empty()) { for (int64_t k = 0; k < mock_fk_parent_table_schema_array.count(); ++k) { mock_fk_parent_table_schema_array.at(k).set_operation_type(MOCK_FK_PARENT_TABLE_OP_UPDATE_SCHEMA_VERSION); } if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to deal_with_mock_fk_parent_table", K(ret), K(mock_fk_parent_table_schema_array)); } } } } if (OB_SUCC(ret) && (0 == i)) { // truncate table needs to rebuild the audit rules for the newly created table common::ObSqlString public_sql_string; for (int64_t i = 0; OB_SUCC(ret) && i < audit_schemas.count(); ++i) { uint64_t new_audit_id = common::OB_INVALID_ID; int64_t new_schema_version = OB_INVALID_VERSION; if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("failed to gen new schema_version", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_service->fetch_new_audit_id(tenant_id, new_audit_id))) { LOG_WARN("failed to fetch new_audit_id", K(ret)); } else { ObSAuditSchema &new_audit_schema = audit_schemas.at(i); new_audit_schema.set_schema_version(new_schema_version); new_audit_schema.set_audit_id(new_audit_id); new_audit_schema.set_owner_id(tmp_schema.get_table_id()); if (OB_FAIL(schema_service->get_audit_sql_service().handle_audit_metainfo( new_audit_schema, AUDIT_MT_ADD, false, new_schema_version, NULL, trans, public_sql_string))) { LOG_WARN("failed to add audit_schema", K(new_audit_schema), K(ret)); } else { LOG_INFO("succ to add audit_schema for truncate", K(new_audit_schema)); } } } } if (OB_SUCC(ret) && (0 == i)) { // truncate table needs to rebuild the audit rules for the newly created table if (OB_FAIL(restore_obj_priv_after_truncation( ddl_operator, trans, orig_obj_privs_ora, tmp_schema.get_table_id(), database_name, tmp_schema.get_table_name_str()))) { LOG_WARN("restore_obj_priv_after_truncation failed", K(ret), K(tmp_schema.get_table_id()), K(database_name), K(tmp_schema.get_table_name_str())); } } } int64_t frozen_scn = 0; share::ObSimpleFrozenStatus frozen_status; if (OB_FAIL(ret)) { } else if (OB_ISNULL(GCTX.root_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("root service is null", KR(ret)); } else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) { LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id)); } else { ObTableCreator table_creator(tenant_id, frozen_scn, *lst_operator_, trans); ObNewTableTabletAllocator new_table_tablet_allocator( tenant_id, schema_guard, sql_proxy_); common::ObArray ls_id_array; const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version(); if (OB_INVALID_VERSION == last_schema_version) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid last schema version", K(ret)); } else if (OB_FAIL(table_creator.init())) { LOG_WARN("fail to init table creator", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.init())) { LOG_WARN("fail to init new table tablet allocator", KR(ret)); } ObArray schemas; for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) { const share::schema::ObTableSchema &this_table = table_schemas.at(i); const int64_t table_id = this_table.get_table_id(); if (!this_table.has_tablet()) { } else if (!this_table.is_global_index_table()) { if (OB_FAIL(schemas.push_back(&this_table))) { LOG_WARN("failed to push_back", KR(ret), K(this_table)); } } else { if (OB_FAIL(new_table_tablet_allocator.prepare(this_table))) { LOG_WARN("fail to prepare ls for index schema tablets"); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array( ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_table_arg( this_table, ls_id_array))) { LOG_WARN("create table partitions failed", KR(ret), K(this_table), K(last_schema_version)); } } if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.insert_ori_schema_version( trans, tenant_id, table_id, last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(tenant_id), K(table_id), K(last_schema_version)); } } } if (OB_FAIL(ret)) { } // virtual table and view skip else if (schemas.count() <= 0) { } else if (OB_FAIL(new_table_tablet_allocator.prepare(*schemas.at(0)))) { LOG_WARN("new table tablet allocator prepared failed", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array( ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg( schemas, ls_id_array))) { LOG_WARN("create table partitions failed", KR(ret), K(orig_table_schema), K(last_schema_version)); } else if (OB_FAIL(table_creator.execute())) { LOG_WARN("execute create partition failed", KR(ret)); } } } // Create the trigger after creating the table,table_schemas.at(0) is new table schema if (OB_SUCC(ret)) { if (OB_FAIL(create_trigger_for_truncate_table(schema_guard, orig_table_schema.get_trigger_list(), table_schemas.at(0), ddl_operator, trans))) { LOG_WARN("failed to create trigger for truncate table", K(ret)); } } if (OB_FAIL(ret)) { } else if (arg.task_id_ > 0 && OB_FAIL(ObDDLRetryTask::update_task_status_succ(trans, tenant_id, arg.task_id_))) { LOG_WARN("update ddl task status failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } return ret; } int ObDDLService::restore_obj_priv_after_truncation( ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, ObIArray &orig_obj_privs_ora, uint64_t new_table_id, const ObString &database_name, const ObString &table_name) { int ret = OB_SUCCESS; for (int i = 0; OB_SUCC(ret) && i < orig_obj_privs_ora.count(); ++i) { ObObjPriv &obj_priv = orig_obj_privs_ora.at(i); ObRawObjPrivArray raw_priv_array_option; // privs with grant option ObRawObjPrivArray raw_priv_array_no_option; // privs without grant option if (OB_FAIL(ObPrivPacker::raw_option_obj_priv_from_pack( obj_priv.get_obj_privs(), raw_priv_array_option))) { LOG_WARN("raw_option_obj_priv_from_pack failed", K(ret), K(obj_priv.get_obj_privs())); } else if (OB_FAIL(ObPrivPacker::raw_no_option_obj_priv_from_pack( obj_priv.get_obj_privs(), raw_priv_array_no_option))) { LOG_WARN("raw_no_option_obj_priv_from_pack failed", K(ret), K(obj_priv.get_obj_privs())); } else { ObTablePrivSortKey table_key(obj_priv.get_tenant_id(), obj_priv.get_grantee_id(), database_name, table_name); obj_priv.set_obj_id(new_table_id); // truncate can change table_id ObObjPrivSortKey obj_priv_key = obj_priv.get_sort_key(); // Restore obj privs without grant option if (OB_FAIL(ddl_operator.grant_table( table_key, 0, // priv_set is null NULL, trans, raw_priv_array_no_option, // add new priv NO_OPTION, obj_priv_key))) { LOG_WARN("failed to add priv", K(ret), K(table_key)); } else if (OB_FAIL(ddl_operator.grant_table( // Restore obj priv with grant option table_key, 0, // priv_set is null NULL, trans, raw_priv_array_option, // add new priv GRANT_OPTION, obj_priv_key))) { LOG_WARN("failed to add priv", K(ret), K(table_key)); } else { LOG_INFO("succ to add priv for truncate table", K(ret), K(table_key)); } } } return ret; } int ObDDLService::drop_aux_table_in_truncate( const ObTableSchema &orig_table_schema, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, ObDDLOperator &ddl_operator, const ObTableType table_type, const bool to_recyclebin) { int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); const bool is_index = USER_INDEX == table_type; ObSEArray aux_vp_tid_array; ObSEArray simple_index_infos; uint64_t lob_meta_table_id; uint64_t lob_piece_table_id; int64_t N = 0; if (table_type == USER_INDEX) { if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type)); } else { N = simple_index_infos.count(); } } else if (table_type == AUX_VERTIAL_PARTITION_TABLE) { if (OB_FAIL(orig_table_schema.get_aux_vp_tid_array(aux_vp_tid_array))) { LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type)); } else { N = aux_vp_tid_array.count(); } } else if (table_type == AUX_LOB_META) { lob_meta_table_id = orig_table_schema.get_aux_lob_meta_tid(); N = orig_table_schema.has_lob_column() ? 1 : 0; } else if (table_type == AUX_LOB_PIECE) { lob_piece_table_id = orig_table_schema.get_aux_lob_piece_tid(); N = orig_table_schema.has_lob_column() ? 1 : 0; } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("table type is invalide", K(ret), K(table_type)); } for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { const ObTableSchema *aux_table_schema = NULL; uint64_t tid = 0; if (table_type == USER_INDEX) { tid = simple_index_infos.at(i).table_id_; } else if (table_type == AUX_VERTIAL_PARTITION_TABLE) { tid = aux_vp_tid_array.at(i); } else if (table_type == AUX_LOB_META) { tid = lob_meta_table_id; } else if (table_type == AUX_LOB_PIECE) { tid = lob_piece_table_id; } if (OB_FAIL(schema_guard.get_table_schema( tenant_id, tid, aux_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", tid, K(ret)); } else if (OB_ISNULL(aux_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (to_recyclebin && !is_inner_table(orig_table_schema.get_table_id())) { // support truncate table when recyclebin on if (aux_table_schema->is_in_recyclebin()) { LOG_WARN("the aux table is already in recyclebin"); } else if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(*aux_table_schema, schema_guard, trans, NULL))) { LOG_WARN("drop aux table to recycle failed", K(ret)); } } else if (aux_table_schema->is_in_recyclebin()) { // if aux table is in recyclebin (not support now), can purge if (OB_FAIL(ddl_operator.purge_table_in_recyclebin( *aux_table_schema, trans, NULL))) { LOG_WARN("purge aux table failed", K(ret), K(*aux_table_schema)); } } else if (OB_FAIL(ddl_operator.drop_table(*aux_table_schema, trans))) { LOG_WARN("ddl_operator drop_table failed,", K(*aux_table_schema), K(ret)); } } return ret; } int ObDDLService::truncate_oracle_temp_table(const ObString &db_name, const ObString &tab_name, const uint64_t tenant_id, const uint64_t session_id, const int64_t sess_create_time) { int ret = OB_SUCCESS; ObSqlString sql; int64_t affect_rows = 0; common::ObOracleSqlProxy oracle_sql_proxy; if (OB_FAIL(oracle_sql_proxy.init(sql_proxy_->get_pool()))) { LOG_WARN("init oracle sql proxy failed", K(ret)); } else if (0 == sess_create_time) { ret = sql.assign_fmt("DELETE FROM \"%.*s\".\"%.*s\" WHERE " "%s = %lu", db_name.length(), db_name.ptr(), tab_name.length(), tab_name.ptr(), OB_HIDDEN_SESSION_ID_COLUMN_NAME, session_id); } else { ret = sql.assign_fmt("DELETE FROM \"%.*s\".\"%.*s\" WHERE " "%s = %lu AND %s <> %lu", db_name.length(), db_name.ptr(), tab_name.length(), tab_name.ptr(), OB_HIDDEN_SESSION_ID_COLUMN_NAME, session_id, OB_HIDDEN_SESS_CREATE_TIME_COLUMN_NAME, sess_create_time); } if (OB_FAIL(ret)) { LOG_WARN("assign sql failed", K(ret)); } else if (OB_FAIL(oracle_sql_proxy.write(tenant_id, sql.ptr(), affect_rows))) { LOG_WARN("execute sql failed", K(ret), K(sql)); } else { LOG_DEBUG("succeed to execute truncate table as delete stmt", K(sql), K(affect_rows)); } return ret; } int ObDDLService::maintain_obj_dependency_info(const obrpc::ObDependencyObjDDLArg &arg) { int ret = OB_SUCCESS; bool is_standby = false; const uint64_t tenant_id = arg.tenant_id_; ObSchemaService *schema_service = schema_service_->get_schema_service(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_TENANT_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant_id", K(ret), K(tenant_id)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("failed to get is standby cluster", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; int64_t refreshed_schema_version = 0; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (!arg.update_dep_objs_.empty() && OB_FAIL(process_schema_object_dependency(tenant_id, is_standby, arg.update_dep_objs_, schema_guard, trans, ddl_operator, ObReferenceObjTable::UPDATE_OP))) { LOG_WARN("failed to process update object dependency", K(ret)); } else if (!arg.insert_dep_objs_.empty() && OB_FAIL(process_schema_object_dependency(tenant_id, is_standby, arg.insert_dep_objs_, schema_guard, trans, ddl_operator, ObReferenceObjTable::INSERT_OP))) { LOG_WARN("failed to process insert object dependency", K(ret)); } else if (!arg.delete_dep_objs_.empty() && OB_FAIL(process_schema_object_dependency(tenant_id, is_standby, arg.delete_dep_objs_, schema_guard, trans, ddl_operator, ObReferenceObjTable::DELETE_OP))) { LOG_WARN("failed to process delete object dependency", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } return ret; } int ObDDLService::process_schema_object_dependency( const uint64_t tenant_id, const bool is_standby, const ObReferenceObjTable::DependencyObjKeyItemPairs &dep_objs, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, ObDDLOperator &ddl_operator, ObReferenceObjTable::ObSchemaRefObjOp op) { int ret = OB_SUCCESS; int64_t new_schema_version = OB_INVALID_VERSION; if (dep_objs.empty()) { // do nothing } else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id)); } else { switch (op) { case ObReferenceObjTable::INSERT_OP: case ObReferenceObjTable::UPDATE_OP: OZ (ObReferenceObjTable::batch_execute_insert_or_update_obj_dependency(tenant_id, is_standby, new_schema_version, dep_objs, trans, schema_guard, ddl_operator)); break; case ObReferenceObjTable::DELETE_OP: OZ (ObReferenceObjTable::batch_execute_delete_obj_dependency(tenant_id, is_standby, dep_objs, trans)); break; default: break; } } return ret; } int ObDDLService::gen_hidden_obj_name(const uint64_t obj_id, const uint64_t table_id, const uint64_t origin_fk_id, ObIAllocator &allocator, ObString &hidden_obj_name, const ObSchemaType schema_type) { int ret = OB_SUCCESS; int64_t pos = 0; char *buf = NULL; int64_t buf_len = OB_MAX_TABLE_NAME_LENGTH; if (OB_ISNULL(buf = static_cast(allocator.alloc(buf_len)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc memory", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "__hidden_"))) { LOG_WARN("append name to buf error", K(ret)); } else if (FK_SCHEMA == schema_type && OB_FAIL(databuff_printf(buf, buf_len, pos, "%ld_", origin_fk_id))) { LOG_WARN("append name to buf error", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%ld", obj_id))) { LOG_WARN("append name to buf error", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "_%ld", table_id))) { LOG_WARN("append name to buf error", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", schema_type_str(schema_type)))) { LOG_WARN("append name to buf error", K(ret)); } else { hidden_obj_name.assign_ptr(buf, pos); } return ret; } int ObDDLService::is_foreign_key_name_prefix_match(const ObForeignKeyInfo &origin_fk_info, const ObForeignKeyInfo &hidden_fk_info, common::ObIAllocator &allocator, bool &is_prefix_match) { int ret = OB_SUCCESS; is_prefix_match = false; int64_t pos = 0; char *buf = nullptr; int64_t buf_len = OB_MAX_TABLE_NAME_LENGTH; if (OB_ISNULL(buf = static_cast(allocator.alloc(buf_len)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc memory", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "__hidden_"))) { LOG_WARN("append name to buf error", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%ld_", origin_fk_info.foreign_key_id_))) { LOG_WARN("append name to buf error", K(ret)); } else { ObString prefix_hidden_fk_name; prefix_hidden_fk_name.assign_ptr(buf, pos); is_prefix_match = hidden_fk_info.foreign_key_name_.prefix_match(prefix_hidden_fk_name); } return ret; } int ObDDLService::prepare_hidden_table_schema(const ObTableSchema &orig_table_schema, const int64_t frozen_version, ObIAllocator &allocator, ObTableSchema &hidden_table_schema) { int ret = OB_SUCCESS; bool is_oracle_mode = false; uint64_t new_table_id = OB_INVALID_ID; ObString new_table_name; ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } else if (OB_FAIL(schema_service->fetch_new_table_id(orig_table_schema.get_tenant_id(), new_table_id))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else { // in the first stage, create a hidden table without creating constraints, foreign keys // and indexes. if it needs to be created, it will be created in the second stage hidden_table_schema.clear_constraint(); hidden_table_schema.clear_foreign_key_infos(); hidden_table_schema.set_data_table_id(0); if (OB_FAIL(gen_hidden_obj_name(new_table_id, orig_table_schema.get_table_id(), common::OB_INVALID_ID, allocator, new_table_name, TABLE_SCHEMA))) { LOG_WARN("failed to gen hidden table name", K(orig_table_schema), K(ret)); } else if (OB_FAIL(try_format_partition_schema(hidden_table_schema))) { LOG_WARN("convert template schema to non template schema", K(ret)); } else if (OB_FAIL(generate_object_id_for_partition_schema(hidden_table_schema))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(hidden_table_schema)); } else if (OB_FAIL(generate_tablet_id(hidden_table_schema))) { LOG_WARN("fail to generate tablet id for hidden table", K(ret), K(hidden_table_schema)); } else { hidden_table_schema.set_database_id(orig_table_schema.get_database_id()); hidden_table_schema.set_table_id(new_table_id); hidden_table_schema.set_table_name(new_table_name); hidden_table_schema.set_association_table_id(orig_table_schema.get_table_id()); // set the hidden attributes of the table hidden_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); // in oracle mode, need to add primary key constraints if (is_oracle_mode && !hidden_table_schema.is_heap_table()) { uint64_t new_cst_id = OB_INVALID_ID; ObString pk_name; ObConstraint cst; cst.set_constraint_type(CONSTRAINT_TYPE_PRIMARY_KEY); if (OB_FAIL(schema_service->fetch_new_constraint_id(orig_table_schema.get_tenant_id(), new_cst_id))) { LOG_WARN("failed to fetch new constraint id", K(ret)); } else if (FALSE_IT(cst.set_constraint_id(new_cst_id))) { } else if (!orig_table_schema.is_heap_table()) { if (OB_FAIL(orig_table_schema.get_pk_constraint_name(pk_name))) { LOG_WARN("failed to get pk constraint name", K(ret)); } } else { if (OB_FAIL(ObTableSchema::create_cons_name_automatically( pk_name, orig_table_schema.get_table_name_str(), allocator, CONSTRAINT_TYPE_PRIMARY_KEY, is_oracle_mode))) { LOG_WARN("create cons name automatically failed", K(ret)); } } OZ(cst.set_constraint_name(pk_name)); OZ(hidden_table_schema.add_constraint(cst)); } } } return ret; } int ObDDLService::rebuild_hidden_table_priv(const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; ObArray orig_obj_privs_pointer_ora; ObArray orig_obj_privs_ora; if (OB_FAIL(schema_guard.get_obj_priv_with_obj_id( orig_table_schema.get_tenant_id(), orig_table_schema.get_table_id(), static_cast(ObObjectType::TABLE), orig_obj_privs_pointer_ora, true /* reset flag */))) { LOG_WARN("get_obj_priv_with_obj_id failed", K(ret), K(orig_table_schema.get_table_id())); } else { for (int i = 0; OB_SUCC(ret) && i < orig_obj_privs_pointer_ora.count(); ++i) { if (OB_ISNULL(orig_obj_privs_pointer_ora.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("orig_obj_privs_pointer_ora contains NULL", K(ret), K(i)); } else { OZ (orig_obj_privs_ora.push_back(*(orig_obj_privs_pointer_ora.at(i)))); } } } if (OB_SUCC(ret)) { // need to rebuild permissions for the newly created table if (OB_FAIL(restore_obj_priv_after_truncation( ddl_operator, trans, orig_obj_privs_ora, hidden_table_schema.get_table_id(), hidden_table_schema.get_link_database_name(), hidden_table_schema.get_table_name_str()))) { LOG_WARN("restore_obj_priv_after_truncation failed", K(ret), K(hidden_table_schema.get_table_id()), K(hidden_table_schema.get_table_name_str())); } } return ret; } int ObDDLService::create_user_hidden_table(const ObTableSchema &orig_table_schema, ObTableSchema &hidden_table_schema, const obrpc::ObSequenceDDLArg *sequence_ddl_arg, const bool bind_tablets, ObSchemaGetterGuard &schema_guard, const int64_t frozen_version, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, ObIAllocator &allocator) { int ret = OB_SUCCESS; const uint64_t tenant_id = hidden_table_schema.get_tenant_id(); ObArray aux_table_schemas; ObSEArray schemas; // 1(hidden_table) + 2(aux_lob_table) bool need_sync_schema_version = false; bool is_add_identity_column = false; int64_t frozen_scn = 0; hidden_table_schema.set_in_offline_ddl_white_list(orig_table_schema.check_can_do_ddl()); // allow offline ddl execute if there's no offline ddl doing if (OB_ISNULL(GCTX.root_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("root service is null", KR(ret)); } else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) { LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id)); } else if (OB_FAIL(check_is_add_identity_column(orig_table_schema, hidden_table_schema, is_add_identity_column))) { LOG_WARN("failed to check is add identity column", K(ret)); } else if (OB_FAIL(prepare_hidden_table_schema(orig_table_schema, frozen_version, allocator, hidden_table_schema))) { LOG_WARN("failed to prepare hidden table schema", K(ret)); } else if (OB_FAIL(ddl_operator.create_sequence_in_create_table(hidden_table_schema, trans, schema_guard, is_add_identity_column ? sequence_ddl_arg : nullptr))) { LOG_WARN("failed to create sequence in create table", K(ret)); } else if (OB_FAIL(build_aux_lob_table_schema_if_need(hidden_table_schema, aux_table_schemas))) { LOG_WARN("failed to build_aux_lob_table_schema_if_need", K(ret), K(hidden_table_schema)); } else if (OB_FAIL(rebuild_hidden_table_priv(orig_table_schema, hidden_table_schema, schema_guard, ddl_operator, trans))) { LOG_WARN("failed to rebuild hidden table priv", K(ret)); } else { if (OB_FAIL(schemas.push_back(&hidden_table_schema))) { LOG_WARN("fail to push back hidden table schema" , K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < aux_table_schemas.count(); i++) { ObTableSchema &table_schema = aux_table_schemas.at(i); // allow offline ddl execute if there's no offline ddl doing table_schema.set_in_offline_ddl_white_list(orig_table_schema.check_can_do_ddl()); if (OB_FAIL(schemas.push_back(&table_schema))) { LOG_WARN("fail to push back aux table schema" , K(ret)); } } } if (OB_SUCC(ret)) { bool need_sync_schema_version = false; ObTableCreator table_creator( tenant_id, frozen_scn, *lst_operator_, trans); ObNewTableTabletAllocator new_table_tablet_allocator( tenant_id, schema_guard, sql_proxy_); common::ObArray ls_id_array; if (OB_FAIL(table_creator.init())) { LOG_WARN("fail to init table creator", KR(ret)); } else if (OB_FAIL(new_table_tablet_allocator.init())) { LOG_WARN("fail to init new table tablet allocator", KR(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < schemas.count(); i++) { share::schema::ObTableSchema *table_schema = const_cast(schemas.at(i)); if (OB_FAIL(ddl_operator.create_table(*table_schema, trans, NULL, need_sync_schema_version))) { LOG_WARN("failed to create table schema", K(ret)); } else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, *table_schema))) { LOG_WARN("failed to insert temp table info", K(ret), KPC(table_schema)); } } const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version(); if (OB_SUCC(ret) && OB_INVALID_VERSION == last_schema_version) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid schema version" , K(last_schema_version)); } for (int64_t i = 0; OB_SUCC(ret) && i < schemas.count(); i++) { share::schema::ObTableSchema *table_schema = const_cast(schemas.at(i)); if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans, tenant_id, table_schema->get_table_id(), last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", K(ret), KPC(table_schema), K(last_schema_version)); } } // when need bind tablets, schemas array only store aux tables, need remove data schema if (OB_SUCC(ret) && bind_tablets) { if (OB_FAIL(schemas.remove(0))) { LOG_WARN("failed to remove data schema.", K(ret)); } } if (OB_SUCC(ret) && hidden_table_schema.has_tablet()) { if (bind_tablets && OB_FAIL(new_table_tablet_allocator.prepare_like(orig_table_schema))) { LOG_WARN("fail to prepare like", KR(ret), K(orig_table_schema)); } else if (!bind_tablets && OB_FAIL(new_table_tablet_allocator.prepare(hidden_table_schema))) { LOG_WARN("fail to prepare", KR(ret), K(hidden_table_schema)); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (!bind_tablets && OB_FAIL(table_creator.add_create_tablets_of_tables_arg( schemas, ls_id_array))) { LOG_WARN("create table tablets failed", K(ret), K(hidden_table_schema)); } else if (bind_tablets && OB_FAIL(table_creator.add_create_bind_tablets_of_hidden_table_arg( orig_table_schema, hidden_table_schema, ls_id_array))) { LOG_WARN("failed to add arg", K(ret), K(hidden_table_schema)); } else if (bind_tablets && schemas.count() > 0 && OB_FAIL(table_creator.add_create_tablets_of_local_aux_tables_arg( schemas, &hidden_table_schema, ls_id_array))) { LOG_WARN("failed to add arg", K(ret), K(aux_table_schemas), K(hidden_table_schema)); } else if (OB_FAIL(table_creator.execute())) { LOG_WARN("fail to execute create tablet", KR(ret)); } } // finishing is always invoked for new table tablet allocator int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) { LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret)); } } if (OB_SUCC(ret)) { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; ObTableSchema table_schema; if (OB_FAIL(table_schema.assign(orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { // here need to change the state of the orig table to TABLE_STATE_OFFLINE_DDL, and at // the same time update association table id table_schema.set_in_offline_ddl_white_list(orig_table_schema.check_can_do_ddl()); table_schema.set_association_table_id(hidden_table_schema.get_table_id()); table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL); if (OB_FAIL(ddl_operator.update_table_attribute(table_schema, trans, operation_type))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } } } return ret; } int ObDDLService::build_aux_lob_table_schema_if_need(ObTableSchema &data_table_schema, ObIArray &table_schemas) { int ret = OB_SUCCESS; ObLobMetaBuilder lob_meta_builder(*this); ObLobPieceBuilder lob_piece_builder(*this); bool lob_col_found = false; for (int64_t i = 0; OB_SUCC(ret) && i < data_table_schema.get_column_count() && !lob_col_found; ++i) { if (is_lob_v2(data_table_schema.get_column_schema_by_idx(i)->get_data_type())) { HEAP_VARS_2((ObTableSchema, lob_meta_schema), (ObTableSchema, lob_piece_schema)) { if (OB_FAIL(lob_meta_builder.generate_aux_lob_meta_schema( schema_service_->get_schema_service(), data_table_schema, lob_meta_schema, true))) { LOG_WARN("generate_schema for lob meta table failed", K(data_table_schema), K(ret)); } else if (OB_FAIL(table_schemas.push_back(lob_meta_schema))) { LOG_WARN("push_back lob meta table failed", K(ret)); } else if (OB_FAIL(lob_piece_builder.generate_aux_lob_piece_schema( schema_service_->get_schema_service(), data_table_schema, lob_piece_schema, true))) { LOG_WARN("generate_schema for lob data table failed", K(data_table_schema), K(ret)); } else if (OB_FAIL(table_schemas.push_back(lob_piece_schema))) { LOG_WARN("push_back lob data table failed", K(ret)); } else { data_table_schema.set_aux_lob_meta_tid(lob_meta_schema.get_table_id()); data_table_schema.set_aux_lob_piece_tid(lob_piece_schema.get_table_id()); } } lob_col_found = true; } } return ret; } int ObDDLService::gen_new_index_table_name( const ObString &orig_index_table_name, const uint64_t orig_table_id, const uint64_t new_table_id, ObIAllocator &allocator, ObString &new_index_table_name) { int ret = OB_SUCCESS; ObString index_name; if (OB_FAIL(ObTableSchema::get_index_name( allocator, orig_table_id, orig_index_table_name, index_name))) { LOG_WARN("error get index table name failed", K(orig_table_id), K(orig_index_table_name), K(ret)); } else if (OB_FAIL(ObTableSchema::build_index_table_name( allocator, new_table_id, index_name, new_index_table_name))) { LOG_WARN("failed to build new index table name!", K(new_table_id), K(new_index_table_name), K(ret)); } return ret; } // col_name_map must live beyond index_schema int ObDDLService::gen_hidden_index_schema_columns(const ObTableSchema &orig_index_schema, const common::ObIArray &drop_cols_id_arr, const ObColumnNameMap &col_name_map, ObTableSchema &new_table_schema, ObTableSchema &index_schema) { int ret = OB_SUCCESS; SMART_VAR(ObCreateIndexArg, create_index_arg) { ObTableSchema::const_column_iterator tmp_begin = index_schema.column_begin(); ObTableSchema::const_column_iterator tmp_end = index_schema.column_end(); ObArray> index_columns; for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) { ObColumnSchemaV2 *col = (*tmp_begin); ObString col_name; bool is_dropped = false; if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is NULL", K(ret)); } else if (col->is_shadow_column()) { // shadown column doesn't exist in data table col_name = col->get_column_name_str(); } else if (OB_FAIL(col_name_map.get(col->get_column_name_str(), col_name))) { if (OB_ENTRY_NOT_EXIST == ret) { ret = OB_SUCCESS; // test whether the column is dropped; FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && !is_dropped) { if (*dropped_col == col->get_column_id()) { is_dropped = true; } } if (!is_dropped) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error, column is null in new table", K(ret), K(col->get_column_name_str())); } } else { LOG_WARN("failed to get new name", K(ret), K(col->get_column_name())); } } if (OB_FAIL(ret)) { } else if (is_dropped) { // drop in index } else if (col->get_index_position() > 0) { obrpc::ObColumnSortItem sort_item; sort_item.column_name_ = col_name; sort_item.order_type_ = col->get_order_in_rowkey(); if (col->is_prefix_column()) { sort_item.prefix_len_ = col->get_data_length(); } if (OB_FAIL(index_columns.push_back(std::make_pair(col->get_index_position(), sort_item)))) { LOG_WARN("fail to add index columns", K(ret)); } } else if (col->is_user_specified_storing_column()) { if (0 < col->get_index_position() || col->is_hidden() || col->is_hidden_pk_column_id(col->get_column_id())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, these columns can not be user specified storing column", K(ret), KPC(col)); } else if (OB_FAIL(create_index_arg.store_columns_.push_back(col_name))) { LOG_WARN("fail to push back storing column", K(ret), KPC(col)); } else { /* do nothing. */ } } } if (OB_SUCC(ret)) { std::sort(index_columns.begin(), index_columns.end(), [](const std::pair &lhs, const std::pair &rhs) -> bool { return lhs.first < rhs.first; }); for (int64_t i = 0; OB_SUCC(ret) && i < index_columns.count(); i++) { if (OB_FAIL(create_index_arg.index_columns_.push_back(index_columns[i].second))) { LOG_WARN("failed to push back index column", K(ret)); } } } if (OB_SUCC(ret)) { index_schema.reset_column_info(); create_index_arg.index_type_ = index_schema.get_index_type(); if (INDEX_TYPE_NORMAL_LOCAL == create_index_arg.index_type_ || INDEX_TYPE_UNIQUE_LOCAL == create_index_arg.index_type_ || INDEX_TYPE_DOMAIN_CTXCAT == create_index_arg.index_type_) { if (OB_FAIL(sql::ObResolverUtils::check_unique_index_cover_partition_column( new_table_schema, create_index_arg))) { LOG_WARN("fail to check unique key cover partition column", K(ret)); if (INDEX_TYPE_UNIQUE_LOCAL == create_index_arg.index_type_ && OB_EER_UNIQUE_KEY_NEED_ALL_FIELDS_IN_PF == ret) { int tmp_ret = OB_SUCCESS; bool allow = false; if (OB_SUCCESS != (tmp_ret = ObDDLResolver::check_uniq_allow( new_table_schema, create_index_arg, allow))) { LOG_WARN("fail to check uniq allow", K(ret)); } else if (allow) { LOG_INFO("uniq index allowd, deduced by constraint", K(ret)); ret = OB_SUCCESS; } } } } } OZ(ObIndexBuilderUtil::set_index_table_columns(create_index_arg, new_table_schema, index_schema)); tmp_begin = orig_index_schema.column_begin(); tmp_end = orig_index_schema.column_end(); for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) { ObColumnSchemaV2 *orig_col = (*tmp_begin); ObString col_name; if (OB_ISNULL(orig_col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is NULL", K(ret)); } else if (orig_col->is_tbl_part_key_column()) { if (OB_UNLIKELY(orig_col->is_shadow_column())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("shadow column as part key", K(ret), KPC(orig_col)); } else if (OB_FAIL(col_name_map.get(orig_col->get_column_name_str(), col_name))) { LOG_WARN("invalid column name", K(ret), KPC(orig_col)); } else { ObColumnSchemaV2 *col = index_schema.get_column_schema(col_name); if (OB_ISNULL(col)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("col is NULL", K(ret)); } else { col->set_tbl_part_key_pos(orig_col->get_tbl_part_key_pos()); col->set_part_key_pos(orig_col->get_part_key_pos()); col->set_subpart_key_pos(orig_col->get_subpart_key_pos()); } } } } } return ret; } int ObDDLService::add_new_index_schema(obrpc::ObAlterTableArg &alter_table_arg, const share::schema::ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, const int64_t frozen_version, ObSArray &new_table_schemas, ObSArray &index_ids) { int ret = OB_SUCCESS; ObSchemaService *schema_service = schema_service_->get_schema_service(); ObTableSchema new_table_schema; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (!alter_table_arg.is_alter_columns_) { } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", K(ret), KP(schema_service)); } else if (OB_FAIL(new_table_schema.assign(hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { ObIndexBuilder index_builder(*this); HEAP_VAR(AddIndexNameHashSet, add_index_name_set) { HEAP_VAR(DropIndexNameHashSet, drop_index_name_set) { const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; int64_t index_count = new_table_schema.get_index_tid_count(); for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) { ObIndexArg *index_arg = const_cast(index_arg_list.at(i)); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("index arg should not be null", K(ret)); } else { if (index_arg->index_action_type_ == ObIndexArg::ADD_INDEX) { if (OB_MAX_INDEX_PER_TABLE <= index_count) { ret = OB_ERR_TOO_MANY_KEYS; LOG_USER_ERROR(OB_ERR_TOO_MANY_KEYS, OB_MAX_INDEX_PER_TABLE); LOG_WARN("too many index for table!", K(index_count), K(OB_MAX_INDEX_PER_TABLE)); } ObCreateIndexArg *create_index_arg = static_cast(index_arg); if (!new_table_schema.is_partitioned_table() && !create_index_arg->index_schema_.is_partitioned_table()) { if (INDEX_TYPE_NORMAL_GLOBAL == create_index_arg->index_type_) { create_index_arg->index_type_ = INDEX_TYPE_NORMAL_GLOBAL_LOCAL_STORAGE; } else if (INDEX_TYPE_UNIQUE_GLOBAL == create_index_arg->index_type_) { create_index_arg->index_type_ = INDEX_TYPE_UNIQUE_GLOBAL_LOCAL_STORAGE; } } if (OB_SUCC(ret)) { if (create_index_arg->index_type_ == INDEX_TYPE_PRIMARY) { if (new_table_schema.get_rowkey_column_num() > 0) { if (new_table_schema.is_heap_table()) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support to add primary key!", K(ret)); } else { ret = OB_ERR_MULTIPLE_PRI_KEY; LOG_WARN("multiple primary key defined", K(ret)); } } continue; } } if (create_index_arg->index_name_.empty()) { // check whether the index table exists based on the origin table rather than the hidden one. if (OB_FAIL(generate_index_name(*create_index_arg, orig_table_schema, add_index_name_set, drop_index_name_set, schema_guard, alter_table_arg.allocator_))) { LOG_WARN("generate index name failed", K(ret)); } } else { ObString index_name = create_index_arg->index_name_; ObString index_table_name; bool is_exist = false; // check index exist should use orig_table_schema, otherwise cannot see old index if (OB_FAIL(check_index_table_exist(orig_table_schema.get_tenant_id(), orig_table_schema.get_database_id(), orig_table_schema.get_table_id(), index_name, schema_guard, is_exist))) { LOG_WARN("failed to check index table", K(ret)); } else { ObIndexNameHashWrapper index_key(index_name); if (!is_exist) { if (OB_HASH_EXIST == add_index_name_set.exist_refactored(index_key)) { //alter table t1 add index c1, add index c1 is_exist = true; } } else { if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) { //alter table t1 drop index c1, add index c1 is_exist = false; } } if (is_exist) { ret = OB_ERR_KEY_NAME_DUPLICATE; LOG_USER_ERROR(OB_ERR_KEY_NAME_DUPLICATE, index_name.length(), index_name.ptr()); LOG_WARN("duplicate index name", K(index_name), K(ret)); } } } if (OB_SUCC(ret)) { HEAP_VAR(ObTableSchema, index_schema) { ObArray gen_columns; bool global_storage = INDEX_TYPE_NORMAL_GLOBAL == create_index_arg->index_type_ || INDEX_TYPE_UNIQUE_GLOBAL == create_index_arg->index_type_; if (global_storage) { if (OB_FAIL(index_schema.assign(create_index_arg->index_schema_))) { LOG_WARN("fail to assign schema", K(ret)); } else if (FALSE_IT(index_schema.set_tenant_id(new_table_schema.get_tenant_id()))) { } } bool global_index_without_column_info = create_index_arg->index_schema_.is_partitioned_table() ? false : true; if (OB_FAIL(ret)) { } else if (OB_FAIL(ObIndexBuilderUtil::adjust_expr_index_args( *create_index_arg, new_table_schema, alter_table_arg.allocator_, gen_columns))) { LOG_WARN("adjust fulltext args failed", K(ret)); } else if (OB_FAIL(index_builder.generate_schema(*create_index_arg, frozen_version, new_table_schema, global_index_without_column_info, index_schema))) { LOG_WARN("failed to generate index schema!", K(ret)); } else { uint64_t new_idx_tid = OB_INVALID_ID; if (OB_FAIL(schema_service->fetch_new_table_id( new_table_schema.get_tenant_id(), new_idx_tid))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else if (OB_FAIL(generate_object_id_for_partition_schema(index_schema))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(index_schema)); } else if (OB_FAIL(generate_tablet_id(index_schema))) { LOG_WARN("fail to generate tablet id for hidden table", K(ret), K(index_schema)); } else { bool is_exist = false; index_schema.set_table_id(new_idx_tid); index_schema.set_data_table_id(new_table_schema.get_table_id()); index_schema.set_index_status(INDEX_STATUS_UNAVAILABLE); // set the hidden attributes of the table index_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); if (OB_FAIL(ret)) { } else if (OB_FAIL(schema_guard.check_table_exist(index_schema.get_tenant_id(), index_schema.get_database_id(), index_schema.get_table_name_str(), true/*is_index*/, ObSchemaGetterGuard::USER_HIDDEN_TABLE_TYPE/*check_type*/, is_exist))) { LOG_WARN("failed to check table exist", K(ret)); } else if (is_exist) { LOG_INFO("index already rebuilt, skip", K(index_schema.get_table_id()), K(index_schema.get_table_name_str())); } else if (OB_FAIL(new_table_schemas.push_back(index_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } else if (OB_FAIL(index_ids.push_back(index_schema.get_table_id()))) { LOG_WARN("failed to add new index id!", K(ret)); } } if (OB_SUCC(ret)) { ObIndexNameHashWrapper index_key(create_index_arg->index_name_); if (OB_FAIL(create_index_arg->index_schema_.assign(index_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(add_index_name_set.set_refactored(index_key))) { LOG_WARN("set index name to hash set failed", K(create_index_arg->index_name_), K(ret)); } else { ++index_count; } } } } } } else if (index_arg->index_action_type_ == ObIndexArg::DROP_FOREIGN_KEY || index_arg->index_action_type_ == ObIndexArg::RENAME_INDEX || index_arg->index_action_type_ == ObIndexArg::ADD_PRIMARY_KEY) { // ignore drop foreign key and rename index, it should be handled outside } else { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement"); } } } //end for } // end add_index_name_set } // end drop_index_name_set } return ret; } // check whether the index table need to be rebuilt. // TODO @yiren 2022-05-05 indexes should not be rebuilt if there is no relationship to drop columns. int ObDDLService::check_index_table_need_rebuild(const share::schema::ObTableSchema &index_table_schema, const common::ObIArray &drop_cols_id_arr, const bool is_oracle_mode, bool &need_rebuild) { int ret = OB_SUCCESS; need_rebuild = true; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (is_oracle_mode) { // For Oracle mode, index is invalid if its' any rowkey/storing column specified by user is dropped; ObSArray check_columns_id; // push back user specified index column. if (OB_FAIL(index_table_schema.get_index_info().get_column_ids(check_columns_id))) { LOG_WARN("fail to get column ids that need be checked", K(ret)); } else { // push back user specified storing column for index.(unique and non-unique) for (ObTableSchema::const_column_iterator iter = index_table_schema.column_begin(); OB_SUCC(ret) && iter != index_table_schema.column_end(); iter++) { const ObColumnSchemaV2 *col_schema = *iter; if (OB_ISNULL(col_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, column is null", K(ret)); } else if (col_schema->is_user_specified_storing_column() && OB_FAIL(check_columns_id.push_back(col_schema->get_column_id()))) { LOG_WARN("fail to push back store column id", K(ret), KPC(col_schema)); } else {/* do nothing. */} } FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && need_rebuild) { for (int64_t i = 0; OB_SUCC(ret) && need_rebuild && i < check_columns_id.count(); i++) { if (*dropped_col == check_columns_id.at(i)) { need_rebuild = false; } } } } } else { // For mysql mode, index is invalid if its' rowkey columns specified by user are all dropped. int64_t drop_check_column_cnt = 0; ObSArray check_columns_id; if (OB_FAIL(index_table_schema.get_index_info().get_column_ids(check_columns_id))) { LOG_WARN("fail to get column ids that need be checked", K(ret)); } else { FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && need_rebuild) { bool is_drop_check_column = false; for (int64_t i = 0; OB_SUCC(ret) && !is_drop_check_column && i < check_columns_id.count(); i++) { if (*dropped_col == check_columns_id.at(i)) { is_drop_check_column = true; drop_check_column_cnt++; } } } if (OB_SUCC(ret) && drop_check_column_cnt == check_columns_id.count()) { need_rebuild = false; } } } return ret; } int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, const common::ObIArray &drop_cols_id_arr, const ObColumnNameMap &col_name_map, const common::ObTimeZoneInfo &tz_info, const int64_t frozen_version, ObIAllocator &allocator, ObSArray &new_table_schemas, ObSArray &index_ids) { int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); bool is_oracle_mode = false; ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", K(ret), KP(schema_service)); } else { ObTableSchema new_table_schema; ObSEArray simple_index_infos; if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } else if (OB_FAIL(new_table_schema.assign(hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to check if oralce compat mode", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; bool need_rebuild = true; if (OB_FAIL(schema_guard.get_table_schema( tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", simple_index_infos.at(i).table_id_, K(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (OB_FAIL(check_index_table_need_rebuild(*index_table_schema, drop_cols_id_arr, is_oracle_mode, need_rebuild))) { LOG_WARN("fail to check index whether to need rebuild", K(ret), KPC(index_table_schema)); } else if (need_rebuild) { ObString new_index_table_name; HEAP_VAR(ObTableSchema, new_index_schema) { // No need to convert hidden table column id since indexes are rebuilt based on column names. if (OB_FAIL(new_index_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(gen_new_index_table_name( index_table_schema->get_table_name_str(), orig_table_schema.get_table_id(), hidden_table_schema.get_table_id(), allocator, new_index_table_name))) { LOG_WARN("failed to build new index table name!", K(hidden_table_schema.get_table_id()), K(new_index_table_name), K(ret)); } else if (new_index_schema.is_global_index_table() && new_index_schema.is_partitioned_table()) { ObArray> changed_names; if (OB_FAIL(col_name_map.get_changed_names(changed_names))) { LOG_WARN("failed to get changed names", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < changed_names.count(); i++) { if (OB_FAIL(modify_part_func_expr( changed_names[i].first, changed_names[i].second, new_index_schema, tz_info, allocator))) { LOG_WARN("failed to modify part func expr", K(ret)); } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(gen_hidden_index_schema_columns( *index_table_schema, drop_cols_id_arr, col_name_map, new_table_schema, new_index_schema))) { LOG_WARN("failed to gen hidden index schema", K(ret)); } else if (hidden_table_schema.get_part_level() > 0 && new_index_schema.is_index_local_storage() && OB_FAIL(new_index_schema.assign_partition_schema(hidden_table_schema))) { LOG_WARN("fail to assign partition schema", K(ret), K(new_index_schema)); } else { uint64_t new_idx_tid = OB_INVALID_ID; if (OB_FAIL(schema_service->fetch_new_table_id( tenant_id, new_idx_tid))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else if (OB_FAIL(generate_object_id_for_partition_schema(new_index_schema))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(new_index_schema)); } else if (OB_FAIL(generate_tablet_id(new_index_schema))) { LOG_WARN("fail to generate tablet id for hidden table", K(ret), K(new_index_schema)); } else { new_index_schema.set_table_id(new_idx_tid); new_index_schema.set_data_table_id(hidden_table_schema.get_table_id()); new_index_schema.set_index_status(INDEX_STATUS_UNAVAILABLE); new_index_schema.set_table_name(new_index_table_name); // set the hidden attributes of the table new_index_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); bool is_exist = false; if (OB_FAIL(ret)) { } else if (OB_FAIL(new_table_schemas.push_back(new_index_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } else if (OB_FAIL(index_ids.push_back(new_idx_tid))) { LOG_WARN("failed to add new index id!", K(ret)); } } } } } } //end for } return ret; } int ObDDLService::rebuild_hidden_table_index_in_trans( ObAlterTableArg &alter_table_arg, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, ObSArray &new_table_schemas) { int ret = OB_SUCCESS; const uint64_t tenant_id = hidden_table_schema.get_tenant_id(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { for (int64_t i = 0; i < new_table_schemas.count() && OB_SUCC(ret); i++) { ObTableSchema &tmp_schema = new_table_schemas.at(i); tmp_schema.set_in_offline_ddl_white_list(true); // allow rebuild table index of hidden table if (OB_FAIL(ddl_operator.create_table(tmp_schema, trans, NULL, true))) { LOG_WARN("failed to create table schema, ", K(ret)); } } if (OB_SUCC(ret)) { const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version(); for (int64_t i = 0; OB_SUCC(ret) && i < new_table_schemas.count(); i++) { const share::schema::ObTableSchema &this_table = new_table_schemas.at(i); uint64_t table_id = this_table.get_table_id(); bool has_tablet = false; if (!is_inner_table(table_id)) { has_tablet = this_table.has_tablet(); } else { has_tablet = is_system_table(table_id); } if (!has_tablet) { } else if (OB_FAIL(create_index_tablet(this_table, trans, schema_guard))) { LOG_WARN("create table tablets failed", K(ret), K(this_table)); } else {} if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.insert_ori_schema_version( trans, tenant_id, table_id, last_schema_version))) { LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(tenant_id), K(table_id), K(last_schema_version)); } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } return ret; } int ObDDLService::rebuild_hidden_table_index(obrpc::ObAlterTableArg &alter_table_arg, const int64_t frozen_version, ObSArray &index_ids) { int ret = OB_SUCCESS; const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); ObColumnNameMap col_name_map; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_ISNULL(tz_info_wrap.get_time_zone_info())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); ObSArray new_table_schemas; ObSchemaGetterGuard schema_guard; const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *hidden_table_schema = NULL; schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; ObArray drop_cols_id_arr; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { LOG_WARN("failed to get orig and hidden table schema", K(ret)); } else if (OB_FAIL(col_name_map.init(*orig_table_schema, alter_table_schema))) { LOG_WARN("failed to init column name map", K(ret), K(alter_table_schema), KPC(orig_table_schema)); } else if (OB_FAIL(get_all_dropped_column_ids(alter_table_arg, *orig_table_schema, drop_cols_id_arr))) { LOG_WARN("fail to get drop cols id set", K(ret)); } else if (OB_FAIL(reconstruct_index_schema(*orig_table_schema, *hidden_table_schema, schema_guard, drop_cols_id_arr, col_name_map, *tz_info_wrap.get_time_zone_info(), frozen_version, alter_table_arg.allocator_, new_table_schemas, index_ids))) { LOG_WARN("failed to reconstruct index schema", K(ret)); } else if (OB_FAIL(add_new_index_schema(alter_table_arg, *orig_table_schema, *hidden_table_schema, schema_guard, frozen_version, new_table_schemas, index_ids))) { LOG_WARN("failed to add new index schema", K(ret)); } else if (OB_FAIL(rebuild_hidden_table_index_in_trans(alter_table_arg, *hidden_table_schema, schema_guard, trans, new_table_schemas))) { LOG_WARN("failed to rebuild hidden table index in trans", K(ret)); } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } return ret; } int ObDDLService::get_hidden_table_column_id_by_orig_column_id( const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, const ObColumnNameMap &col_name_map, const uint64_t orig_column_id, uint64_t &hidden_column_id) const { int ret = OB_SUCCESS; ObString orig_column_name; ObString col_name; const ObColumnSchemaV2 *orig_column = nullptr; const ObColumnSchemaV2 *hidden_column = nullptr; if (OB_ISNULL(orig_column = orig_table_schema.get_column_schema(orig_column_id))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get column from orig table schema", K(ret)); } else if (OB_FALSE_IT(orig_column_name = orig_column->get_column_name_str())) { } else if (OB_FAIL(col_name_map.get(orig_column_name, col_name))) { LOG_WARN("invalid column name", K(ret), K(orig_column_name)); } else if (OB_ISNULL(hidden_column = hidden_table_schema.get_column_schema(col_name))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get column from hidden table schema", K(ret), K(col_name)); } else { hidden_column_id = hidden_column->get_column_id(); } return ret; } int ObDDLService::copy_constraint_for_hidden_table( const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, const ObColumnNameMap &col_name_map, const ObConstraint &orig_constraint, ObConstraint &hidden_constraint) { int ret = OB_SUCCESS; ObSEArray column_ids; if (OB_FAIL(hidden_constraint.assign(orig_constraint))) { LOG_WARN("failed to assign constraint", K(ret)); } for (ObConstraint::const_cst_col_iterator it = hidden_constraint.cst_col_begin(); OB_SUCC(ret) && it != hidden_constraint.cst_col_end(); it++) { uint64_t orig_column_id = *it; uint64_t column_id = 0; if (OB_FAIL(get_hidden_table_column_id_by_orig_column_id(orig_table_schema, hidden_table_schema, col_name_map, orig_column_id, column_id))) { LOG_WARN("failed to get column id", K(ret), K(orig_column_id)); } else if (OB_FAIL(column_ids.push_back(column_id))) { LOG_WARN("failed to push back column id", K(ret)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(hidden_constraint.assign_column_ids(column_ids))) { LOG_WARN("failed to assign new column ids to constraint", K(ret)); } else { // do nothing } return ret; } // confirm constraints that need to rebuild. int ObDDLService::get_rebuild_constraints( const ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, ObIArray &rebuild_constraints) { int ret = OB_SUCCESS; ObArray drop_cols_id_arr; rebuild_constraints.reset(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(get_all_dropped_column_ids(alter_table_arg, orig_table_schema, drop_cols_id_arr))) { LOG_WARN("fail to get drop cols id set", K(ret)); } else { // remove invalid constraint caused by drop column. for (ObTableSchema::const_constraint_iterator iter = orig_table_schema.constraint_begin(); OB_SUCC(ret) && iter != orig_table_schema.constraint_end(); ++iter) { if (OB_ISNULL(*iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("iter is NULL", K(ret)); } else if (CONSTRAINT_TYPE_PRIMARY_KEY == (*iter)->get_constraint_type()) { // the primary key has been added when creating the hidden table, so won't add it again } else { bool need_rebuild = true; const ObString &cst_name = (*iter)->get_constraint_name_str(); // check whether the constraint is invalid caused by drop constraint. if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == alter_table_arg.alter_constraint_type_) { const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; for (ObTableSchema::const_constraint_iterator dropped_cst = alter_table_schema.constraint_begin(); OB_SUCC(ret) && need_rebuild && dropped_cst != alter_table_schema.constraint_end(); dropped_cst++) { if (0 == cst_name.case_compare((*dropped_cst)->get_constraint_name_str())) { need_rebuild = false; } } } // check whether the constraint is invalid caused by drop column. for (ObConstraint::const_cst_col_iterator it_col = (*iter)->cst_col_begin(); OB_SUCC(ret) && need_rebuild && it_col != (*iter)->cst_col_end(); it_col++) { FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && need_rebuild) { if (*dropped_col == *it_col) { need_rebuild = false; } } } if (OB_FAIL(ret)) { } else if (need_rebuild && OB_FAIL(rebuild_constraints.push_back(**iter))) { LOG_WARN("failed to push back constraint", K(ret)); } } } } return ret; } int ObDDLService::rebuild_hidden_table_constraints_in_trans( const ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObMySQLTransaction &trans, ObSArray &cst_ids) { int ret = OB_SUCCESS; ObTableSchema new_table_schema; ObColumnNameMap col_name_map; ObArray rebuild_constraints; ObSchemaService *schema_service = schema_service_->get_schema_service(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_FAIL(new_table_schema.assign(hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(get_rebuild_constraints(alter_table_arg, orig_table_schema, rebuild_constraints))) { LOG_WARN("fail to get constraints that need to rebuild", K(ret)); } else if (OB_FAIL(col_name_map.init(orig_table_schema, alter_table_arg.alter_table_schema_))) { LOG_WARN("failed to init column name map", K(ret)); } else { new_table_schema.clear_constraint(); new_table_schema.set_in_offline_ddl_white_list(true); for (int64_t i = 0; OB_SUCC(ret) && i < rebuild_constraints.size(); i++) { ObConstraint constraint; if (OB_FAIL(copy_constraint_for_hidden_table(orig_table_schema, hidden_table_schema, col_name_map, rebuild_constraints.at(i), constraint))) { LOG_WARN("failed to assign constraint", K(ret)); } else if (OB_FAIL(new_table_schema.add_constraint(constraint))) { LOG_WARN("failed to add constraint", K(ret)); } else {/* do nothing. */} } if (OB_SUCC(ret) && new_table_schema.has_constraint()) { if (OB_FAIL(ddl_operator.add_table_constraints(new_table_schema, new_table_schema, trans, &cst_ids))) { LOG_WARN("failed to add constraints", K(ret)); } else { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema, trans, operation_type))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } return ret; } int ObDDLService::rebuild_hidden_table_constraints(ObAlterTableArg &alter_table_arg, ObSArray &cst_ids) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *hidden_table_schema = NULL; ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { LOG_WARN("failed to get orig and hidden table schema", K(ret)); } else if (OB_FAIL(rebuild_hidden_table_constraints_in_trans(alter_table_arg, *orig_table_schema, *hidden_table_schema, trans, cst_ids))) { LOG_WARN("failed to rebuild hidden table constraints in trans", K(ret)); } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } return ret; } int ObDDLService::rebuild_triggers_on_hidden_table( const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { const uint64_t tenant_id = orig_table_schema.get_tenant_id(); const ObIArray &trigger_list = orig_table_schema.get_trigger_list(); const ObTriggerInfo *trigger_info = NULL; ObTriggerInfo new_trigger_info; ObErrorInfo error_info; for (int i = 0; OB_SUCC(ret) && i < trigger_list.count(); i++) { OZ (schema_guard.get_trigger_info(tenant_id, trigger_list.at(i), trigger_info)); OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_list.at(i)); OZ (new_trigger_info.assign(*trigger_info)); OX (new_trigger_info.set_base_object_id(hidden_table_schema.get_table_id())); OX (new_trigger_info.set_trigger_id(OB_INVALID_ID)); OX (new_trigger_info.set_tenant_id(hidden_table_schema.get_tenant_id())); OZ (ddl_operator.drop_trigger(*trigger_info, trans, nullptr, false/*is_update_table_schema_version*/)); OZ (ddl_operator.create_trigger(new_trigger_info, trans, error_info, nullptr, false/*for_insert_errors*/, false/*is_update_table_schema_version*/)); } } return ret; } int ObDDLService::swap_child_table_fk_name( const uint64_t child_table_id, const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, ObIAllocator &allocator) { int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObTableSchema new_table_schema; const ObTableSchema *child_table_schema; ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, child_table_id, child_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id)); } else if (OB_ISNULL(child_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to get child table schema", K(ret)); } else if (OB_FAIL(new_table_schema.assign(*child_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { ObIArray &new_foreign_key_infos = new_table_schema.get_foreign_key_infos(); const ObIArray &orig_foreign_key_infos = child_table_schema->get_foreign_key_infos(); for (int64_t i = 0; OB_SUCC(ret) && i < new_foreign_key_infos.count(); i++) { ObForeignKeyInfo &fk_info1 = new_foreign_key_infos.at(i); // rename hidden fk_name to orig fk_name if (fk_info1.parent_table_id_ == hidden_table_schema.get_table_id()) { bool is_prefix_match = false; for (int64_t j = 0; OB_SUCC(ret) && !is_prefix_match && j < orig_foreign_key_infos.count(); j++) { const ObForeignKeyInfo &fk_info2 = orig_foreign_key_infos.at(j); if (OB_FAIL(is_foreign_key_name_prefix_match(fk_info2, fk_info1, allocator, is_prefix_match))) { LOG_WARN("is foreign key name prefix match", K(ret)); } else if (is_prefix_match && fk_info1.child_table_id_ == fk_info2.child_table_id_ && fk_info2.parent_table_id_ == orig_table_schema.get_table_id()) { fk_info1.set_is_modify_fk_state(true); fk_info1.set_is_modify_fk_name_flag(true); if (OB_FAIL(ob_write_string(allocator, fk_info2.foreign_key_name_, fk_info1.foreign_key_name_))) { LOG_WARN("Can not malloc space for fk name", K(ret)); } } } if (OB_SUCC(ret) && !is_prefix_match) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error, can not found match hidden fk info", K(ret), K(fk_info1), K(orig_foreign_key_infos)); } } else if (fk_info1.parent_table_id_ == orig_table_schema.get_table_id()) { // rename orig fk_name to hidden fk_name bool is_prefix_match = false; for (int64_t j = 0; OB_SUCC(ret) && !is_prefix_match && j < orig_foreign_key_infos.count(); j++) { const ObForeignKeyInfo &fk_info2 = orig_foreign_key_infos.at(j); if (OB_FAIL(is_foreign_key_name_prefix_match(fk_info1, fk_info2, allocator, is_prefix_match))) { LOG_WARN("is foreign key name prefix match", K(ret)); } else if (is_prefix_match && fk_info1.child_table_id_ == fk_info2.child_table_id_ && fk_info2.parent_table_id_ == hidden_table_schema.get_table_id()) { fk_info1.set_is_modify_fk_state(true); fk_info1.set_is_modify_fk_name_flag(true); if (OB_FAIL(ob_write_string(allocator, fk_info2.foreign_key_name_, fk_info1.foreign_key_name_))) { LOG_WARN("Can not malloc space for fk name", K(ret)); } } } if (OB_SUCC(ret) && !is_prefix_match) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error, can not found match hidden fk info", K(ret), K(fk_info1), K(orig_foreign_key_infos)); } } } // update foreign key name if (OB_SUCC(ret)) { new_table_schema.set_in_offline_ddl_white_list(true); if (OB_FAIL(new_table_schema.add_depend_table_id(orig_table_schema.get_table_id())) || OB_FAIL(new_table_schema.add_depend_table_id(hidden_table_schema.get_table_id()))) { LOG_WARN("failed to add depend table id", K(ret)); } else if (OB_FAIL(ddl_operator.update_table_foreign_keys( new_table_schema, trans, true/*in_offline_ddl_white_list*/))) { LOG_WARN("failed to update foreign key", K(ret)); } } } return ret; } int ObDDLService::drop_child_table_fk( const ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; ObSchemaService *schema_service = schema_service_->get_schema_service(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObArray cst_ids; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else { common::ObSEArray child_table_ids; const ObIArray &foreign_key_infos = orig_table_schema.get_foreign_key_infos(); const bool prev_in_offline_ddl_white_list = orig_table_schema.get_in_offline_ddl_white_list(); for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); if (foreign_key_info.parent_table_id_ == orig_table_schema.get_table_id() && foreign_key_info.child_table_id_ != orig_table_schema.get_table_id()) { ObDropForeignKeyArg drop_fk_arg; drop_fk_arg.foreign_key_name_ = foreign_key_info.foreign_key_name_; const ObTableSchema *child_table_schema = nullptr; const ObForeignKeyInfo *parent_table_mock_fk_info = nullptr; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, foreign_key_info.child_table_id_, child_table_schema))) { LOG_WARN("failed to get child table schema", K(ret)); } else if (OB_ISNULL(child_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid child table schema", K(ret)); } else if (OB_FAIL(ddl_operator.alter_table_drop_foreign_key(*child_table_schema, drop_fk_arg, trans, parent_table_mock_fk_info, true/*parent_table_in_offline_ddl_white_list*/))) { LOG_WARN("failed to drop child table foreign key", K(ret)); } } } } return ret; } int ObDDLService::swap_all_child_table_fk_name( const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, ObIAllocator &allocator) { int ret = OB_SUCCESS; ObSchemaService *schema_service = schema_service_->get_schema_service(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else { common::ObSEArray child_table_ids; const ObIArray &foreign_key_infos = orig_table_schema.get_foreign_key_infos(); // get all child table id for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); if (foreign_key_info.parent_table_id_ == orig_table_schema.get_table_id() && foreign_key_info.child_table_id_ != orig_table_schema.get_table_id()) { if (OB_FAIL(child_table_ids.push_back(foreign_key_info.child_table_id_))) { LOG_WARN("failed to add table id", K(ret)); } } } // modify all child table foreign key for (int64_t i = 0; OB_SUCC(ret) && i < child_table_ids.count(); i++) { uint64_t child_table_id = child_table_ids.at(i); if (OB_FAIL(swap_child_table_fk_name(child_table_id, orig_table_schema, hidden_table_schema, schema_guard, ddl_operator, trans, allocator))) { LOG_WARN("fail to swap child table fk name", K(ret));; } } } return ret; } int ObDDLService::build_hidden_index_table_map( const ObTableSchema &hidden_table_schema, ObSchemaGetterGuard &schema_guard, hash::ObHashMap &new_index_table_map) { int ret = OB_SUCCESS; const uint64_t tenant_id = hidden_table_schema.get_tenant_id(); ObSEArray simple_index_infos; if (OB_FAIL(hidden_table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObSimpleTableSchemaV2 *index_table_schema = NULL; if (OB_FAIL(schema_guard.get_simple_table_schema( tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", simple_index_infos.at(i).table_id_, K(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (OB_FAIL(new_index_table_map.set_refactored(index_table_schema->get_table_name_str(), index_table_schema->get_table_id()))) { LOG_WARN("failed to insert into table map", K(ret)); } } return ret; } int ObDDLService::convert_hidden_table_column_ids_by_orig_column_ids( const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, const ObColumnNameMap &col_name_map, ObIArray &column_ids) const { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < column_ids.count(); i++) { const uint64_t orig_column_id = column_ids.at(i); uint64_t &hidden_column_id = column_ids.at(i); if (OB_FAIL(get_hidden_table_column_id_by_orig_column_id(orig_table_schema, hidden_table_schema, col_name_map, orig_column_id, hidden_column_id))) { LOG_WARN("failed to get column id", K(ret), K(orig_column_id)); } else { // do nothing } } return ret; } // remove invalid foreign key, caused by drop column and drop fk. int ObDDLService::get_rebuild_foreign_key_infos( const ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const bool rebuild_child_table_fk, ObArray &rebuild_fk_infos) { int ret = OB_SUCCESS; rebuild_fk_infos.reset(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { const ObSArray &index_arg_list = alter_table_arg.index_arg_list_; const ObIArray &orig_fk_infos = orig_table_schema.get_foreign_key_infos(); for (int64_t i = 0; OB_SUCC(ret) && i < orig_fk_infos.count(); i++) { bool need_rebuild = true; ObForeignKeyInfo new_fk_info = orig_fk_infos.at(i); for (int64_t j = 0; OB_SUCC(ret) && need_rebuild && j < index_arg_list.size(); j++) { ObIndexArg *index_arg = const_cast(index_arg_list.at(j)); if (OB_ISNULL(index_arg)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument, index arg should not be null", K(ret)); } else if (ObIndexArg::DROP_FOREIGN_KEY == index_arg->index_action_type_) { const ObDropForeignKeyArg *drop_fk_arg = static_cast(index_arg); if (0 == (drop_fk_arg->foreign_key_name_).case_compare(new_fk_info.foreign_key_name_)) { need_rebuild = false; } else {/* do nothing.*/ } } } if (OB_SUCC(ret) && need_rebuild) { const int64_t orig_table_id = orig_table_schema.get_table_id(); if (new_fk_info.child_table_id_ != orig_table_id && new_fk_info.parent_table_id_ == orig_table_id) { need_rebuild = rebuild_child_table_fk; } else if (new_fk_info.child_table_id_ == orig_table_id) { need_rebuild = !rebuild_child_table_fk; } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected fk", K(ret), K(new_fk_info), K(orig_table_id)); } } if (OB_FAIL(ret)) { } else if (need_rebuild && OB_FAIL(rebuild_fk_infos.push_back(new_fk_info))) { LOG_WARN("fail to push back fk infos that need to rebuild", K(ret)); } else {/* do nothing. */} } } return ret; } int ObDDLService::rebuild_hidden_table_foreign_key_in_trans( ObAlterTableArg &alter_table_arg, const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, const bool rebuild_child_table_fk, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, ObSArray &cst_ids) { int ret = OB_SUCCESS; bool is_oracle_mode = false; ObTableSchema inc_table_schema; ObArray rebuild_fk_infos; ObColumnNameMap col_name_map; common::hash::ObHashMap new_index_table_map; ObSchemaService *schema_service = schema_service_->get_schema_service(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); const uint64_t tenant_id = hidden_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_FAIL(inc_table_schema.assign(hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(new_index_table_map.create(16, "HashBucIdxTabMa"))) { LOG_WARN("failed to add create ObHashMap", K(ret)); } else if (OB_FAIL(build_hidden_index_table_map(hidden_table_schema, schema_guard, new_index_table_map))) { LOG_WARN("failed to build hidden index table map", K(ret)); } else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to check if oralce compat mode", K(ret)); } else if (OB_FAIL(get_rebuild_foreign_key_infos(alter_table_arg, orig_table_schema, rebuild_child_table_fk, rebuild_fk_infos))) { LOG_WARN("fail to get fk infos that need to rebuild", K(ret)); } else if (OB_FAIL(inc_table_schema.set_foreign_key_infos(rebuild_fk_infos))) { LOG_WARN("fail to set fk infos", K(ret)); } else if (OB_FAIL(col_name_map.init(orig_table_schema, alter_table_arg.alter_table_schema_))) { LOG_WARN("failed to init column name map", K(ret)); } else { ObIArray &foreign_key_infos = inc_table_schema.get_foreign_key_infos(); for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); const uint64_t origin_fk_id = foreign_key_info.foreign_key_id_; foreign_key_info.foreign_key_id_ = OB_INVALID_ID; // add depend table id if (foreign_key_info.parent_table_id_ != orig_table_schema.get_table_id()) { if (!foreign_key_info.is_parent_table_mock_) { if (OB_FAIL(inc_table_schema.add_depend_table_id(foreign_key_info.parent_table_id_))) { LOG_WARN("fail to add depend table id", K(ret)); } } else if (OB_FAIL(inc_table_schema.add_depend_mock_fk_parent_table_id(foreign_key_info.parent_table_id_))) { LOG_WARN("fail to add depend table id", K(ret)); } } if (OB_FAIL(ret)) { } else if (foreign_key_info.child_table_id_ != orig_table_schema.get_table_id() && OB_FAIL(inc_table_schema.add_depend_table_id(foreign_key_info.child_table_id_))) { LOG_WARN("fail to add depend table id", K(ret)); } else if (OB_FAIL(schema_service->fetch_new_constraint_id( tenant_id, foreign_key_info.foreign_key_id_))) { LOG_WARN("failed to fetch new foreign key id", K(ret), K(tenant_id)); } else if (foreign_key_info.parent_table_id_ == orig_table_schema.get_table_id()) { // update referenced constraint id if (CONSTRAINT_TYPE_PRIMARY_KEY == foreign_key_info.ref_cst_type_) { if (is_oracle_mode) { const ObConstraint *pk_cst = hidden_table_schema.get_pk_constraint(); if (OB_ISNULL(pk_cst)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("primary key cst is null", K(ret)); } else { foreign_key_info.ref_cst_id_ = pk_cst->get_constraint_id(); } } else { foreign_key_info.ref_cst_id_ = common::OB_INVALID_ID; } } else { const ObSimpleTableSchemaV2 *orig_index_table_schema = NULL; ObString new_index_table_name; uint64_t new_ref_cst_id = OB_INVALID_ID; if (OB_FAIL(schema_guard.get_simple_table_schema( tenant_id, foreign_key_info.ref_cst_id_, orig_index_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", foreign_key_info.ref_cst_id_, K(ret)); } else if (OB_ISNULL(orig_index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (OB_FAIL(gen_new_index_table_name( orig_index_table_schema->get_table_name_str(), orig_table_schema.get_table_id(), hidden_table_schema.get_table_id(), alter_table_arg.allocator_, new_index_table_name))) { LOG_WARN("failed to build new index table name!", K(hidden_table_schema.get_table_id()), K(new_index_table_name), K(ret)); } else if (OB_FAIL(new_index_table_map.get_refactored(new_index_table_name, new_ref_cst_id))) { LOG_WARN("failed to get new ref cst id", K(new_index_table_name), K(ret)); } else { foreign_key_info.ref_cst_id_ = new_ref_cst_id; } } } if (OB_SUCC(ret)) { if (orig_table_schema.is_parent_table() && foreign_key_info.parent_table_id_ == orig_table_schema.get_table_id()) { // self-dependent if (foreign_key_info.child_table_id_ == foreign_key_info.parent_table_id_) { foreign_key_info.parent_table_id_ = hidden_table_schema.get_table_id(); foreign_key_info.child_table_id_ = hidden_table_schema.get_table_id(); if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.parent_column_ids_))) { LOG_WARN("failed to convert parent column ids of foreign key", K(ret)); } else if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.child_column_ids_))) { LOG_WARN("failed to convert child column ids of foreign key", K(ret)); } } else { foreign_key_info.parent_table_id_ = hidden_table_schema.get_table_id(); if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.parent_column_ids_))) { LOG_WARN("failed to convert parent column ids of foreign key", K(ret)); } } } else { foreign_key_info.child_table_id_ = hidden_table_schema.get_table_id(); if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.child_column_ids_))) { LOG_WARN("failed to convert child column ids of foreign key", K(ret)); } } if (OB_SUCC(ret) && foreign_key_info.validate_flag_) { if (OB_FAIL(cst_ids.push_back(foreign_key_info.foreign_key_id_))) { LOG_WARN("failed to add new foreign key id!", K(ret)); } } } } if (OB_SUCC(ret)) { inc_table_schema.set_in_offline_ddl_white_list(true); if (OB_FAIL(ddl_operator.add_table_foreign_keys(hidden_table_schema, inc_table_schema, trans))) { LOG_WARN("failed to add foreign key", K(ret)); } else { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; if (OB_FAIL(ddl_operator.update_table_attribute(inc_table_schema, trans, operation_type))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } } } } return ret; } int ObDDLService::rebuild_hidden_table_foreign_key(ObAlterTableArg &alter_table_arg, ObSArray &cst_ids) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *hidden_table_schema = NULL; ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { LOG_WARN("failed to get orig and hidden table schema", K(ret)); } else if (OB_FAIL(rebuild_hidden_table_foreign_key_in_trans(alter_table_arg, *orig_table_schema, *hidden_table_schema, false/*rebuild_child_table_fk*/, schema_guard, trans, cst_ids))) { LOG_WARN("failed to rebuild hidden table foreign key in trans", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } return ret; } int ObDDLService::get_orig_and_hidden_table_schema( const ObAlterTableArg &alter_table_arg, ObSchemaGetterGuard &schema_guard, const AlterTableSchema &alter_table_schema, const ObTableSchema *&orig_table_schema, const ObTableSchema *&hidden_table_schema) { int ret = OB_SUCCESS; const int64_t orig_table_id = alter_table_arg.table_id_; const int64_t hidden_table_id = alter_table_arg.hidden_table_id_; const uint64_t tenant_id = alter_table_arg.alter_table_schema_.get_tenant_id(); if (OB_FAIL(schema_guard.get_table_schema(tenant_id, orig_table_id, orig_table_schema))) { LOG_WARN("failed to get orig table schema", K(ret), K(orig_table_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, hidden_table_id, hidden_table_schema))) { LOG_WARN("fail to get hidden table schema", K(ret), K(hidden_table_id)); } else if (OB_ISNULL(orig_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("failed to get orig table schema", K(ret), K(orig_table_id)); } else if (OB_ISNULL(hidden_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("fail to get hidden table schema", K(ret), K(hidden_table_id)); } else if (OB_UNLIKELY(orig_table_schema->get_association_table_id() != hidden_table_schema->get_table_id())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("not in offline ddl", K(ret), K(orig_table_id), K(hidden_table_id), K(orig_table_schema->get_association_table_id())); } return ret; } int ObDDLService::get_tablets( const ObTableSchema &table_schema, const uint64_t session_id, ObIArray &tablets, ObDDLSQLTransaction &trans) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_schema.get_tenant_id(); ObArray tablet_ids; ObArray ls_ids; if (OB_FAIL(table_schema.get_tablet_ids(tablet_ids))) { LOG_WARN("failed to get tablets", K(ret)); } else if (OB_FAIL(ObTabletToLSTableOperator::batch_get_ls(trans, tenant_id, tablet_ids, ls_ids))) { LOG_WARN("failed to batch get ls", K(ret)); } else if (OB_UNLIKELY(tablet_ids.count() != ls_ids.count())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid tablet ids ls ids", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); i++) { if (OB_FAIL(tablets.push_back({ls_ids[i], tablet_ids[i]}))) { LOG_WARN("failed to push back tablet id and ls id", K(ret)); } } return ret; } int ObDDLService::build_modify_tablet_binding_args( const ObTableSchema &table_schema, const bool is_hidden_tablets, const int64_t schema_version, const uint64_t session_id, ObIArray &modify_args, ObDDLSQLTransaction &trans) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_schema.get_tenant_id(); ObArray tablets; if (OB_FAIL(get_tablets(table_schema, session_id, tablets, trans))) { LOG_WARN("failed to get tablet ids of orig table", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < tablets.count(); i++) { const ObLSID &ls_id = tablets[i].first; int64_t j = 0; for (; j < modify_args.count(); j++) { if (modify_args.at(j).ls_id_ == ls_id && modify_args.at(j).tenant_id_ == tenant_id) { break; } } if (j == modify_args.count()) { ObBatchUnbindTabletArg modify_arg; modify_arg.tenant_id_ = tenant_id; modify_arg.ls_id_ = ls_id; modify_arg.schema_version_ = schema_version; if (OB_FAIL(modify_args.push_back(modify_arg))) { LOG_WARN("failed to push back modify arg", K(ret)); } } if (OB_SUCC(ret)) { ObBatchUnbindTabletArg &modify_arg = modify_args.at(j); const ObTabletID &tablet_id = tablets[i].second; if (is_hidden_tablets) { if (OB_FAIL(modify_arg.hidden_tablet_ids_.push_back(tablet_id))) { LOG_WARN("failed to push back", K(ret)); } } else { if (OB_FAIL(modify_arg.orig_tablet_ids_.push_back(tablet_id))) { LOG_WARN("failed to push back", K(ret)); } } } } LOG_DEBUG("build modify tablet binding args", K(ret), K(modify_args)); return ret; } int ObDDLService::unbind_hidden_tablets( const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, const int64_t schema_version, const uint64_t session_id, ObDDLSQLTransaction &trans) { int ret = OB_SUCCESS; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObArray args; if (OB_FAIL(build_modify_tablet_binding_args( orig_table_schema, false/*is_hidden_tablets*/, schema_version, session_id, args, trans))) { LOG_WARN("failed to build reuse index args", K(ret)); } else if (OB_FAIL(build_modify_tablet_binding_args( hidden_table_schema, true/*is_hidden_tablets*/, schema_version, session_id, args, trans))) { LOG_WARN("failed to build reuse index args", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < args.count(); i++) { int64_t pos = 0; int64_t size = args[i].get_serialize_size(); ObArenaAllocator allocator; char *buf = nullptr; if (OB_ISNULL(buf = static_cast(allocator.alloc(size)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to allocate", K(ret)); } else if (OB_FAIL(args[i].serialize(buf, size, pos))) { LOG_WARN("failed to serialize arg", K(ret)); } else if (OB_FAIL(trans.register_tx_data(args[i].tenant_id_, args[i].ls_id_, transaction::ObTxDataSourceType::MODIFY_TABLET_BINDING, buf, pos))) { LOG_WARN("failed to register tx data", K(ret)); } } return ret; } int ObDDLService::write_ddl_barrier( const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, const uint64_t session_id, ObDDLSQLTransaction &trans) { UNUSED(orig_table_schema); int ret = OB_SUCCESS; const uint64_t tenant_id = hidden_table_schema.get_tenant_id(); ObArray hidden_tablets; if (OB_FAIL(get_tablets(hidden_table_schema, session_id, hidden_tablets, trans))) { LOG_WARN("failed to get tablet", K(ret)); } else { ObArenaAllocator allocator("DDLSrvBarrier"); ObArray logs; for (int64_t i = 0; OB_SUCC(ret) && i < hidden_tablets.count(); i++) { const ObLSID &ls_id = hidden_tablets[i].first; const ObTabletID &tablet_id = hidden_tablets[i].second; int64_t j = 0; for (; j < logs.count(); j++) { if (logs[j].ls_id_ == ls_id) { break; } } if (j == logs.count()) { ObDDLBarrierLog log; log.ls_id_ = ls_id; if (OB_FAIL(logs.push_back(log))) { LOG_WARN("failed to push back log", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(logs[j].hidden_tablet_ids_.push_back(tablet_id))) { LOG_WARN("failed to push back hidden tablet", K(ret)); } } } for (int64_t i = 0; OB_SUCC(ret) && i < logs.count(); i++) { int64_t pos = 0; int64_t size = logs[i].get_serialize_size(); char *buf = nullptr; allocator.reuse(); if (OB_UNLIKELY(!logs[i].is_valid())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid ddl barrier log", K(ret), K(logs[i])); } else if (OB_ISNULL(buf = static_cast(allocator.alloc(size)))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to allocate", K(ret)); } else if (OB_FAIL(logs[i].serialize(buf, size, pos))) { LOG_WARN("failed to serialize arg", K(ret)); } else if (OB_FAIL(trans.register_tx_data(tenant_id, logs[i].ls_id_, transaction::ObTxDataSourceType::DDL_BARRIER, buf, pos))) { LOG_WARN("failed to register tx data", K(ret)); } } } return ret; } int ObDDLSQLTransaction::register_tx_data( const uint64_t tenant_id, const share::ObLSID &ls_id, const transaction::ObTxDataSourceType &type, const char *buf, const int64_t buf_len) { int ret = OB_SUCCESS; sqlclient::ObISQLConnection *isql_conn = nullptr; if (OB_ISNULL(isql_conn = get_connection())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid connection", K(ret)); } else { observer::ObInnerSQLConnection *conn = static_cast(isql_conn); if (OB_FAIL(conn->register_multi_data_source(tenant_id, ls_id, type, buf, buf_len))) { LOG_WARN("failed to register tx data", K(ret)); } } return ret; } int ObDDLService::swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *hidden_table_schema = NULL; HEAP_VARS_2((ObTableSchema, new_orig_table_schema), (ObTableSchema, new_hidden_table_schema)) { ObSchemaGetterGuard schema_guard; ObSArray table_schemas; ObSArray fk_cst_ids; ObSEArray orig_simple_index_infos; ObSEArray new_simple_index_infos; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { LOG_WARN("failed to get orig and hidden table schema", K(ret)); } else if (OB_FAIL(orig_table_schema->get_simple_index_infos( orig_simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } else if (OB_FAIL(hidden_table_schema->get_simple_index_infos( new_simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } else if (OB_FAIL(new_orig_table_schema.assign(*orig_table_schema)) || OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (0 == hidden_table_schema->get_table_name_str().case_compare(alter_table_arg.alter_table_schema_.get_origin_table_name())) { // ignore case sensitivity is expected. ret = OB_NO_NEED_UPDATE; LOG_WARN("already swapped", K(ret)); } else if (OB_FAIL(drop_child_table_fk(alter_table_arg, new_orig_table_schema, *hidden_table_schema, schema_guard, trans))) { LOG_WARN("failed to drop origin table fk", K(ret)); } else if (OB_FAIL(rebuild_hidden_table_foreign_key_in_trans(alter_table_arg, *orig_table_schema, *hidden_table_schema, true/*rebuild_child_table_fk*/, schema_guard, trans, fk_cst_ids))) { LOG_WARN("failed to rebuild hidden table fk", K(ret)); } else { if (OB_SUCC(ret) && alter_table_arg.need_rebuild_trigger_) { if (OB_FAIL(rebuild_triggers_on_hidden_table(*orig_table_schema, *hidden_table_schema, schema_guard, ddl_operator, trans))) { LOG_WARN("failed to create triggers on hidden table", K(ret)); } } // prepare schema if (OB_SUCC(ret)) { ObTableSchema tmp_schema; for (int64_t i = 0; OB_SUCC(ret) && i < orig_simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; tmp_schema.reset(); if (OB_FAIL(schema_guard.get_table_schema( tenant_id, orig_simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", orig_simple_index_infos.at(i).table_id_, K(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (OB_FAIL(tmp_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { // modify the state of the all original index tables to hidden tmp_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); if (OB_FAIL(table_schemas.push_back(tmp_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } } for (int64_t i = 0; OB_SUCC(ret) && i < new_simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; tmp_schema.reset(); if (OB_FAIL(schema_guard.get_table_schema( tenant_id, new_simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", new_simple_index_infos.at(i).table_id_, K(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (OB_FAIL(tmp_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { // modify the state of the all hidden index tables to non-hidden tmp_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL); if (OB_FAIL(table_schemas.push_back(tmp_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } } } if (OB_SUCC(ret)) { new_orig_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); new_hidden_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL); new_orig_table_schema.set_table_name(hidden_table_schema->get_table_name_str()); new_hidden_table_schema.set_table_name(orig_table_schema->get_table_name_str()); if (OB_FAIL(table_schemas.push_back(new_orig_table_schema)) || OB_FAIL(table_schemas.push_back(new_hidden_table_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) { const ObString *ddl_stmt_str = NULL; ObTableSchema &tmp_schema = table_schemas.at(i); ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; tmp_schema.set_in_offline_ddl_white_list(true); if (tmp_schema.get_table_id() == orig_table_schema->get_table_id()) { ddl_stmt_str = &alter_table_arg.ddl_stmt_str_; } if (OB_FAIL(ddl_operator.update_table_attribute(tmp_schema, trans, operation_type, ddl_stmt_str))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } } if (OB_SUCC(ret)) { int64_t schema_version = table_schemas[table_schemas.count()-1].get_schema_version(); if (OB_FAIL(unbind_hidden_tablets(*orig_table_schema, *hidden_table_schema, schema_version, alter_table_arg.session_id_, trans))) { LOG_WARN("failed to unbind hidden tablets", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(write_ddl_barrier(*orig_table_schema, *hidden_table_schema, alter_table_arg.session_id_, trans))) { LOG_WARN("failed to write ddl barrier", K(ret)); } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } DEBUG_SYNC(SWAP_ORIG_AND_HIDDEN_TABLE_BEFORE_PUBLISH_SCHEMA); int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } if (OB_NO_NEED_UPDATE == ret) { ret = OB_SUCCESS; } return ret; } int ObDDLService::modify_hidden_table_fk_state(obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (alter_table_arg.foreign_key_arg_list_.empty()) { // do nothing } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); bool is_oracle_mode = false; bool to_recyclebin = false; const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *hidden_table_schema = NULL; ObTableSchema new_hidden_table_schema; ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { LOG_WARN("failed to get orig and hidden table schema", K(ret)); } else if (OB_FAIL(hidden_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to check if oralce compat mode", K(ret)); } else if (OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { new_hidden_table_schema.reset_foreign_key_infos(); // must reset, to avoid insert all fk records into all_foreign_key. for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.foreign_key_arg_list_.count(); i++) { const ObCreateForeignKeyArg &foreign_key_arg = alter_table_arg.foreign_key_arg_list_.at(i); if (foreign_key_arg.is_modify_fk_state_) { bool is_found = false; ObForeignKeyInfo found_foreign_key_info; found_foreign_key_info.reset(); const ObIArray &foreign_key_infos = hidden_table_schema->get_foreign_key_infos(); for (int64_t j = 0; OB_SUCC(ret) && !is_found && j < foreign_key_infos.count(); ++j) { if (is_oracle_mode) { // case sensitive. is_found = 0 == foreign_key_arg.foreign_key_name_.compare(foreign_key_infos.at(j).foreign_key_name_) ? true : false; } else { is_found = 0 == foreign_key_arg.foreign_key_name_.case_compare(foreign_key_infos.at(j).foreign_key_name_) ? true : false; } if (is_found) { if (OB_FAIL(found_foreign_key_info.assign(foreign_key_infos.at(j)))) { LOG_WARN("fail to assign foreign key info", K(ret)); } else { found_foreign_key_info.is_modify_fk_state_ = true; if (foreign_key_arg.is_modify_enable_flag_) { found_foreign_key_info.is_modify_enable_flag_ = true; found_foreign_key_info.enable_flag_ = foreign_key_arg.enable_flag_; } if (foreign_key_arg.is_modify_validate_flag_) { found_foreign_key_info.is_modify_validate_flag_ = true; found_foreign_key_info.validate_flag_ = foreign_key_arg.validate_flag_; } if (foreign_key_arg.is_modify_rely_flag_) { found_foreign_key_info.is_modify_rely_flag_ = true; found_foreign_key_info.rely_flag_ = foreign_key_arg.rely_flag_; } } } } if (!is_found) { ret = OB_ERR_UNEXPECTED; LOG_WARN("no such fk constraint", K(ret), K(foreign_key_arg.foreign_key_name_)); } if (OB_SUCC(ret)) { if (found_foreign_key_info.child_table_id_ != found_foreign_key_info.parent_table_id_) { // If the reference table is itself, there is no need to update sync_versin_for_cascade_table if (new_hidden_table_schema.get_table_id() != found_foreign_key_info.parent_table_id_ && new_hidden_table_schema.get_table_id() != found_foreign_key_info.child_table_id_) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected new hidden table schema", K(ret), K(new_hidden_table_schema), K(found_foreign_key_info)); } else if (new_hidden_table_schema.get_table_id() == found_foreign_key_info.parent_table_id_ && OB_FAIL(new_hidden_table_schema.add_depend_table_id(found_foreign_key_info.child_table_id_))) { LOG_WARN("failed to add dependent table id", K(ret), K(found_foreign_key_info)); } else if (new_hidden_table_schema.get_table_id() == found_foreign_key_info.child_table_id_) { if (!found_foreign_key_info.is_parent_table_mock_) { if (OB_FAIL(new_hidden_table_schema.add_depend_table_id(found_foreign_key_info.parent_table_id_))) { LOG_WARN("failed to add depend table id", K(ret), K(found_foreign_key_info)); } } else if (OB_FAIL(new_hidden_table_schema.add_depend_mock_fk_parent_table_id(found_foreign_key_info.parent_table_id_))) { LOG_WARN("failed to add depend table id", K(ret), K(found_foreign_key_info)); } } } if (OB_FAIL(ret)) { } else if (OB_FAIL(new_hidden_table_schema.add_foreign_key_info(found_foreign_key_info))) { LOG_WARN("fail to add foreign key info", K(ret), K(found_foreign_key_info)); } } } } if (OB_SUCC(ret)) { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; new_hidden_table_schema.set_in_offline_ddl_white_list(true); if (OB_FAIL(alter_table_foreign_keys(*hidden_table_schema, new_hidden_table_schema, ddl_operator, trans))) { LOG_WARN("alter table foreign keys failed", K(ret)); } else if (OB_FAIL(ddl_operator.update_table_attribute( new_hidden_table_schema, trans, operation_type))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } return ret; } int ObDDLService::modify_hidden_table_not_null_column_state(const obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); bool is_oracle_mode = false; const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *hidden_table_schema = NULL; ObTableSchema new_hidden_table_schema; ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, alter_table_arg.hidden_table_id_, hidden_table_schema))) { LOG_WARN("failed to and hidden table schema", K(ret)); } else if (OB_ISNULL(hidden_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("hidden table schema is null", K(ret)); } else if (OB_FAIL(hidden_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to check if oralce compat mode", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, hidden_table_schema->get_association_table_id(), orig_table_schema))) { LOG_WARN("failed to and hidden table schema", K(ret)); } else if (OB_ISNULL(orig_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("orig table schema is null", K(ret)); } else if (OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { ObColumnNameMap col_name_map; const ObConstraint *cst = *alter_table_schema.constraint_begin_for_non_const_iter(); const uint64_t orig_column_id = *(cst->cst_col_begin()); uint64_t hidden_column_id = 0; if (nullptr == orig_table_schema->get_column_schema(orig_column_id)) { hidden_column_id = orig_column_id; } else if (OB_FAIL(col_name_map.init(*orig_table_schema, alter_table_schema))) { LOG_WARN("failed to init column name map", K(ret)); } else if (OB_FAIL(get_hidden_table_column_id_by_orig_column_id(*orig_table_schema, *hidden_table_schema, col_name_map, orig_column_id, hidden_column_id))) { LOG_WARN("failed to get hidden table column id", K(ret), K(orig_column_id)); } if (OB_SUCC(ret)) { const ObColumnSchemaV2 *col_schema = new_hidden_table_schema.get_column_schema(hidden_column_id); if (OB_ISNULL(col_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column schema not found", K(ret), K(hidden_column_id)); } else { ObColumnSchemaV2 new_col_schema = *col_schema; new_col_schema.set_nullable(false); new_col_schema.drop_not_null_cst(); new_hidden_table_schema.set_in_offline_ddl_white_list(true); if (OB_FAIL(new_hidden_table_schema.alter_column(new_col_schema, ObTableSchema::CHECK_MODE_ONLINE))) { LOG_WARN("failed to alter column", K(ret)); } else if (OB_FAIL(ddl_operator.update_single_column(trans, *hidden_table_schema, new_hidden_table_schema, new_col_schema))) { LOG_WARN("failed to update single column", K(ret)); } else if (!is_oracle_mode && OB_FAIL(ddl_operator.drop_table_constraints(*hidden_table_schema, alter_table_schema, new_hidden_table_schema, trans))) { LOG_WARN("failed to drop constraint", K(ret)); } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } return ret; } int ObDDLService::cleanup_garbage(ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); bool to_recyclebin = false; const ObTableSchema *old_orig_table_schema = NULL; const ObTableSchema *old_hidden_table_schema = NULL; const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *hidden_table_schema = NULL; HEAP_VARS_2((ObTableSchema, new_orig_table_schema), (ObTableSchema, new_hidden_table_schema)) { ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(alter_table_arg.session_id_); int64_t refreshed_schema_version = 0; bool ddl_succ = false; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, /* get orig_table_schema through its name, thus the hidden_table_schema is orign table schema */ schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { LOG_WARN("failed to get orig and hidden table schema", K(ret)); } else { const ObString &orig_table_name = alter_table_arg.alter_table_schema_.get_origin_table_name(); // ignore case sensitivity is expected. if (0 == orig_table_schema->get_table_name_str().case_compare(orig_table_name)) { ddl_succ = false; } else if (0 == hidden_table_schema->get_table_name_str().case_compare(orig_table_name)) { ddl_succ = true; } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected orig and hidden table name", K(ret), K(orig_table_schema->get_table_name_str()), K(hidden_table_schema->get_table_name_str()), K(orig_table_name)); } } if (OB_SUCC(ret)) { if (ddl_succ) { if (OB_FAIL(new_orig_table_schema.assign(*hidden_table_schema)) || OB_FAIL(new_hidden_table_schema.assign(*orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } } else { if (OB_FAIL(new_orig_table_schema.assign(*orig_table_schema)) || OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } } } if (OB_SUCC(ret)) { // if the previous step fails, the newly created hidden table should be deleted. // if the previous steps are successful, the original table is changed to a hidden table // so regardless of success or failure, only need to delete the hidden table new_hidden_table_schema.set_association_table_id(OB_INVALID_ID); new_hidden_table_schema.set_in_offline_ddl_white_list(true); if (!ddl_succ) { if (OB_FAIL(unbind_hidden_tablets(*orig_table_schema, *hidden_table_schema, OB_INVALID_VERSION, alter_table_arg.session_id_, trans))) { LOG_WARN("failed to unbind hidden tablets", K(ret)); } } else { // previous step succeeds, no need to unbind } if (OB_FAIL(ret)) { } else if (OB_FAIL(drop_table_in_trans(schema_guard, new_hidden_table_schema, false, new_hidden_table_schema.is_index_table(), to_recyclebin, NULL, &trans, NULL, NULL))) { LOG_WARN("failed to drop table in trans", K(ret)); } else { // update table state flag ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; new_orig_table_schema.set_association_table_id(OB_INVALID_ID); new_orig_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL); new_orig_table_schema.set_in_offline_ddl_white_list(true); if (OB_FAIL(ddl_operator.update_table_attribute( new_orig_table_schema, trans, operation_type))) { LOG_WARN("failed to update data table schema attribute", K(ret)); } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { ret = tmp_ret; } if (OB_NO_NEED_UPDATE == ret) { ret = OB_SUCCESS; } return ret; } int ObDDLService::truncate_table(const ObTruncateTableArg &arg, const obrpc::ObDDLRes &ddl_res, const int64_t frozen_version) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(arg.session_id_); ObSchemaService *schema_service = NULL; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); ObDDLSQLTransaction trans(schema_service_); uint64_t tenant_id = arg.tenant_id_; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_INVALID_ID == tenant_id || arg.database_name_.empty() || arg.table_name_.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(arg), K(ret)); } if (OB_SUCC(ret)) { const ObTableSchema *orig_table_schema = NULL; bool is_index = false; bool is_db_in_recyclebin = false; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, arg.database_name_, arg.table_name_, is_index, orig_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(arg.database_name_), K(arg.table_name_)); } else if (NULL == orig_table_schema) { ret = OB_TABLE_NOT_EXIST; LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(arg.database_name_), to_cstring(arg.table_name_)); } else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id, orig_table_schema->get_database_id(), is_db_in_recyclebin))) { LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), K(*orig_table_schema)); } else if (orig_table_schema->is_in_recyclebin() || is_db_in_recyclebin) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("Can not truncate table in recyclebin", K(ret), K(arg), K(*orig_table_schema)); } else if(orig_table_schema->is_index_table() || orig_table_schema->is_aux_vp_table() || orig_table_schema->is_aux_lob_table()) { ret = OB_NOT_SUPPORTED; LOG_WARN("truncate table is not supported on index or aux vp table", K(ret)); } else if (!orig_table_schema->is_user_table() && !orig_table_schema->is_tmp_table()) { if (orig_table_schema->is_sys_table()) { ret = OB_NOT_SUPPORTED; LOG_WARN("truncate table is not supported on system table", K(ret)); } else { ret = OB_TABLE_NOT_EXIST; LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(arg.database_name_), to_cstring(arg.table_name_)); } } else if (OB_FAIL(check_enable_sys_table_ddl(*orig_table_schema, OB_DDL_TRUNCATE_TABLE_CREATE))) { LOG_WARN("ddl is not allowed on system table", K(ret)); } else if (orig_table_schema->is_materialized_view() || orig_table_schema->has_materialized_view()) { ret = OB_NOT_SUPPORTED; LOG_WARN("truncate materialized view or table which has materialized view is not supported", K(ret), K(*orig_table_schema)); } else if (!orig_table_schema->check_can_do_ddl()) { ret = OB_NOT_SUPPORTED; LOG_WARN("offline ddl is being executed, other ddl operations are not allowed", K(orig_table_schema), K(ret)); } else { // else-start // materialized checking bool has_mv = false; if (OB_FAIL(check_table_has_materialized_view(schema_guard, *orig_table_schema, has_mv))) { LOG_WARN("fail to check table has materialized view", K(ret), K(*orig_table_schema)); } else if (has_mv) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support trunate table has materialized view", K(ret)); } if (OB_SUCC(ret)) { // When truncate table, check whether the table being truncate is the parent table // of the foreign key constraint // if it is parent table, not allow truncate if (orig_table_schema->is_parent_table()) { char *err_desciption_buf = NULL; int64_t err_desciption_buf_len = OB_MAX_VARCHAR_LENGTH; int64_t pos = 0; // Check if it is self-referential if (orig_table_schema->is_child_table()) { const ObIArray &foreign_key_infos = orig_table_schema->get_foreign_key_infos(); FOREACH_CNT_X(foreign_key_info, foreign_key_infos, OB_SUCC(ret)) { if (orig_table_schema->get_table_id() != foreign_key_info->child_table_id_) { // Not self-referencing, truncate is not allowed // Need to set the error code to OB_ERR_TRUNCATE_ILLEGAL_FK if (OB_ISNULL(err_desciption_buf = static_cast(allocator.alloc(err_desciption_buf_len)))) { ret = OB_ALLOCATE_MEMORY_FAILED; SERVER_LOG(ERROR, "fail to alloc table_def_buf", K(ret)); } else if(OB_FAIL(fill_truncate_table_fk_err_msg(*foreign_key_info, schema_guard, *orig_table_schema, err_desciption_buf, err_desciption_buf_len, pos))) { LOG_WARN("Cannot print err def of truncate table with foreign keys", K(ret), K(orig_table_schema->get_table_name_str())); } else { ret = OB_ERR_TRUNCATE_ILLEGAL_FK; ObString truncate_table_err_desciption(static_cast(pos), static_cast(pos), err_desciption_buf); LOG_USER_ERROR(OB_ERR_TRUNCATE_ILLEGAL_FK, truncate_table_err_desciption.length(), truncate_table_err_desciption.ptr()); } } else { } // do-nothing This table is only its own parent table in self-reference, and can be truncate } } else { // Need to set the error code to OB_ERR_TRUNCATE_ILLEGAL_FK const ObIArray &foreign_key_infos = orig_table_schema->get_foreign_key_infos(); FOREACH_CNT_X(foreign_key_info, foreign_key_infos, OB_SUCC(ret)) { if (orig_table_schema->get_table_id() == foreign_key_info->parent_table_id_) { if (OB_UNLIKELY(NULL == (err_desciption_buf = static_cast(allocator.alloc(err_desciption_buf_len))))) { ret = OB_ALLOCATE_MEMORY_FAILED; SERVER_LOG(ERROR, "fail to alloc table_def_buf", K(ret)); } else if (OB_FAIL(fill_truncate_table_fk_err_msg(*foreign_key_info, schema_guard, *orig_table_schema, err_desciption_buf, err_desciption_buf_len, pos))) { LOG_WARN("Cannot print err def of truncate table with foreign keys", K(ret), K(orig_table_schema->get_table_name_str())); } else { ret = OB_ERR_TRUNCATE_ILLEGAL_FK; ObString truncate_table_err_desciption(static_cast(pos), static_cast(pos), err_desciption_buf); LOG_USER_ERROR(OB_ERR_TRUNCATE_ILLEGAL_FK, truncate_table_err_desciption.length(), truncate_table_err_desciption.ptr()); } } } } } } // lock table when drop data table if (OB_FAIL(ret)) { } else if (OB_FAIL(lock_table(trans, *orig_table_schema))) { LOG_WARN("fail to lock_table", KR(ret), KPC(orig_table_schema)); // for ddl retry task, upper layer only focus on `OB_TRY_LOCK_ROW_CONFLICT`, and then retry it. const bool is_ddl_scheduled_task = arg.task_id_ > 0 ? true : false; ret = is_ddl_scheduled_task && ObDDLUtil::is_table_lock_retry_ret_code(ret) ? OB_TRY_LOCK_ROW_CONFLICT : ret; } if (OB_SUCC(ret) && orig_table_schema->is_oracle_tmp_table()) { ret = OB_NOT_SUPPORTED; LOG_WARN("truncate temp table not supported on RS", K(ret)); } else if (OB_SUCC(ret)) { ObSArray table_schemas; ObSArray index_recycle_objs; uint64_t new_table_id = OB_INVALID_ID; ObTableSchema new_table_schema; if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(schema_service->fetch_new_table_id(tenant_id, new_table_id))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else { new_table_schema.set_table_id(new_table_id); // reset auto increment after truncate table new_table_schema.set_auto_increment(1); if (orig_table_schema->is_primary_vp_table()) { // After truncate table, the new table is still a VP table new_table_schema.set_data_table_id(new_table_id); } if (OB_FAIL(table_schemas.push_back(new_table_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } //reconstruct index schema if (OB_SUCC(ret)) { ObSEArray simple_index_infos; if (OB_FAIL(orig_table_schema->get_simple_index_infos( simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema( tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", simple_index_infos.at(i).table_id_, K(ret)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else { ObString index_name; ObString new_index_table_name; ObTableSchema new_index_schema; if (OB_FAIL(new_index_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(ObTableSchema::get_index_name(allocator, orig_table_schema->get_table_id(), index_table_schema->get_table_name_str(), index_name))) { const ObString &index_table_name = index_table_schema->get_table_name_str(); uint64_t orig_tid = orig_table_schema->get_table_id(); LOG_WARN("error get index table name failed", K(orig_tid), K(index_table_name), K(ret)); } else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator, new_table_id, index_name, new_index_table_name))) { LOG_WARN("failed to build new index table name!", K(new_table_id), K(new_index_table_name), K(ret)); } else { uint64_t new_idx_tid = OB_INVALID_ID; if (OB_FAIL(schema_service->fetch_new_table_id(tenant_id, new_idx_tid))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else { new_index_schema.set_table_id(new_idx_tid); new_index_schema.set_data_table_id(new_table_id); // The index table generated by truncate is empty, set directly to AVAILABLE new_index_schema.set_index_status(INDEX_STATUS_AVAILABLE); if (new_index_schema.is_in_recyclebin()) { new_index_schema.set_table_name(index_table_schema->get_table_name_str()); ObArray recycle_objs; ObRecycleObject::RecycleObjType recycle_type = ObRecycleObject::get_type_by_table_schema(new_index_schema); if (OB_INVALID_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant_id is invalid", K(ret)); } else if (OB_FAIL(schema_service->fetch_recycle_object( tenant_id, new_index_schema.get_table_name_str(), recycle_type, trans, recycle_objs))) { LOG_WARN("get recycle object failed", K(tenant_id), K(ret)); } else if (recycle_objs.size() != 1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected recycle object num", K(ret), "table_name", new_index_schema.get_table_name_str(), "size", recycle_objs.size()); } else { ObRecycleObject &recycle_obj = recycle_objs.at(0); if (OB_FAIL(index_recycle_objs.push_back(recycle_obj))) { LOG_WARN("fail to push back recyclebin object", KR(ret), K(recycle_obj)); } } } else { new_index_schema.set_table_name(new_index_table_name); } if (OB_FAIL(table_schemas.push_back(new_index_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } } } } //end for } if (OB_SUCC(ret) && new_table_schema.has_lob_column()) { ObLobMetaBuilder lob_meta_builder(*this); ObLobPieceBuilder lob_data_builder(*this); ObTableSchema lob_meta_schema; ObTableSchema lob_piece_schema; if (OB_FAIL(lob_meta_builder.generate_aux_lob_meta_schema(schema_service, new_table_schema, lob_meta_schema, false))) { LOG_WARN("generate_schema for lob meta table failed", K(new_table_schema), K(ret)); } else if (OB_FAIL(table_schemas.push_back(lob_meta_schema))) { LOG_WARN("push_back lob meta table failed", K(ret)); } else if (OB_FAIL(lob_data_builder.generate_aux_lob_piece_schema(schema_service, new_table_schema, lob_piece_schema, false))) { LOG_WARN("generate_schema for lob data table failed", K(new_table_schema), K(ret)); } else if (OB_FAIL(table_schemas.push_back(lob_piece_schema))) { LOG_WARN("push_back lob data table failed", K(ret)); } else { table_schemas.at(0).set_aux_lob_meta_tid(lob_meta_schema.get_table_id()); table_schemas.at(0).set_aux_lob_piece_tid(lob_piece_schema.get_table_id()); } } if (FAILEDx(generate_object_id_for_partition_schemas(table_schemas))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(table_schemas)); } else if (OB_FAIL(generate_tables_tablet_id(table_schemas))) { LOG_WARN("failed to generate_tables_id", KR(ret), K(table_schemas)); } // rebuild VP table if (OB_SUCC(ret) && orig_table_schema->is_primary_vp_table()) { ObSEArray aux_vp_tid_array; if (OB_FAIL(orig_table_schema->get_aux_vp_tid_array(aux_vp_tid_array))) { LOG_WARN("get_aux_tid_array failed", K(ret)); } int64_t N = aux_vp_tid_array.count(); for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { const ObTableSchema *aux_vp_table_schema = NULL; uint64_t new_aux_vp_tid = OB_INVALID_ID; if (OB_FAIL(schema_guard.get_table_schema( tenant_id, aux_vp_tid_array.at(i), aux_vp_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", aux_vp_tid_array.at(i), K(ret)); } else if (OB_ISNULL(aux_vp_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (OB_FAIL(schema_service->fetch_new_table_id( tenant_id, new_aux_vp_tid))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else { ObVertialPartitionBuilder vp_builder(*this); const int64_t buf_size = 64; char buf[buf_size]; MEMSET(buf, 0, buf_size); int64_t pos = 0; if (OB_FAIL(vp_builder.generate_vp_table_name(new_aux_vp_tid, buf, buf_size, pos))) { LOG_WARN("failed to generate_vp_table_name", K(ret), K(new_table_id)); } else { ObString aux_vp_table_name(pos, buf); ObTableSchema new_aux_vp_schema; if (OB_FAIL(new_aux_vp_schema.assign(*aux_vp_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else { new_aux_vp_schema.set_table_id(new_aux_vp_tid); new_aux_vp_schema.set_table_name(aux_vp_table_name); new_aux_vp_schema.set_data_table_id(new_table_id); if (OB_FAIL(table_schemas.push_back(new_aux_vp_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } } } } } //check table if (OB_SUCC(ret)) { if (OB_FAIL(truncate_table_in_trans(arg, *orig_table_schema, table_schemas, index_recycle_objs, schema_guard, trans, &arg.ddl_stmt_str_, arg.database_name_))) { LOG_WARN("truncate table in trans failed", K(ret)); } else if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", KR(ret), K(tenant_id)); } } } } // else-end } } allocator.clear(); return ret; } // description: when error code is OB_ERR_TRUNCATE_ILLEGAL_FK, print error // Because there are very detailed error messages in mysql mode when truncate a parent table with foreign key constraints, // so in order to be compatible with mysql, the following interface is used to generate error messages // // @param [in] foreign_key_info ObForeignKeyInfo // @param [in] schema_guard ObSchemaGetterGuard // @param [in] buf print buff // @param [in] buf_len OB_MAX_VARCHAR_LENGTH // @param [in] pos eror info last byte offset // // @return oceanbase error code defined in lib/ob_errno.def int ObDDLService::fill_truncate_table_fk_err_msg(const ObForeignKeyInfo &foreign_key_info, ObSchemaGetterGuard &schema_guard, const ObTableSchema &parent_table_schema, char *buf, const int64_t &buf_len, int64_t &pos) const { int ret = OB_SUCCESS; ObSchemaPrinter schema_printer(schema_guard); const ObDatabaseSchema *parent_db_schema = NULL; const ObDatabaseSchema *child_db_schema = NULL; const ObTableSchema *child_table_schema = NULL; const uint64_t tenant_id = parent_table_schema.get_tenant_id(); if (OB_FAIL(schema_guard.get_table_schema(tenant_id, foreign_key_info.child_table_id_, child_table_schema))) { LOG_WARN("fail to get child table schema", K(ret), K(tenant_id), K(foreign_key_info.child_table_id_)); } else if (NULL == child_table_schema) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unknow table", K(ret), K(foreign_key_info.parent_table_id_)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, child_table_schema->get_database_id(), child_db_schema))) { LOG_WARN("failed to get database", K(ret), K(tenant_id), K(child_table_schema->get_database_id())); } else if (NULL == child_db_schema) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unknow database", K(ret), K(child_table_schema->get_database_id())); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "(`%s`.`%s`, ", child_db_schema->get_database_name(), child_table_schema->get_table_name()))) { // print "(`child_db_name`.`child_table_name`, " LOG_WARN("fail to print database and table name", K(ret), K(child_db_schema->get_database_name_str()), K(child_table_schema->get_table_name_str())); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "CONSTRAINT "))) { // print "CONSTRAINT " LOG_WARN("fail to print CONSTRAINT", K(ret)); } else if (!foreign_key_info.foreign_key_name_.empty() && OB_FAIL(databuff_printf(buf, buf_len, pos, "`%.*s` ", foreign_key_info.foreign_key_name_.length(), foreign_key_info.foreign_key_name_.ptr()))) { // print "`constraint_name` " LOG_WARN("fail to print foreign key name", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "FOREIGN KEY ("))) { // print "FOREIGN KEY (" LOG_WARN("fail to print FOREIGN KEY (", K(ret)); } else if (OB_FAIL(schema_printer.print_column_list(*child_table_schema, foreign_key_info.child_column_ids_, buf, buf_len, pos))) { // print "child table foreign key column list", eg: "`c1`, `c2`" LOG_WARN("fail to print_column_list", K(ret), K(child_table_schema->get_table_name_str())); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, ") REFERENCES "))) { // print ") REFERENCES " LOG_WARN("fail to print ) REFERENCES ", K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, parent_table_schema.get_database_id(), parent_db_schema))) { SHARE_SCHEMA_LOG(WARN, "failed to get database", K(ret), K(tenant_id), K(parent_table_schema.get_database_id())); } else if(OB_FAIL(databuff_printf(buf, buf_len, pos, "`%s`.`%s` (", parent_db_schema->get_database_name(), parent_table_schema.get_table_name()))) { // print "`child_db_name`.`child_table_name`" LOG_WARN("fail to print parent_database_name.parent_table_name", K(ret)); } else if (OB_FAIL(schema_printer.print_column_list(parent_table_schema, foreign_key_info.parent_column_ids_, buf, buf_len, pos))) { // print "parent table foreign key column list" and "))", eg: "`c1`, `c2`))" LOG_WARN("fail to print_column_list", K(ret), K(parent_table_schema.get_table_name_str())); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "))"))) { LOG_WARN("fail to print ) ", K(ret)); } return ret; } //the first schema in new_schemas is data table schema //other is the index table schema //only used create table like int ObDDLService::rebuild_table_schema_with_new_id(const ObTableSchema &orig_table_schema, const ObDatabaseSchema &new_database_schema, const ObString &new_table_name, const ObString &create_host, const int64_t session_id, const share::schema::ObTableType table_type_, ObSchemaService &schema_service, const uint64_t frozen_version, ObIArray &new_schemas, ObArenaAllocator &allocator, const uint64_t define_user_id) { int ret = OB_SUCCESS; uint64_t new_table_id = OB_INVALID_ID; ObSchemaGetterGuard schema_guard; const uint64_t tenant_id = orig_table_schema.get_tenant_id(); ObTableSchema new_table_schema; bool is_oracle_mode = false; if (OB_FAIL(new_table_schema.assign(orig_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(schema_service.fetch_new_table_id(orig_table_schema.get_tenant_id(), new_table_id))) { LOG_WARN("failed to fetch new table id", K(ret), K(orig_table_schema.get_tenant_id())); } else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("failed to check_if_oracle_compat_mode", K(ret), K(orig_table_schema.get_tenant_id())); } else { new_table_schema.set_table_id(new_table_id); new_table_schema.set_table_name(new_table_name); new_table_schema.set_database_id(new_database_schema.get_database_id()); new_table_schema.set_define_user_id(define_user_id); new_table_schema.reset_foreign_key_infos(); new_table_schema.reset_trigger_list(); if (new_table_schema.has_constraint()) { // reset check constraint name in mysql mode ObTableSchema::const_constraint_iterator iter = new_table_schema.constraint_begin(); ObTableSchema::const_constraint_iterator iter_last = iter; ObString new_constraint_name; bool is_constraint_name_exist = false; for (; OB_SUCC(ret) && iter != new_table_schema.constraint_end();++iter) { (*iter)->set_table_id(new_table_id); (*iter)->set_tenant_id(tenant_id); do { if (OB_FAIL(ObTableSchema::create_cons_name_automatically( new_constraint_name, new_table_name, allocator, (*iter)->get_constraint_type(), is_oracle_mode))) { SQL_RESV_LOG(WARN, "create cons name automatically failed", K(ret)); } else if (OB_UNLIKELY(0 == new_constraint_name.case_compare((*iter_last)->get_constraint_name_str()))) { is_constraint_name_exist = true; } else if (OB_FAIL(check_constraint_name_is_exist( schema_guard, new_table_schema, new_constraint_name, false, is_constraint_name_exist))) { LOG_WARN("fail to check check constraint name is exist or not", K(ret), K(new_constraint_name)); } } while (OB_SUCC(ret) && is_constraint_name_exist); if (OB_SUCC(ret)) { (*iter)->set_constraint_name(new_constraint_name); } iter_last = iter; } } if (orig_table_schema.is_sys_table() || orig_table_schema.is_vir_table()) { new_table_schema.set_table_type(USER_TABLE); } else if (orig_table_schema.is_sys_view()) { new_table_schema.set_table_type(USER_VIEW); } if (new_table_schema.is_user_table() && (TMP_TABLE == table_type_ || TMP_TABLE_ORA_SESS == table_type_)) { new_table_schema.set_table_type(table_type_); new_table_schema.set_create_host(create_host); new_table_schema.set_sess_active_time(ObTimeUtility::current_time()); new_table_schema.set_session_id(session_id); } if (orig_table_schema.is_primary_vp_table()) { new_table_schema.set_data_table_id(0); // VP not support } if (OB_FAIL(ret)) { } else if (OB_FAIL(new_schemas.push_back(new_table_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } if (OB_SUCC(ret)) { //reconstruct index schema ObSEArray simple_index_infos; if (OB_FAIL(new_table_schema.get_simple_index_infos( simple_index_infos))) { LOG_WARN("get simple_index_infos failed", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) { const ObTableSchema *index_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) { LOG_WARN("get_table_schema failed", K(ret), K(tenant_id), "table id", simple_index_infos.at(i).table_id_); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (index_table_schema->is_in_recyclebin()) { continue; } else { ObString index_name; ObString new_index_table_name; HEAP_VAR(ObTableSchema, new_index_schema) { if (OB_FAIL(new_index_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); } else if (OB_FAIL(ObTableSchema::get_index_name(allocator, orig_table_schema.get_table_id(), index_table_schema->get_table_name_str(), index_name))) { const ObString &index_table_name = index_table_schema->get_table_name_str(); uint64_t orig_tid = orig_table_schema.get_table_id(); LOG_WARN("error get index table name failed", K(orig_tid), K(index_table_name), K(ret)); } else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator, new_table_id, index_name, new_index_table_name))) { LOG_WARN("failed to build new index table name!", K(new_table_id), K(new_index_table_name), K(ret)); } else { uint64_t new_idx_tid = OB_INVALID_ID; if (OB_FAIL(schema_service.fetch_new_table_id(orig_table_schema.get_tenant_id(), new_idx_tid))) { LOG_WARN("failed to fetch_new_table_id", K(ret)); } else { new_index_schema.set_table_id(new_idx_tid); new_index_schema.set_data_table_id(new_table_id); new_index_schema.set_table_name(new_index_table_name); new_index_schema.set_database_id(new_database_schema.get_database_id()); //create table like, index always is valid new_index_schema.set_index_status(INDEX_STATUS_AVAILABLE); if (OB_FAIL(new_schemas.push_back(new_index_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } } } } } //end for if (OB_SUCC(ret) && new_table_schema.has_lob_column()) { ObLobMetaBuilder lob_meta_builder(*this); ObLobPieceBuilder lob_data_builder(*this); HEAP_VARS_2((ObTableSchema, lob_meta_schema), (ObTableSchema, lob_piece_schema)) { if (OB_FAIL(lob_meta_builder.generate_aux_lob_meta_schema(&schema_service, new_table_schema, lob_meta_schema, false))) { LOG_WARN("generate_schema for lob meta table failed", K(new_table_schema), K(ret)); } else if (OB_FAIL(new_schemas.push_back(lob_meta_schema))) { LOG_WARN("push_back lob meta table failed", K(ret)); } else if (OB_FAIL(lob_data_builder.generate_aux_lob_piece_schema(&schema_service, new_table_schema, lob_piece_schema, false))) { LOG_WARN("generate_schema for lob data table failed", K(new_table_schema), K(ret)); } else if (OB_FAIL(new_schemas.push_back(lob_piece_schema))) { LOG_WARN("push_back lob data table failed", K(ret)); } else { new_schemas.at(0).set_aux_lob_meta_tid(lob_meta_schema.get_table_id()); new_schemas.at(0).set_aux_lob_piece_tid(lob_piece_schema.get_table_id()); } } } } #if 0 if (OB_SUCC(ret) && (orig_table_schema.is_primary_vp_table())) { //reconstruct VP info // only need to update all the vertical partition tables // including the data_table_id of the main table itself. uint64_t aux_vp_tid_array[OB_MAX_PARTITION_NUM]; int64_t aux_vp_cnt = OB_MAX_PARTITION_NUM; if (OB_FAIL(orig_table_schema.get_aux_vp_tid_array(aux_vp_tid_array, aux_vp_cnt))) { LOG_WARN("get_aux_vp_tid_array failed", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < aux_vp_cnt; ++i) { const ObTableSchema *aux_vp_table_schema = NULL; if (OB_FAIL(schema_guard.get_table_schema(aux_vp_tid_array[i], aux_vp_table_schema))) { LOG_WARN("get_table_schema failed", "table id", aux_vp_tid_array[i], K(ret)); } else if (OB_ISNULL(aux_vp_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (aux_vp_table_schema->is_in_recyclebin()) { continue; } else { ObTableSchema new_aux_vp_schema = *aux_vp_table_schema; new_aux_vp_schema.set_data_table_id(new_table_id); if (OB_FAIL(new_scheams.push_back(new_aux_vp_schema))) { LOG_WARN("failed to add table schema!", K(ret)); } } } } #endif return ret; } int ObDDLService::create_table_like(const ObCreateTableLikeArg &arg, const int64_t frozen_version) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; ObSchemaService *schema_service = NULL; bool object_exist = false; uint64_t synonym_id = OB_INVALID_ID; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); uint64_t tenant_id = arg.tenant_id_; schema_guard.set_session_id(arg.session_id_); ObArray mock_fk_parent_table_schema_array; if (arg.sequence_ddl_arg_.get_stmt_type() != common::OB_INVALID_ID) { ret = OB_ERR_UNEXPECTED; LOG_WARN("create table like not suppotted identity column", K(ret)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_WARN("schema_service must not null", K(ret)); } else if (OB_INVALID_ID == tenant_id || arg.origin_db_name_.empty() || arg.origin_table_name_.empty() || arg.new_db_name_.empty() || arg.new_table_name_.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(arg), K(ret)); } else { const ObTableSchema *orig_table_schema = NULL; const ObTableSchema *new_table_schema = NULL; const ObDatabaseSchema *new_db_schema = NULL; bool is_db_in_recyclebin = false; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, arg.origin_db_name_, arg.origin_table_name_, false, orig_table_schema))) { LOG_WARN("get_table_schema failed", K(arg), K(tenant_id)); } else if (NULL == orig_table_schema) { ret = OB_TABLE_NOT_EXIST; LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(arg.origin_db_name_), to_cstring(arg.origin_table_name_)); } else if (OB_FAIL(schema_guard.check_database_in_recyclebin( tenant_id, orig_table_schema->get_database_id(), is_db_in_recyclebin))) { LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), K(*orig_table_schema)); } else if (orig_table_schema->is_in_recyclebin() || is_db_in_recyclebin) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can create table like table in recyclebin", K(ret), K(*orig_table_schema), K(is_db_in_recyclebin)); } else if (!orig_table_schema->is_user_table() && !orig_table_schema->is_sys_table()) { ret = OB_ERR_WRONG_OBJECT; LOG_USER_ERROR(OB_ERR_WRONG_OBJECT, to_cstring(arg.origin_db_name_), to_cstring(arg.origin_table_name_), "BASE TABLE"); } else if (is_inner_table(orig_table_schema->get_table_id())) { // tablegroup of system table is oceanbase, // Including the user table in it may cause some unexpected problems, please ban it here // https://work.aone.alibaba-inc.com/issue/22213436 ret = OB_ERR_WRONG_OBJECT; LOG_USER_ERROR(OB_ERR_WRONG_OBJECT, to_cstring(arg.origin_db_name_), to_cstring(arg.origin_table_name_), "BASE TABLE"); LOG_WARN("create table like inner table not allowed", K(ret), K(arg)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, arg.new_db_name_, arg.new_table_name_, false, new_table_schema))) { } else if (NULL != new_table_schema) { ret = OB_ERR_TABLE_EXIST; LOG_WARN("target table already exist", K(arg), K(tenant_id), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, arg.new_db_name_, new_db_schema))) { } else if (NULL == new_db_schema) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, arg.new_db_name_.length(), arg.new_db_name_.ptr()); LOG_WARN("database not exist", K(arg), K(ret)); } else if (new_db_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not create table in recyclebin", K(ret), K(*new_db_schema)); } else if (OB_FAIL(schema_guard.check_synonym_exist_with_name(tenant_id, new_db_schema->get_database_id(), arg.new_table_name_, object_exist, synonym_id))) { LOG_WARN("fail to check synonym exist", K(arg), K(ret)); } else if (object_exist) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(arg), K(ret)); } if (OB_SUCC(ret)) { ObSArray table_schemas; share::schema::ObErrorInfo error_info; if (OB_FAIL(rebuild_table_schema_with_new_id(*orig_table_schema, *new_db_schema, arg.new_table_name_, arg.create_host_, arg.session_id_, arg.table_type_, *schema_service, frozen_version, table_schemas, allocator, arg.define_user_id_))) { LOG_WARN("failed to rebuild table schema with new id", KR(ret)); } else if (OB_FAIL(generate_object_id_for_partition_schemas(table_schemas))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(table_schemas)); } else if (OB_FAIL(generate_tables_tablet_id(table_schemas))) { LOG_WARN("failed to generate_tables_id", KR(ret), K(table_schemas)); } if (OB_SUCC(ret)) { // deal with mock parent table const ObMockFKParentTableSchema *ori_mock_parent_table_schema_ptr = NULL; if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name( new_db_schema->get_tenant_id(), new_db_schema->get_database_id(), arg.new_table_name_, ori_mock_parent_table_schema_ptr))) { SQL_RESV_LOG(WARN, "failed to check_mock_fk_parent_table_exist_with_name", K(ret)); } else if (OB_NOT_NULL(ori_mock_parent_table_schema_ptr)) { // will replace mock_parent_table with this new table ObMockFKParentTableSchema mock_fk_parent_table_schema; ObArray index_schemas; for (int64_t i = 1; OB_SUCC(ret) && i < table_schemas.count(); ++i) { if (table_schemas.at(i).is_unique_index() && OB_FAIL(index_schemas.push_back(&table_schemas.at(i)))) { LOG_WARN("failed to push back index_schemas", K(ret)); } } if (FAILEDx(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table( schema_guard, ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id(), table_schemas.at(0), index_schemas, mock_fk_parent_table_schema))) { LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table", K(ret), K(ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id())); } else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) { LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema)); } } } if (FAILEDx(create_user_tables(arg.if_not_exist_, arg.ddl_stmt_str_, error_info, table_schemas, frozen_version, schema_guard, arg.sequence_ddl_arg_, 0, NULL, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to create user tables"); } } } } allocator.clear(); return ret; } // Delete index information and write to system tables in a transaction. // If sql_trans is NULL, you need to create a transaction inside the function int ObDDLService::drop_table_in_trans( ObSchemaGetterGuard &schema_guard, const ObTableSchema &table_schema, const bool is_rebuild_index, const bool is_index, const bool to_recyclebin, const ObString *ddl_stmt_str, ObMySQLTransaction *sql_trans, DropTableIdHashSet *drop_table_set, ObMockFKParentTableSchema *mock_fk_parent_table_ptr /* will use it when drop a fk_parent_table */) { int ret = OB_SUCCESS; UNUSED(is_index); if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", KR(ret)); } //do drop table in trans if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction tmp_trans(schema_service_); ObMySQLTransaction &trans = OB_ISNULL(sql_trans) ? tmp_trans : *sql_trans; const uint64_t tenant_id = table_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_ISNULL(sql_trans) && OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (!table_schema.is_aux_table()) { if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard, table_schema, USER_INDEX, to_recyclebin)))) { LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret)); } else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard, table_schema, AUX_VERTIAL_PARTITION_TABLE, to_recyclebin)))) { LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret)); } else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard, table_schema, AUX_LOB_META, to_recyclebin)))) { LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret)); } else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard, table_schema, AUX_LOB_PIECE, to_recyclebin)))) { LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret)); } else if (OB_FAIL(drop_trigger_in_drop_table(trans, ddl_operator, schema_guard, table_schema, to_recyclebin))) { LOG_WARN("drop_trigger_in_drop_table failed", KR(ret)); } } // delete error info added when create force view if (OB_SUCC(ret) && table_schema.is_user_view()) { bool is_oracle_mode = false; if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check if tenant mode is oracle mode", K(ret)); } else if (is_oracle_mode) { ObErrorInfo error_info; error_info.set_obj_id(table_schema.get_table_id()); error_info.set_obj_type(static_cast(ObObjectType::VIEW)); error_info.set_database_id(table_schema.get_database_id()); error_info.set_tenant_id(table_schema.get_tenant_id()); error_info.set_schema_version(table_schema.get_schema_version()); error_info.set_error_status(ERROR_STATUS_NO_ERROR);// set as no error to delete exists error if (OB_FAIL(error_info.handle_error_info(trans, NULL))) { LOG_WARN("insert create error info failed.", K(ret)); } } } if (OB_SUCC(ret)) { if (to_recyclebin && !table_schema.is_index_table() && !is_inner_table(table_schema.get_table_id()) && !table_schema.is_aux_lob_table() && !table_schema.is_aux_vp_table()) { //index/aux_vp/aux_lob table and inner table will drop directly if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(table_schema, schema_guard, trans, ddl_stmt_str))) { LOG_WARN("drop table to recyclebin failed", KR(ret)); } } else { if (!is_rebuild_index && table_schema.is_index_table()) { ObTableSchema tmp_schema; tmp_schema.reset(); if (OB_FAIL(ddl_operator.drop_inner_generated_index_column( trans, schema_guard, table_schema, tmp_schema))) { LOG_WARN("drop inner generated index column failed", KR(ret)); } } if (OB_SUCC(ret) && OB_FAIL(ddl_operator.drop_table( table_schema, trans, ddl_stmt_str, false/*is_truncate_table*/, drop_table_set, false))) { LOG_WARN("ddl_operator drop_table failed", K(table_schema), KR(ret)); } } } // deal with mock_fk_parent_table in drop_table begin ObArray mock_fk_parent_table_schema_array; // If table dropped is child table which has mock parent tables, will gen_mock_fk_parent_tables_for_drop_fks if (OB_SUCC(ret) && table_schema.get_foreign_key_real_count() > 0) { const ObIArray &foreign_key_infos = table_schema.get_foreign_key_infos(); ObArray parent_table_mock_foreign_key_info_array; ObArray mock_fk_parent_table_schema_ptr_array; for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); if (foreign_key_info.is_parent_table_mock_) { // TODO:@xiaofeng.lby, delete this restriction, https://yuque.antfin-inc.com/ob/product_functionality_review/si89mc if (OB_NOT_NULL(drop_table_set)) { if (drop_table_set->count() > 1) { ret = OB_NOT_SUPPORTED; LOG_WARN("drop muti tables with mock fks in one sql is not supported ", K(ret)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(parent_table_mock_foreign_key_info_array.push_back(&foreign_key_info))) { LOG_WARN("fail to push back to parent_table_mock_foreign_key_info_array", K(ret), K(foreign_key_info)); } } } if (OB_SUCC(ret) && !parent_table_mock_foreign_key_info_array.empty()) { if (OB_FAIL(prepare_gen_mock_fk_parent_tables_for_drop_fks( schema_guard, table_schema.get_tenant_id(), parent_table_mock_foreign_key_info_array, mock_fk_parent_table_schema_ptr_array, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to prepare_gen_mock_fk_parent_tables_for_drop_fks", K(ret)); } else if (!mock_fk_parent_table_schema_array.empty()){ if (OB_FAIL(gen_mock_fk_parent_tables_for_drop_fks( schema_guard, mock_fk_parent_table_schema_ptr_array, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to gen_mock_fk_parent_tables_for_drop_fks", K(ret)); } } } } // If table dropped is real parent table, create mock fk parent table if (OB_SUCC(ret) && OB_NOT_NULL(mock_fk_parent_table_ptr)) { int64_t new_schema_version = OB_INVALID_VERSION; if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema version", K(ret), K(tenant_id)); } else if (FALSE_IT(mock_fk_parent_table_ptr->set_schema_version(new_schema_version))) { } else if (FALSE_IT(mock_fk_parent_table_ptr->set_operation_type(MOCK_FK_PARENT_TABLE_OP_CREATE_TABLE_BY_DROP_PARENT_TABLE))) { } else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(*mock_fk_parent_table_ptr))) { LOG_WARN("fail to push mock_fk_parent_table_schema_array", K(ret), KPC(mock_fk_parent_table_ptr)); } } if (OB_SUCC(ret) && !mock_fk_parent_table_schema_array.empty()) { if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) { LOG_WARN("failed to deal_with_mock_fk_parent_table", K(ret), K(mock_fk_parent_table_schema_array)); } } // deal with mock_fk_parent_table_schema in drop_table end if (OB_ISNULL(sql_trans) && trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } return ret; } int ObDDLService::drop_aux_table_in_drop_table( ObMySQLTransaction &trans, ObDDLOperator &ddl_operator, share::schema::ObSchemaGetterGuard &schema_guard, const ObTableSchema &table_schema, ObTableType table_type, const bool to_recyclebin) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_schema.get_tenant_id(); ObSEArray simple_index_infos; ObSEArray aux_tid_array; // for aux_vp or aux_lob bool is_index = false; if (USER_INDEX == table_type) { is_index = true; if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get_simple_index_infos failed", K(ret)); } } else if (AUX_LOB_META == table_type) { const uint64_t aux_lob_meta_tid = table_schema.get_aux_lob_meta_tid(); if (OB_INVALID_ID != aux_lob_meta_tid && OB_FAIL(aux_tid_array.push_back(aux_lob_meta_tid))) { LOG_WARN("push back aux_lob_meta_tid failed", K(ret)); } } else if (AUX_LOB_PIECE == table_type) { const uint64_t aux_lob_piece_tid = table_schema.get_aux_lob_piece_tid(); if (OB_INVALID_ID != aux_lob_piece_tid && OB_FAIL(aux_tid_array.push_back(aux_lob_piece_tid))) { LOG_WARN("push back aux_lob_meta_tid failed", K(ret)); } } else if (AUX_VERTIAL_PARTITION_TABLE == table_type) { if (OB_FAIL(table_schema.get_aux_vp_tid_array(aux_tid_array))) { LOG_WARN("get_aux_vp_tid_array failed", K(ret)); } } else { ret = OB_INVALID_ARGUMENT; LOG_WARN("invaid aux table type", K(ret), K(table_type)); } ObTableSchema new_table_schema; int64_t N = is_index ? simple_index_infos.count() : aux_tid_array.count(); for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { const ObTableSchema *aux_table_schema = NULL; uint64_t tid = is_index ? simple_index_infos.at(i).table_id_ : aux_tid_array.at(i); if (OB_FAIL(schema_guard.get_table_schema(tenant_id, tid, aux_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", tid, K(ret)); } else if (OB_ISNULL(aux_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (OB_FAIL(new_table_schema.assign(*aux_table_schema))) { LOG_WARN("assign table schema failed", K(ret)); } else { // If the data table of the delayed index table is placed in the recycle bin, // the delayed index will also go in, and a row of data will be inserted into __all_recyclebin new_table_schema.set_in_offline_ddl_white_list(table_schema.get_in_offline_ddl_white_list()); if (to_recyclebin && !is_inner_table(table_schema.get_table_id())) { if (new_table_schema.is_in_recyclebin()) { LOG_INFO("aux table is already in recyclebin"); } else if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(new_table_schema, schema_guard, trans, NULL /* ddl_stmt_str */))) { LOG_WARN("drop aux table to recycle failed", K(ret)); } } else if (OB_FAIL(ddl_operator.drop_table(new_table_schema, trans))) { LOG_WARN("ddl_operator drop_table failed", K(*aux_table_schema), K(ret)); } } } return ret; } int ObDDLService::drop_trigger_in_drop_table(ObMySQLTransaction &trans, ObDDLOperator &ddl_operator, ObSchemaGetterGuard &schema_guard, const ObTableSchema &table_schema, const bool drop_to_recyclebin) { int ret = OB_SUCCESS; uint64_t trigger_id = OB_INVALID_ID; const ObTriggerInfo *trigger_info = NULL; const uint64_t tenant_id = table_schema.get_tenant_id(); const ObIArray &trigger_id_list = table_schema.get_trigger_list(); for (int64_t i = 0; OB_SUCC(ret) && i < trigger_id_list.count(); i++) { OX (trigger_id = trigger_id_list.at(i)); OZ (schema_guard.get_trigger_info(tenant_id, trigger_id, trigger_info), trigger_id); OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_id); OV (!trigger_info->is_in_recyclebin(), OB_ERR_UNEXPECTED, trigger_id); if (drop_to_recyclebin && !table_schema.is_view_table()) { // 兼容oracle, drop view的时候trigger不进回收站 OZ (ddl_operator.drop_trigger_to_recyclebin(*trigger_info, schema_guard, trans)); } else { OZ (ddl_operator.drop_trigger(*trigger_info, trans, NULL, true /*is_update_table_schema_version, default true*/, table_schema.get_in_offline_ddl_white_list())); } } return ret; } int ObDDLService::flashback_table_from_recyclebin_in_trans(const ObTableSchema &table_schema, const uint64_t new_db_id, const ObString &new_table_name, const ObString &ddl_stmt_str, ObSchemaGetterGuard &guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); const uint64_t tenant_id = table_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_FAIL(guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator, new_db_id, USER_INDEX))) { LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema)); } else if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator, new_db_id, AUX_VERTIAL_PARTITION_TABLE))) { LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema)); } else if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator, new_db_id, AUX_LOB_META))) { LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema)); } else if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator, new_db_id, AUX_LOB_PIECE))) { LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema)); } else if (OB_FAIL(flashback_trigger(table_schema, new_db_id, new_table_name, guard, trans, ddl_operator))) { LOG_WARN("flashback_trigger failed", K(ret), K(table_schema)); } if (OB_SUCC(ret)) { ObTableSchema new_table_schema; if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin(table_schema, new_table_schema, trans, new_db_id, new_table_name, &ddl_stmt_str, guard))) { LOG_WARN("flashback table failed", K(ret)); } else { lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID; if (OB_FAIL(guard.get_tenant_compat_mode(tenant_id, compat_mode))) { LOG_WARN("fail to get tenant compat mode", K(ret), K(tenant_id), K(compat_mode)); } else if (lib::Worker::CompatMode::MYSQL == compat_mode) { const ObMockFKParentTableSchema *ori_mock_parent_table_schema_ptr = NULL; if (OB_FAIL(guard.get_mock_fk_parent_table_schema_with_name(tenant_id, new_table_schema.get_database_id(), new_table_schema.get_table_name_str(), ori_mock_parent_table_schema_ptr))) { SQL_RESV_LOG(WARN, "failed to check_mock_fk_parent_table_exist_with_name", K(ret)); } else if (OB_NOT_NULL(ori_mock_parent_table_schema_ptr)) { ObArray simple_index_infos; ObArray unique_index_schemas; if (OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) { SERVER_LOG(WARN, "get simple_index_infos without delay_deleted_tid failed", K(ret)); } else { const ObTableSchema *index_schema = NULL; for (int64_t j = 0; OB_SUCC(ret) && j < simple_index_infos.count(); ++j) { if (OB_FAIL(guard.get_table_schema(ori_mock_parent_table_schema_ptr->get_tenant_id(), simple_index_infos.at(j).table_id_, index_schema))) { LOG_WARN("fail to get to_table_schema schema", K(ret)); } else if (OB_ISNULL(index_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index_schema is null", K(ret)); } else if (index_schema->is_unique_index() && OB_FAIL(unique_index_schemas.push_back(index_schema))) { LOG_WARN("fail to push_back index_schema to unique_index_schemas", K(ret)); } } if (OB_SUCC(ret)) { ObArray mock_fk_parent_table_schema_array; ObMockFKParentTableSchema mock_fk_parent_table_schema; if (OB_FAIL(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table( guard, ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id(), new_table_schema, unique_index_schemas, mock_fk_parent_table_schema))) { LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table", K(ret)); } else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) { LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema)); } else if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, guard, mock_fk_parent_table_schema_array))) { LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(tenant_id), K(mock_fk_parent_table_schema_array)); } } } } } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } return ret; } int ObDDLService::flashback_aux_table( const ObTableSchema &table_schema, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, ObDDLOperator &ddl_operator, const uint64_t new_db_id, const ObTableType table_type) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_schema.get_tenant_id(); ObSEArray aux_vp_tid_array; ObSEArray simple_index_infos; uint64_t lob_meta_table_id; uint64_t lob_piece_table_id; int64_t N = 0; if (table_type == USER_INDEX) { if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) { LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type)); } else { N = simple_index_infos.count(); } } else if (table_type == AUX_VERTIAL_PARTITION_TABLE) { if (OB_FAIL(table_schema.get_aux_vp_tid_array(aux_vp_tid_array))) { LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type)); } else { N = aux_vp_tid_array.count(); } } else if (table_type == AUX_LOB_META) { lob_meta_table_id = table_schema.get_aux_lob_meta_tid(); N = table_schema.has_lob_column() ? 1 : 0; } else if (table_type == AUX_LOB_PIECE) { lob_piece_table_id = table_schema.get_aux_lob_piece_tid(); N = table_schema.has_lob_column() ? 1 : 0; } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("Invalid table type.", K(ret), K(table_type)); } for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { const ObTableSchema *aux_table_schema = NULL; ObTableSchema new_table_schema; uint64_t tid = 0; if (table_type == USER_INDEX) { tid = simple_index_infos.at(i).table_id_; } else if (table_type == AUX_VERTIAL_PARTITION_TABLE) { tid = aux_vp_tid_array.at(i); } else if (table_type == AUX_LOB_META) { tid = lob_meta_table_id; } else if (table_type == AUX_LOB_PIECE) { tid = lob_piece_table_id; } if (OB_FAIL(schema_guard.get_table_schema(tenant_id, tid, aux_table_schema))) { LOG_WARN("get_table_schema failed", K(tenant_id), "table id", tid, K(ret)); } else if (OB_ISNULL(aux_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (aux_table_schema->is_drop_index()) { // Temporarily keep the drop index into the recycle bin code ret = OB_SUCCESS; LOG_INFO("index table is dropped, can't flashback", K(ret)); } else if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin( *aux_table_schema, new_table_schema, trans, new_db_id, //set empty, because ensure that the index table flashback, use your own name ObString(), NULL, //ddl_stmt_str schema_guard))) { LOG_WARN("flashback table failed", K(ret)); } } return ret; } int ObDDLService::flashback_trigger(const ObTableSchema &table_schema, const uint64_t new_database_id, const ObString &new_table_name, ObSchemaGetterGuard &schema_guard, ObMySQLTransaction &trans, ObDDLOperator &ddl_operator) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_schema.get_tenant_id(); const ObIArray &trigger_id_list = table_schema.get_trigger_list(); const ObTriggerInfo *trigger_info = NULL; for (int i = 0; OB_SUCC(ret) && i < trigger_id_list.count(); i++) { uint64_t trigger_id = trigger_id_list.at(i); OZ (schema_guard.get_trigger_info(tenant_id, trigger_id, trigger_info), trigger_id); OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_id); OZ (ddl_operator.flashback_trigger(*trigger_info, new_database_id, new_table_name, schema_guard, trans)); } return ret; } int ObDDLService::flashback_table_to_time_point(const obrpc::ObFlashBackTableToScnArg &arg) { UNUSEDx(arg); return OB_NOT_SUPPORTED; } // check whether org db of object name and arg.origin_db_name_ is same int ObDDLService::check_object_name_matches_db_name( const uint64_t tenant_id, const ObString &origin_table_name, const uint64_t database_id, bool &is_match) { int ret = OB_SUCCESS; ObSqlString sql; HEAP_VAR(ObMySQLProxy::MySQLResult, res) { common::sqlclient::ObMySQLResult *result = NULL; is_match = false; if (OB_FAIL(sql.append_fmt( "SELECT 1 FROM __all_recyclebin WHERE OBJECT_NAME = '%.*s' \ AND DATABASE_ID = %lu", static_cast(origin_table_name.length()), origin_table_name.ptr(), database_id))) { LOG_WARN("failed to append sql", K(ret), K(tenant_id), K(origin_table_name), K(database_id)); } else if (OB_FAIL(sql_proxy_->read(res, tenant_id, sql.ptr()))) { LOG_WARN("failed to execute sql", K(sql), K(ret)); } else if (OB_ISNULL(result = res.get_result())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get result", K(ret)); } else if (OB_FAIL(result->next())) { if (OB_ITER_END == ret) { ret = OB_SUCCESS; is_match = false; } else { LOG_WARN("iterate next result fail", K(ret), K(sql)); } } else { is_match = true; } } return ret; } int ObDDLService::get_object_name_with_origin_name_in_recyclebin( const uint64_t tenant_id, const ObString &origin_table_name, const uint64_t database_id, const ObRecycleObject::RecycleObjType recycle_type, ObString &object_name, const bool is_newest, common::ObIAllocator *allocator, common::ObMySQLProxy *sql_proxy) { int ret = OB_SUCCESS; ObSqlString sql; HEAP_VAR(ObMySQLProxy::MySQLResult, res) { common::sqlclient::ObMySQLResult *result = NULL; if (ObRecycleObject::TABLE != recycle_type && ObRecycleObject::INDEX != recycle_type) { ret = OB_ERR_UNEXPECTED; LOG_WARN("recycle object type is wrong", K(ret), K(recycle_type)); } else if (ObRecycleObject::TABLE == recycle_type && OB_FAIL(sql.append_fmt("select database_id, object_name from oceanbase.__all_recyclebin where (type = %lu or type = %lu) and database_id = %lu and original_name = '%.*s' ", static_cast(ObRecycleObject::TABLE), static_cast(ObRecycleObject::VIEW), database_id, static_cast(origin_table_name.length()), origin_table_name.ptr()))) { LOG_WARN("failed to append sql", K(ret), K(tenant_id), K(origin_table_name), K(database_id), K(is_newest), K(sql)); } else if (ObRecycleObject::INDEX == recycle_type && OB_FAIL(sql.append_fmt("select database_id, object_name from oceanbase.__all_recyclebin where type = %lu and database_id = %lu and substr(original_name, 7 + instr(substr(original_name, 7), '_')) = '%.*s' ", static_cast(ObRecycleObject::INDEX), database_id, static_cast(origin_table_name.length()), origin_table_name.ptr()))) { LOG_WARN("failed to append sql", K(ret), K(tenant_id), K(origin_table_name), K(database_id), K(is_newest), K(sql)); } else if (OB_FAIL(sql.append_fmt("order by gmt_create %s limit 1", is_newest ? "desc" : "asc"))) { LOG_WARN("failed to append sql", K(ret), K(tenant_id), K(origin_table_name), K(database_id), K(is_newest), K(sql)); } else if (OB_FAIL(sql_proxy->read(res, tenant_id, sql.ptr()))) { LOG_WARN("failed to execute sql", K(sql), K(ret)); } else if (OB_ISNULL(result = res.get_result())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get result", K(ret)); } else if (OB_FAIL(result->next())) { if (OB_ITER_END == ret) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("origin table_name not exist in recyclebin", K(ret), K(sql)); } else { LOG_WARN("iterate next result fail", K(ret), K(sql)); } } else { ObString tmp_object_name; EXTRACT_VARCHAR_FIELD_MYSQL(*result, "object_name", tmp_object_name); if (OB_FAIL(deep_copy_ob_string(*allocator, tmp_object_name, object_name))) { LOG_WARN("failed to deep copy member list", K(ret), K(object_name)); } } if (OB_FAIL(ret)) { } else if (OB_ITER_END != result->next()) { // No more than one line ret = OB_ERR_UNEXPECTED; LOG_WARN("result failed", K(ret), K(sql)); } } return ret; } int ObDDLService::flashback_table_from_recyclebin(const ObFlashBackTableFromRecyclebinArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTableSchema *table_schema = NULL; const uint64_t tenant_id = arg.tenant_id_; uint64_t database_id = OB_INVALID_ID; bool is_match = false; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_database_id(tenant_id, arg.origin_db_name_, database_id))) { LOG_WARN("get database id with database name failed", K(ret)); } else if (OB_INVALID_ID == database_id) { ret = OB_ERR_BAD_DATABASE; LOG_WARN("database does not exist", K(ret), K(arg.origin_db_name_)); LOG_USER_ERROR(OB_ERR_BAD_DATABASE, arg.origin_db_name_.length(), arg.origin_db_name_.ptr()); } else if (OB_FAIL(schema_guard.get_table_schema( tenant_id, OB_RECYCLEBIN_SCHEMA_ID, arg.origin_table_name_, false, /* is_index */ table_schema))) { LOG_WARN("get_table_schema failed", K(ret)); } else if (OB_ISNULL(table_schema)) { // use origin table name to flashback ObString object_name; if (OB_FAIL(get_object_name_with_origin_name_in_recyclebin( tenant_id, arg.origin_table_name_, database_id, ObRecycleObject::TABLE, object_name, true, &allocator, sql_proxy_))) { LOG_WARN("get object_name with origin_name in recyclebin failed", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema( tenant_id, OB_RECYCLEBIN_SCHEMA_ID, object_name, false, /*is_index*/ table_schema))) { LOG_WARN("get_table_schema failed", K(ret)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_shema is null", K(ret), K(database_id), K(arg.origin_table_name_), K(object_name)); } } else if (OB_FAIL(check_object_name_matches_db_name(tenant_id, arg.origin_table_name_, database_id, is_match))) { // use object name to flashback LOG_WARN("fail to check object name matches db name", K(ret)); } else if (!is_match) { // The specified object name does not exist under the specified db ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("origin table_name not exist in recyclebin", K(ret), K(arg.origin_table_name_), K(database_id)); } if (OB_SUCC(ret)) { uint64_t new_db_id = OB_INVALID_ID; uint64_t synonym_id = OB_INVALID_ID; lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID; if (OB_FAIL(schema_guard.get_tenant_compat_mode(arg.tenant_id_, compat_mode))) { LOG_WARN("fail to get tenant compat mode", K(ret), K(tenant_id), K(compat_mode)); } else if (!arg.new_db_name_.empty() && !arg.new_table_name_.empty()) { bool is_table_exist = false; bool object_exist = false; const ObDatabaseSchema *new_db_schema = NULL; if (ObString(OB_RECYCLEBIN_SCHEMA_NAME) == arg.new_db_name_ || ObString(OB_PUBLIC_SCHEMA_NAME) == arg.new_db_name_) { ret = OB_OP_NOT_ALLOW; LOG_WARN("flashback table to __recyclebin database is not allowed", K(arg), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(arg.tenant_id_, arg.new_db_name_, new_db_schema))) { LOG_WARN("check database exist failed", K(ret), K(arg)); } else if (OB_ISNULL(new_db_schema)) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, arg.new_db_name_.length(), arg.new_db_name_.ptr()); } else if (new_db_schema->is_in_recyclebin()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("flashback table to recyclebin db is not allowed", K(ret), K(arg)); } if (OB_SUCC(ret) && lib::Worker::CompatMode::ORACLE == compat_mode) { ObArray conflict_schema_types; if (OB_FAIL(schema_guard.check_oracle_object_exist(arg.tenant_id_, new_db_schema->get_database_id(), arg.new_table_name_, OB_MAX_SCHEMA, INVALID_ROUTINE_TYPE, false, conflict_schema_types))) { LOG_WARN("fail to check oracle_object exist", K(ret), K(table_schema)); } else if (conflict_schema_types.count() > 0) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(ret), K(table_schema)); } else { new_db_id = new_db_schema->get_database_id(); } } if (OB_SUCC(ret) && lib::Worker::CompatMode::MYSQL == compat_mode) { if (OB_FAIL(schema_guard.check_synonym_exist_with_name(arg.tenant_id_, new_db_schema->get_database_id(), arg.new_table_name_, object_exist, synonym_id))) { LOG_WARN("fail to check synonym exist", K(arg), K(ret)); } else if (object_exist) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(arg), K(ret)); } else if (OB_FAIL(schema_guard.check_table_exist(arg.tenant_id_, new_db_schema->get_database_id(), arg.new_table_name_, table_schema->is_index_table(), ObSchemaGetterGuard::ALL_NON_HIDDEN_TYPES, is_table_exist))) { LOG_WARN("check table exist failed,", K(ret), K(arg), K(new_db_id)); } else if (is_table_exist) { ret = OB_ERR_TABLE_EXIST; LOG_USER_ERROR(OB_ERR_TABLE_EXIST, arg.new_table_name_.length(), arg.new_table_name_.ptr()); LOG_WARN("talbe exist", K(new_db_id), K(arg), K(ret)); } else { new_db_id = new_db_schema->get_database_id(); } } } else { if (OB_SUCC(ret) && lib::Worker::CompatMode::ORACLE == compat_mode) { ObArray conflict_schema_types; if (OB_FAIL(schema_guard.check_oracle_object_exist(arg.tenant_id_, database_id, arg.new_table_name_.empty() ? arg.origin_table_name_ : arg.new_table_name_, OB_MAX_SCHEMA, INVALID_ROUTINE_TYPE, false, conflict_schema_types))) { LOG_WARN("fail to check oracle_object exist", K(ret), K(table_schema)); } else if (conflict_schema_types.count() > 0) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(ret), K(table_schema)); } } } if (OB_SUCC(ret)) { if (table_schema->is_index_table() || table_schema->is_aux_vp_table() || table_schema->is_aux_lob_table()) { ret = OB_NOT_SUPPORTED; LOG_WARN("flash back index table is not supported now", K(ret)); } else if (OB_FAIL(flashback_table_from_recyclebin_in_trans(*table_schema, new_db_id, arg.new_table_name_, arg.ddl_stmt_str_, schema_guard))) { LOG_WARN("flashback table from recyclebin in trans failed", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } } return ret; } int ObDDLService::flashback_index(const ObFlashBackIndexArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTableSchema *index_schema = NULL; const ObTableSchema *data_table_schema = NULL; const uint64_t tenant_id = arg.tenant_id_; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, OB_RECYCLEBIN_SCHEMA_ID, arg.origin_table_name_, true, /*is_index*/ index_schema))) { LOG_WARN("get_table_schema failed", K(ret)); } else if (OB_ISNULL(index_schema)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("table schema is null", K(arg), K(ret)); } else if (!index_schema->is_index_table()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("the table is not index, flashback index failed", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema( tenant_id, index_schema->get_data_table_id(), data_table_schema))) { LOG_WARN("get data table schema failed", K(ret)); } else if (OB_ISNULL(data_table_schema)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("data table schema is null", K(ret)); } else if (data_table_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("the data table is in recyclebin", K(ret)); } if (OB_SUCC(ret)) { uint64_t synonym_id = OB_INVALID_ID; if (!arg.new_table_name_.empty()) { bool is_table_exist = false; bool object_exist = false; const ObDatabaseSchema *database_schema = NULL; if (OB_FAIL(schema_guard.get_database_schema(tenant_id, data_table_schema->get_database_id(), database_schema))) { LOG_WARN("get database schema failed", K(ret), K(tenant_id)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database schema is null", K(ret)); } else if (ObString(OB_RECYCLEBIN_SCHEMA_NAME) == database_schema->get_database_name() || ObString(OB_PUBLIC_SCHEMA_NAME) == database_schema->get_database_name()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("flashback table to __recyclebin database is not allowed", K(ret), K(*database_schema)); } else if (OB_FAIL(schema_guard.check_synonym_exist_with_name(arg.tenant_id_, data_table_schema->get_database_id(), arg.new_table_name_, object_exist, synonym_id))) { LOG_WARN("fail to check synonym exist", K(arg), K(ret)); } else if (object_exist) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(arg), K(ret)); } else if (OB_FAIL(schema_guard.check_table_exist(arg.tenant_id_, data_table_schema->get_database_id(), arg.new_table_name_, index_schema->is_index_table(), ObSchemaGetterGuard::ALL_NON_HIDDEN_TYPES, is_table_exist))) { LOG_WARN("check table exist failed,", K(ret), K(arg), K(data_table_schema->get_database_id())); } else if (is_table_exist) { ret = OB_ERR_TABLE_EXIST; LOG_USER_ERROR(OB_ERR_TABLE_EXIST, arg.new_table_name_.length(), arg.new_table_name_.ptr()); LOG_WARN("talbe exist", K(data_table_schema->get_database_id()), K(arg), K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(flashback_index_in_trans(schema_guard, *index_schema, data_table_schema->get_database_id(), arg.new_table_name_, arg.ddl_stmt_str_))) { LOG_WARN("falshback index in trans failed", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } } return ret; } int ObDDLService::flashback_index_in_trans(share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObTableSchema &table_schema, const uint64_t new_db_id, const common::ObString &new_table_name, const common::ObString &ddl_stmt_str) { int ret = OB_SUCCESS; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); ObString new_index_table_name; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); const uint64_t tenant_id = table_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator, table_schema.get_data_table_id(), new_table_name, new_index_table_name))) { LOG_WARN("build index table name failed", K(ret)); } else { ObTableSchema new_table_schema; if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin(table_schema, new_table_schema, trans, new_db_id, new_index_table_name, &ddl_stmt_str, schema_guard))) { LOG_WARN("flashback table in trans failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } return ret; } int ObDDLService::purge_index(const obrpc::ObPurgeIndexArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTableSchema *table_schema = NULL; const uint64_t tenant_id = arg.tenant_id_; uint64_t database_id = arg.database_id_; bool is_match = false; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, OB_RECYCLEBIN_SCHEMA_ID, arg.table_name_, true, /*is_index*/ table_schema))) { LOG_WARN("get_table_schema failed", K(ret)); } else if (NULL == table_schema) { // try to purge with original table name ObString object_name; if (OB_FAIL(get_object_name_with_origin_name_in_recyclebin(tenant_id, arg.table_name_, database_id, ObRecycleObject::INDEX, object_name, false /* the oldest one */, &allocator, sql_proxy_))) { LOG_WARN("get object_name with origin_name in recyclebin failed", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema( tenant_id, OB_RECYCLEBIN_SCHEMA_ID, object_name, true, /*is_index*/ table_schema))) { LOG_WARN("get_table_schema failed", K(ret)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_shema is null", K(ret), K(database_id), K(arg.table_name_), K(object_name)); } } else if (OB_FAIL(check_object_name_matches_db_name(tenant_id, arg.table_name_, database_id, is_match))) { // purge with object name LOG_WARN("fail to check object name matches db name", K(ret)); } else if (!is_match) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("table is not in recyclebin", K(ret), K(arg)); } if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.purge_table_in_recyclebin( *table_schema, trans, &arg.ddl_stmt_str_))) { LOG_WARN("purge index failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } } return ret; } int ObDDLService::purge_table( const ObPurgeTableArg &arg, ObMySQLTransaction *pr_trans) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTableSchema *table_schema = NULL; const uint64_t tenant_id = arg.tenant_id_; uint64_t database_id = arg.database_id_; bool is_match = false; ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP); if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, OB_RECYCLEBIN_SCHEMA_ID, arg.table_name_, false, /*is_index*/ table_schema))) { LOG_WARN("get_table_schema failed", K(ret)); } else if (NULL == table_schema) { // try to purge with original table name ObString object_name; if (OB_FAIL(get_object_name_with_origin_name_in_recyclebin(tenant_id, arg.table_name_, database_id, ObRecycleObject::TABLE, object_name, false /* the oldest one */, &allocator, sql_proxy_))) { LOG_WARN("get object_name with origin_name in recyclebin failed", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema( tenant_id, OB_RECYCLEBIN_SCHEMA_ID, object_name, false, /*is_index */ table_schema))) { LOG_WARN("get_table_schema failed", K(ret)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_shema is null", K(ret), K(database_id), K(arg.table_name_), K(object_name)); } } else if (OB_FAIL(check_object_name_matches_db_name(tenant_id, arg.table_name_, database_id, is_match))) { // purge with object name LOG_WARN("fail to check object name matches db name", K(ret)); } else if (!is_match) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("table is not in recyclebin", K(ret), K(arg)); } if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_ISNULL(pr_trans) && OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.purge_table_with_aux_table(*table_schema, schema_guard, OB_ISNULL(pr_trans) ? trans : *pr_trans, &arg.ddl_stmt_str_))) { LOG_WARN("purge table with aux table failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_ISNULL(pr_trans)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } } } return ret; } int ObDDLService::flashback_database_in_trans(const ObDatabaseSchema &db_schema, const ObString &new_db_name, ObSchemaGetterGuard &schema_guard, const ObString &ddl_stmt_str) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); const uint64_t tenant_id = db_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.flashback_database_from_recyclebin(db_schema, trans, new_db_name, schema_guard, ddl_stmt_str))) { LOG_WARN("flashback database from recyclebin failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } return ret; } int ObDDLService::flashback_database(const ObFlashBackDatabaseArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObDatabaseSchema *database_schema = NULL; const uint64_t tenant_id = arg.tenant_id_; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, arg.origin_db_name_, database_schema))) { LOG_WARN("get_database_schema failed", K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("database schema is not in recyclebin", K(ret), K(arg)); } else if (!database_schema->is_in_recyclebin()) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("database schema is not in recyclebin", K(ret), K(arg), K(*database_schema)); } else { ObString new_db_name; if (!arg.new_db_name_.empty()) { bool is_db_exist = false; if (OB_FAIL(schema_guard.check_database_exist(arg.tenant_id_, arg.new_db_name_, is_db_exist))) { LOG_WARN("check database exist failed", K(ret), K(arg)); } else if (is_db_exist) { ret = OB_DATABASE_EXIST; LOG_USER_ERROR(OB_DATABASE_EXIST, arg.new_db_name_.length(), arg.new_db_name_.ptr()); LOG_WARN("new db name already exist", K(arg), K(ret)); } else { new_db_name = arg.new_db_name_; } } if (OB_SUCC(ret)) { if (OB_FAIL(flashback_database_in_trans(*database_schema, new_db_name, schema_guard, arg.ddl_stmt_str_))) { LOG_WARN("flashback table in trans failed", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } } return ret; } int ObDDLService::purge_database( const ObPurgeDatabaseArg &arg, ObMySQLTransaction *pr_trans) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObDatabaseSchema *database_schema = NULL; const uint64_t tenant_id = arg.tenant_id_; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, arg.db_name_, database_schema))) { LOG_WARN("get_database_schema failed", K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("database not in recyclebin, can not be purge", K(arg), K(ret)); } else if (!database_schema->is_in_recyclebin()) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("database not in recyclebin, can not be purge", K(arg), K(*database_schema), K(ret)); } else {/*do nothing*/} if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_ISNULL(pr_trans) && OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.purge_database_in_recyclebin(*database_schema, OB_ISNULL(pr_trans) ? trans : *pr_trans, schema_guard, &arg.ddl_stmt_str_))) { LOG_WARN("purge database failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_ISNULL(pr_trans)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } } } return ret; } /* purge the objects in the recycle bin, the current recycle bin object types * include database, table(view), index, trigger * Each round shares a transaction, each round of objects are purged, and then publish schema in: ObPurgeRecycleBinArg, include purge tenant ID and timestamp, Objects in the recycle bin earlier than the timestamp are purged out: purged_objects, The number of objects that rpc successfully purges each time, the default one is DEFAULT_PURGE_EACH_TIME (10) */ int ObDDLService::purge_tenant_expire_recycle_objects(const ObPurgeRecycleBinArg &arg, int64_t &purged_objects) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObArray recycle_objs; if (OB_FAIL(ddl_operator.fetch_expire_recycle_objects(arg.tenant_id_, arg.expire_time_, recycle_objs))) { LOG_WARN("fetch expire recycle objects failed", K(ret), K(arg)); } else { LOG_INFO("purge expire recycle object of tenant start", K(arg), K(recycle_objs.size()), K(ret)); if (recycle_objs.size() == 0) { } else if (OB_FAIL(purge_recyclebin_except_tenant(arg, recycle_objs, purged_objects))) { LOG_WARN("fail to purge recyclebin except tenant", K(ret)); } else if (!arg.auto_purge_ && OB_FAIL(purge_recyclebin_tenant(arg, recycle_objs, purged_objects))) { LOG_WARN("fail to purge recyclebin tenant", K(ret)); } } } return ret; } int ObDDLService::purge_recyclebin_except_tenant( const ObPurgeRecycleBinArg &arg, const ObIArray &recycle_objs, int64_t &purged_objects) { int ret = OB_SUCCESS; LOG_INFO("start purge recyclebin execpt tenant", K(arg)); const uint64_t tenant_id = arg.tenant_id_; purged_objects = 0; ObSqlString ddl_stmt; ObSchemaGetterGuard schema_guard; ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (recycle_objs.count() == 0) { } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table( tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { for (int i = 0; OB_SUCC(ret) && i < recycle_objs.count() && purged_objects < arg.purge_num_; ++i) { ddl_stmt.reset(); const ObRecycleObject &recycle_obj = recycle_objs.at(i); switch(recycle_obj.get_type()) { case ObRecycleObject::TRIGGER: case ObRecycleObject::INDEX: case ObRecycleObject::AUX_LOB_META: case ObRecycleObject::AUX_LOB_PIECE: case ObRecycleObject::AUX_VP: { // Indexes/VP/AUX_LOB tables do not enter the recycle bin separately, // and the purge table will synchronize purge index/aux_vp/aux_lob, // so there is no need for a separate purge index trigger as above continue; } case ObRecycleObject::VIEW: case ObRecycleObject::TABLE: { //purge table if (OB_FAIL(ddl_stmt.assign_fmt("PURGE TABLE %.*s", recycle_obj.get_object_name().length(), recycle_obj.get_object_name().ptr()))) { LOG_WARN("append sql failed", K(ret)); } else { ObPurgeTableArg purge_table_arg; purge_table_arg.tenant_id_ = arg.tenant_id_; purge_table_arg.database_id_ = recycle_obj.get_database_id(); purge_table_arg.table_name_ = recycle_obj.get_object_name(); purge_table_arg.ddl_stmt_str_ = ddl_stmt.string(); if (OB_FAIL(purge_table(purge_table_arg, &trans))) { if (OB_ERR_OBJECT_NOT_IN_RECYCLEBIN == ret) { LOG_WARN("recycle object maybe purge by database", K(ret), K(recycle_obj)); ret = OB_SUCCESS; } else { LOG_WARN("purge table failed", K(purge_table_arg), K(recycle_obj), K(ret)); } } else { ++purged_objects; } } break; } case ObRecycleObject::DATABASE: { if (OB_FAIL(ddl_stmt.assign_fmt("PURGE DATABASE %.*s", recycle_obj.get_object_name().length(), recycle_obj.get_object_name().ptr()))) { LOG_WARN("append sql failed", K(ret)); } else { ObPurgeDatabaseArg purge_database_arg; purge_database_arg.tenant_id_ = arg.tenant_id_; purge_database_arg.db_name_ = recycle_obj.get_object_name(); purge_database_arg.ddl_stmt_str_ = ddl_stmt.string(); if (OB_FAIL(purge_database(purge_database_arg, &trans))) { LOG_WARN("purge database failed", K(purge_database_arg), K(recycle_obj), K(ret)); } else { ++purged_objects; } } break; } case ObRecycleObject::TENANT: { //nothing todo. Unified processing in purge_recyclebin_tenant break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("unknown recycle object type", K(recycle_obj)); } } LOG_INFO("purge expire recycle object execpt tenant finished", K(recycle_obj), K(arg), K(ret)); } } LOG_INFO("purge expire recycle object of tenant finished", K(arg), K(ret)); if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret) && purged_objects > 0) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish_schema failed", K(ret)); } } return ret; } int ObDDLService::purge_recyclebin_tenant( const ObPurgeRecycleBinArg &arg, const ObIArray &recycle_objs, int64_t &purged_objects) { int ret = OB_SUCCESS; LOG_INFO("start purge recyclebin tenant", K(arg)); const uint64_t tenant_id = arg.tenant_id_; ObSqlString ddl_stmt; for (int i = 0; OB_SUCC(ret) && i < recycle_objs.count() && purged_objects < arg.purge_num_; ++i) { ddl_stmt.reset(); const ObRecycleObject &recycle_obj = recycle_objs.at(i); if (ObRecycleObject::TENANT == recycle_obj.get_type()) { bool is_standby = false; if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("fail to get", K(ret)); } else if (!is_standby) { if (tenant_id != OB_SYS_TENANT_ID) { ret = OB_ERR_UNEXPECTED; LOG_WARN("purge tenant only in sys tenant", K(ret)); } else if (OB_FAIL(ddl_stmt.assign_fmt("PURGE TENANT %.*s", recycle_obj.get_object_name().length(), recycle_obj.get_object_name().ptr()))) { LOG_WARN("append sql failed", K(ret)); } else { ObPurgeTenantArg purge_tenant_arg; purge_tenant_arg.tenant_id_ = OB_SYS_TENANT_ID; purge_tenant_arg.tenant_name_ = recycle_obj.get_object_name(); purge_tenant_arg.ddl_stmt_str_ = ddl_stmt.string(); if (OB_FAIL(purge_tenant(purge_tenant_arg))) { LOG_WARN("purge tenant failed", K(purge_tenant_arg), K(recycle_obj), K(ret)); } else { ++purged_objects; } } } else { // standalone cluster is not executed, but it should be counted normally ++purged_objects; } } } LOG_INFO("purge expire recycle object tenant finish", K(ret)); return ret; } /** * success on the table schema */ int ObDDLService::check_table_exists(const uint64_t tenant_id, const ObTableItem &table_item, const ObTableType expected_table_type, ObSchemaGetterGuard &guard, const ObTableSchema **table_schema) { int ret = OB_SUCCESS; //check database uint64_t database_id = OB_INVALID_ID; const ObTableSchema *tmp_table_schema = NULL; bool exist = false; if (OB_INVALID_ID == tenant_id || table_item.database_name_.empty() || table_item.table_name_.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(tenant_id), K(table_item), K(ret)); } else if (OB_FAIL(guard.check_database_exist(tenant_id, table_item.database_name_, exist, &database_id))) { LOG_WARN("failed to check database exist!", K(tenant_id),K(table_item), K(database_id), K(exist), K(ret)); } else if (!exist) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, table_item.database_name_.length(), table_item.database_name_.ptr()); LOG_WARN("database not exist!", K(table_item), K(tenant_id), K(ret)); } else if (OB_INVALID_ID == database_id) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, table_item.database_name_.length(), table_item.database_name_.ptr()); LOG_WARN("database id is invalid!", K(database_id), K(table_item), K(tenant_id), K(ret)); } //check table exist if (OB_SUCC(ret)) { bool is_view = false; uint64_t org_sess_id = guard.get_session_id(); if (USER_VIEW == expected_table_type || MATERIALIZED_VIEW == expected_table_type) { is_view = true; guard.set_session_id(0); } else { is_view = false; } if (OB_FAIL(guard.get_table_schema(tenant_id, database_id, table_item.table_name_, USER_INDEX == expected_table_type, tmp_table_schema, table_item.is_hidden_))) { LOG_WARN("get_table_schema failed", K(tenant_id), KT(database_id), K(table_item), K(expected_table_type), K(ret)); } else if (NULL == tmp_table_schema) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("not find this table schema:", K(ret), K(tenant_id), K(database_id), K(table_item)); } else { if (is_view) { guard.set_session_id(org_sess_id); } if (expected_table_type == tmp_table_schema->get_table_type() || (TMP_TABLE_ALL == expected_table_type && tmp_table_schema->is_tmp_table()) || (TMP_TABLE_ORA_SESS == expected_table_type && tmp_table_schema->is_oracle_trx_tmp_table())) { //ignore } else if (TMP_TABLE == expected_table_type) { if (!tmp_table_schema->is_tmp_table()) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("Table type not equal!", K(expected_table_type), K(table_item), K(*tmp_table_schema), K(ret)); } } else if (USER_TABLE == expected_table_type) { if (!tmp_table_schema->is_table() && !tmp_table_schema->is_tmp_table()) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("Table type not equal!", K(expected_table_type), K(table_item), K(ret)); } else { /*maybe SYS_TABLE or VIRTUAL TABLE */ } } else if (is_view) { if (SYSTEM_VIEW == tmp_table_schema->get_table_type()) { // let it go, for case compatible } else if (expected_table_type != tmp_table_schema->get_table_type()) { ret = OB_ERR_WRONG_OBJECT; LOG_USER_ERROR(OB_ERR_WRONG_OBJECT, to_cstring(table_item.database_name_), to_cstring(table_item.table_name_), "VIEW"); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unknow table type", K(expected_table_type), "table_type", tmp_table_schema->get_table_type()); } } } if (OB_SUCC(ret)) { if (table_schema != NULL) { *table_schema = tmp_table_schema; } } return ret; } int ObDDLService::construct_drop_sql(const ObTableItem &table_item, const ObTableType table_type, ObSqlString &sql, bool is_oracle_mode, bool is_cascade_constrains) { int ret = OB_SUCCESS; ObArenaAllocator allocator("TmpDropSql"); ObString table_type_name = table_type == USER_VIEW ? "VIEW" : "TABLE"; ObString new_db_name; ObString new_tbl_name; if (OB_FAIL(sql::ObSQLUtils::generate_new_name_with_escape_character( allocator, table_item.database_name_, new_db_name, is_oracle_mode))) { LOG_WARN("fail to generate new name with escape character", K(ret), K(table_item.database_name_)); } else if (OB_FAIL(sql::ObSQLUtils::generate_new_name_with_escape_character( allocator, table_item.table_name_, new_tbl_name, is_oracle_mode))) { LOG_WARN("fail to generate new name with escape character", K(ret), K(table_item.database_name_)); } else if (OB_FAIL(sql.append_fmt( is_oracle_mode ? "DROP %.*s \"%.*s\".\"%.*s\"" : "DROP %.*s `%.*s`.`%.*s`", table_type_name.length(), table_type_name.ptr(), new_db_name.length(), new_db_name.ptr(), new_tbl_name.length(), new_tbl_name.ptr()))) { LOG_WARN("failed to append sql", K(ret)); } else if (is_cascade_constrains && OB_FAIL(sql.append_fmt(" CASCADE CONSTRAINTS"))) { LOG_WARN("failed to append CASCADE CONSTRAINTS", K(ret)); } return ret; } /** * @table_item table to drop * @if_exists drop table if exists ... * @err_table_list use for if_exists == false */ int ObDDLService::log_drop_warn_or_err_msg(const ObTableItem table_item, bool if_exists, ObSqlString &err_table_list) { //if exists == true, log to warning //else log to err table list int ret = OB_SUCCESS; if (if_exists) { ObSqlString warning_str; if (OB_FAIL(warning_str.append_fmt("%.*s.%.*s", table_item.database_name_.length(), table_item.database_name_.ptr(), table_item.table_name_.length(), table_item.table_name_.ptr()))) { LOG_WARN("append warning str failed", K(ret), K(table_item), K(if_exists)); } else { LOG_USER_NOTE(OB_ERR_BAD_TABLE, static_cast(warning_str.length()), warning_str.ptr()); LOG_WARN("table not exist", K(table_item), K(ret)); } } else { if (OB_FAIL(err_table_list.append_fmt("%.*s.%.*s,", table_item.database_name_.length(), table_item.database_name_.ptr(), table_item.table_name_.length(), table_item.table_name_.ptr()))) { LOG_WARN("failed to append err table", K(ret)); } } return ret; } int ObDDLService::log_rebuild_warn_or_err_msg(const ObRebuildIndexArg &arg, ObSqlString &err_table_list) { int ret = OB_SUCCESS; if (OB_FAIL(err_table_list.append_fmt("%.*s.%.*s,", arg.database_name_.length(), arg.database_name_.ptr(), arg.table_name_.length(), arg.table_name_.ptr()))) { LOG_WARN("failed to append err table list!", K(ret)); } return ret; } // // check whether exist materialized view based on this tale // int ObDDLService::check_table_has_materialized_view( ObSchemaGetterGuard &schema_guard, const ObTableSchema &table_schema, bool &has_mv) { int ret = OB_SUCCESS; has_mv = false; ObArray mv_ids; const uint64_t tenant_id = table_schema.get_tenant_id(); if (OB_FAIL(schema_guard.get_tenant_mv_ids(tenant_id, mv_ids))) { LOG_WARN("fail to fetch all mv ids", K(ret), "vesion", table_schema.get_schema_version()); } else { for (int64_t i = 0; OB_SUCC(ret) && i < mv_ids.count(); i++) { const ObTableSchema *mv = NULL; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, mv_ids.at(i), mv))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(mv_ids.at(i))); } else if (OB_ISNULL(mv)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("mv is null", K(ret)); } else if (mv->has_table(table_schema.get_table_id())) { has_mv = true; break; } } } return ret; } // Check whether the oracle temporary table has data for the specified session id int ObDDLService::check_sessid_exist_in_temp_table(const ObString &db_name, const ObString &tab_name, const uint64_t tenant_id, const uint64_t session_id, bool &exists) { int ret = OB_SUCCESS; HEAP_VAR(common::ObMySQLProxy::MySQLResult, res) { common::sqlclient::ObMySQLResult *result = NULL; ObSqlString sql; exists = false; if (OB_FAIL(sql.assign_fmt("SELECT 1 FROM `%.*s`.`%.*s` WHERE %s = %ld LIMIT 1", db_name.length(), db_name.ptr(), tab_name.length(), tab_name.ptr(), OB_HIDDEN_SESSION_ID_COLUMN_NAME, session_id))) { LOG_WARN("assign sql failed", K(ret)); } else if (OB_FAIL(sql_proxy_->read(res, tenant_id, sql.ptr()))) { LOG_WARN("execute sql failed", K(ret), K(sql)); } else if (OB_ISNULL(result = res.get_result())) { ret = OB_ERR_UNEXPECTED; SERVER_LOG(WARN, "result set from read is NULL", K(ret)); } else if (OB_FAIL(result->next())) { if (OB_ITER_END == ret) { ret = OB_SUCCESS; /* empty table */ } else { LOG_WARN("get next result failed", K(ret)); } } else { exists = true; } LOG_INFO("check temp table has session id specified", K(exists), K(sql), K(ret)); } return ret; } // Whether the current temporary table needs to be cleaned up, // the session id can be the same under mysql mode // For oracle, it is necessary to make sure that T has the data of the session // or a table under the same tenant as T(session id should only be cleaned up for one tenant) int ObDDLService::need_collect_current_temp_table(ObSchemaGetterGuard &schema_guard, ObDropTableArg &drop_table_arg, const ObSimpleTableSchemaV2 *table_schema, bool &need_collect) { int ret = OB_SUCCESS; need_collect = false; if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (drop_table_arg.session_id_ == table_schema->get_session_id()) { need_collect = true; drop_table_arg.tenant_id_ = table_schema->get_tenant_id();//maybe it's alter system ... } else if ((TMP_TABLE_ORA_SESS == drop_table_arg.table_type_ && table_schema->is_oracle_tmp_table()) || (TMP_TABLE_ORA_TRX == drop_table_arg.table_type_ && table_schema->is_oracle_trx_tmp_table())) { if (drop_table_arg.tenant_id_ == table_schema->get_tenant_id()) { need_collect = true; } } else if (TMP_TABLE_ALL == drop_table_arg.table_type_ && table_schema->is_oracle_tmp_table()) { if (drop_table_arg.tenant_id_ == table_schema->get_tenant_id()) { need_collect = true; } else { const ObDatabaseSchema *database_schema = NULL; if (OB_FAIL(schema_guard.get_database_schema(table_schema->get_tenant_id(), table_schema->get_database_id(), database_schema))) { LOG_WARN("failed to get database schema", K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database schema is null", K(ret)); } else if (OB_FAIL(check_sessid_exist_in_temp_table(database_schema->get_database_name_str(), table_schema->get_table_name_str(), table_schema->get_tenant_id(), drop_table_arg.session_id_, need_collect))) { LOG_WARN("failed to check sessid exists", K(ret), K(table_schema->get_table_name_str())); } else if (need_collect) { drop_table_arg.tenant_id_ = table_schema->get_tenant_id(); drop_table_arg.table_type_ = TMP_TABLE_ORA_SESS; } } } return ret; } // The syntax for cleaning up temporary tables when the session is disconnected is similar to // DROP TABLES IN SESSION #1, so first collect all temporary tables created by SESSION #1 // The session requested by this DDL is still in a state of being disconnected // regardless of whether it is directly connected or obproxy. // The session id cannot be reused, so the situation of table_schema->session_id reuse is not considered // alter system drop tables in session 1234 is special, // it will delete all temporary tables under the specified session 1234(mysql & oracle) // FIXME: this function should move to observer int ObDDLService::collect_temporary_tables_in_session(const ObDropTableArg &const_drop_table_arg) { int ret = OB_SUCCESS; ObTableItem table_item; ObSchemaGetterGuard schema_guard; const ObDatabaseSchema *database_schema = NULL; ObSEArray table_schemas; ObDropTableArg &drop_table_arg = const_cast(const_drop_table_arg); ObArray tenant_ids; if (0 == drop_table_arg.session_id_) { //do nothing, in case of alter system drop tables in session 0 } else if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service is null", KR(ret)); } else if (OB_FAIL(schema_service_->get_tenant_ids(tenant_ids))) { LOG_WARN("fail to get tenant_ids", KR(ret)); } else { bool found = false; FOREACH_CNT_X(tenant_id, tenant_ids, OB_SUCC(ret) && !found) { if (OB_FAIL(schema_service_->get_tenant_schema_guard(*tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), "tenant_id", *tenant_id); } else if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(*tenant_id, table_schemas))) { LOG_WARN("fail to get table schema", K(ret), "tenant_id", *tenant_id); } else { for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); i++) { const ObSimpleTableSchemaV2 *table_schema = table_schemas.at(i); bool need_collect = false; if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("got invalid schema", K(ret), K(i)); } else if (OB_FAIL(need_collect_current_temp_table(schema_guard, drop_table_arg, table_schema, need_collect))) { LOG_WARN("check need collect temporary table failed", K(ret), K(table_schema->get_table_name_str())); } else if (need_collect) { found = true; // session_id should not across tenant database_schema = NULL; table_item.table_name_ = table_schema->get_table_name_str(); table_item.mode_ = table_schema->get_name_case_mode(); if (OB_FAIL(schema_guard.get_database_schema(*tenant_id, table_schema->get_database_id(), database_schema))) { LOG_WARN("failed to get database schema", K(ret), "tenant_id", tenant_id); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database schema is null", K(ret)); } else if (database_schema->is_in_recyclebin() || table_schema->is_in_recyclebin()) { LOG_INFO("skip table schema in recyclebin", K(*table_schema)); } else if (FALSE_IT(table_item.database_name_ = database_schema->get_database_name_str())) { //impossible } else if (OB_FAIL(drop_table_arg.tables_.push_back(table_item))) { LOG_WARN("failed to add table item!", K(table_item), K(ret)); } else { LOG_DEBUG("a temporary table is about to be dropped since session disconnected", K(table_item), K(drop_table_arg)); } } } // end for } } // end FOREACH_CNT_X } return ret; } //same api for drop table, drop index, drop view // // mv rule: // If the deleted table has mv, you must first delete mv to delete this table // If it is mv, you need to update the version of the base/depend table, and mv does not enter the recycle bin // drop table process (after add temporary table featrue) // drop table t1; The temporary table is matched first, and the ordinary table cannot be found to be matched again; // drop temporary table t1; must by temp table; // drop tables in session 123456; --> collect all temp table in 23456, and drop int ObDDLService::drop_table(const ObDropTableArg &drop_table_arg, const obrpc::ObDDLRes &ddl_res) { int ret = OB_SUCCESS; // Record whether the reason for the drop table failure is due to foreign key constraints bool fail_for_fk_cons = false; ObSqlString err_table_list; ObSchemaGetterGuard schema_guard; bool check_tmp_table_only = false; //drop temporary table schema_guard.set_session_id(drop_table_arg.session_id_); uint64_t tenant_id = drop_table_arg.tenant_id_; ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat error", K(ret)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must not null", K(ret)); } else { bool is_db_in_recyclebin = false; int64_t refreshed_schema_version = 0; ObDDLSQLTransaction trans(schema_service_); DropTableIdHashSet drop_table_set; if (TMP_TABLE == drop_table_arg.table_type_ || TMP_TABLE_ORA_TRX == drop_table_arg.table_type_ || TMP_TABLE_ORA_SESS == drop_table_arg.table_type_ || TMP_TABLE_ALL == drop_table_arg.table_type_) { if (drop_table_arg.tables_.count() == 0) { if (OB_FAIL(collect_temporary_tables_in_session(drop_table_arg))) { LOG_WARN("failed to collect temporary tables in session", K(ret)); } else { LOG_INFO("collect temporary tables to be dropped succeed", K(drop_table_arg)); } } else { check_tmp_table_only = true; } tenant_id = drop_table_arg.tenant_id_; LOG_DEBUG("drop temporary tables", K(drop_table_arg), K(check_tmp_table_only)); } if (OB_FAIL(ret)) { } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), KR(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("fail to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { // lock table when drop data table for (int64_t i = 0; OB_SUCC(ret) && i < drop_table_arg.tables_.count(); ++i) { const ObTableItem &table_item = drop_table_arg.tables_.at(i); const ObTableSchema *table_schema = NULL; //ensure use the newest schema of each table if (OB_FAIL(check_table_exists( tenant_id, table_item, drop_table_arg.table_type_, schema_guard, &table_schema))) { LOG_WARN("check_table_exist failed", KR(ret)); if (OB_TABLE_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret) { ret = OB_SUCCESS; } } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schema should not be null", KR(ret)); } else if (OB_FAIL(drop_table_set.set_refactored(table_schema->get_table_id()))) { LOG_WARN("set table_id to hash set failed", K(table_schema->get_table_id()), K(ret)); } else if (OB_FAIL(lock_table(trans, *table_schema))) { LOG_WARN("fail to lock_table", KR(ret), KPC(table_schema)); // for ddl retry task, upper layer only focus on `OB_TRY_LOCK_ROW_CONFLICT`, and then retry it. const bool is_ddl_scheduled_task = drop_table_arg.task_id_ > 0 ? true : false; ret = is_ddl_scheduled_task && ObDDLUtil::is_table_lock_retry_ret_code(ret) ? OB_TRY_LOCK_ROW_CONFLICT : ret; } } } ObMockFKParentTableSchema mock_fk_parent_table_schema; SMART_VAR(ObTableSchema, tmp_table_schema) { for (int64_t i = 0; OB_SUCC(ret) && i < drop_table_arg.tables_.count(); ++i) { ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL; // will use it when drop a fk_parent_table mock_fk_parent_table_schema.reset(); const ObTableItem &table_item = drop_table_arg.tables_.at(i); const ObTableSchema *table_schema = NULL; tmp_table_schema.reset(); is_db_in_recyclebin = false; //ensure use the newest schema of each table if (OB_FAIL(check_table_exists(tenant_id, table_item, drop_table_arg.table_type_, schema_guard, &table_schema))) { LOG_WARN("check_table_exist failed", K(ret)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schema should not be null", KR(ret)); } else if (check_tmp_table_only && false == table_schema->is_tmp_table()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schema should be temporary", KR(ret)); } else if (!drop_table_arg.force_drop_ && table_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not drop table in recyclebin, use purge instead", K(ret), K(table_item)); } else if (OB_FAIL(tmp_table_schema.assign(*table_schema))) { LOG_WARN("fail to assign table schema", K(ret)); } else if (FALSE_IT(tmp_table_schema.set_in_offline_ddl_white_list(table_item.is_hidden_))) { } else if (OB_FAIL(schema_guard.check_database_in_recyclebin( tenant_id, table_schema->get_database_id(), is_db_in_recyclebin))) { LOG_WARN("check database in recyclebin failed", K(ret)); } else if (!drop_table_arg.force_drop_ && is_db_in_recyclebin) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not drop table in recyclebin", K(ret), K(*table_schema)); } else if (OB_FAIL(check_enable_sys_table_ddl(*table_schema, OB_DDL_DROP_TABLE))) { LOG_WARN("ddl is not allowed on sys table", K(ret)); } else if (!tmp_table_schema.check_can_do_ddl()) { ret = OB_NOT_SUPPORTED; LOG_WARN("offline ddl is being executed, other ddl operations are not allowed", K(tmp_table_schema), K(ret)); } if (OB_SUCC(ret)) { ObString ddl_stmt_str; ObSqlString drop_sql; bool is_cascade_constrains = false; lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::MYSQL; if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) { LOG_WARN("fail to get tenant mode", K(ret), K(tenant_id)); } else if (lib::Worker::CompatMode::ORACLE == compat_mode) { // oracle cascade constarints use if_exist_ flag is_cascade_constrains = drop_table_arg.if_exist_; } if (OB_FAIL(ret)) { } else if (USER_INDEX == drop_table_arg.table_type_) { ddl_stmt_str = drop_table_arg.ddl_stmt_str_; } else if (OB_FAIL(construct_drop_sql(table_item, drop_table_arg.table_type_, drop_sql, lib::Worker::CompatMode::ORACLE == compat_mode, is_cascade_constrains))) { LOG_WARN("construct_drop_sql failed", K(ret)); } else { ddl_stmt_str = drop_sql.string(); } if (OB_SUCC(ret)) { if (drop_table_arg.table_type_ == USER_TABLE) { bool has_mv = false; if (OB_FAIL(check_table_has_materialized_view(schema_guard, *table_schema, has_mv))) { LOG_WARN("fail to check drop table has materialized view", K(ret), K(*table_schema)); } else if (has_mv) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support dropping table has materialized view", K(ret)); } } } // Check foreign key constraints if (OB_SUCC(ret)) { // If it is the parent table in the foreign key constraint, drop is not allowed if (OB_FAIL(ret)) { } else if (!is_cascade_constrains && table_schema->is_parent_table()) { int64_t violated_fk_index = -1; const ObIArray &foreign_key_infos = table_schema->get_foreign_key_infos(); if (table_schema->is_child_table()) { // deal with self reference fk for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); if (OB_HASH_EXIST == drop_table_set.exist_refactored(foreign_key_info.child_table_id_)) { // do-nothing, Because multiple transactions update the schema, but the schema cache is not refreshed, // you need to check that the filter sub-table has been dropped, and the current table can be dropped } else if (table_schema->get_table_id() != foreign_key_info.child_table_id_) { violated_fk_index = i; ret = OB_ERR_TABLE_IS_REFERENCED; LOG_WARN("Cannot drop table that is referenced by foreign key", K(ret), K(table_schema->get_table_name_str())); } else { } // do-nothing This table is only a child table, not a parent table of any table, can drop } } else { // Check whether the sub-tables are all drop for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i); if (OB_HASH_EXIST == drop_table_set.exist_refactored(foreign_key_info.child_table_id_)) { // do-nothing } else { violated_fk_index = i; ret = OB_ERR_TABLE_IS_REFERENCED; LOG_WARN("Cannot drop table that is referenced by foreign key", K(ret), K(table_schema->get_table_name_str())); } } } if (lib::Worker::CompatMode::MYSQL == compat_mode && OB_ERR_TABLE_IS_REFERENCED == ret) { const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(violated_fk_index); if (OB_FAIL(gen_mock_fk_parent_table_for_drop_table( schema_service, schema_guard, drop_table_arg, drop_table_set, foreign_key_infos, foreign_key_info, table_schema, mock_fk_parent_table_schema))) { LOG_WARN("gen_mock_fk_parent_table_for_drop_table failed", K(ret), K(table_schema->get_table_name_str())); } else { mock_fk_parent_table_ptr = &mock_fk_parent_table_schema; } } } else { // normal table or table_schema->is_child_table() // do-nothing } } if (OB_SUCC(ret)) { // The oracle temporary table is only to clean up the data of the session unless you explicitly drop the table if (TMP_TABLE_ORA_SESS == drop_table_arg.table_type_ || TMP_TABLE_ORA_TRX == drop_table_arg.table_type_) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support to clear oracle temp table", K(ret)); /* if (OB_FAIL(truncate_oracle_temp_table(table_item.database_name_, table_item.table_name_, drop_table_arg.tenant_id_, drop_table_arg.session_id_, drop_table_arg.sess_create_time_))) { LOG_WARN("delete data from oracle temporary table failed", K(ret), K(table_item)); } */ } else { bool to_recyclebin = drop_table_arg.to_recyclebin_; if (table_schema->get_table_type() == MATERIALIZED_VIEW || table_schema->is_tmp_table()) { to_recyclebin = false; } if (OB_FAIL(drop_table_in_trans(schema_guard, tmp_table_schema, false, USER_INDEX == drop_table_arg.table_type_, to_recyclebin, &ddl_stmt_str, &trans, &drop_table_set, mock_fk_parent_table_ptr /* will use it when drop a fk_parent_table */))) { LOG_WARN("ddl_service_ drop_table failed", K(table_item), K(tenant_id), K(ret)); } } } } LOG_INFO("finish drop table", K(tenant_id), K(table_item), K(ret)); if (OB_ERR_TABLE_IS_REFERENCED == ret) { fail_for_fk_cons = true; ret = OB_SUCCESS; } else if (OB_TABLE_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret) { int tmp_ret = OB_SUCCESS; ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = log_drop_warn_or_err_msg(table_item, drop_table_arg.if_exist_, err_table_list))) { ret = tmp_ret; LOG_WARN("log_drop_warn_or_err_msg failed", K(ret)); } else { ret = OB_SUCCESS; } } } } if (OB_FAIL(ret)) { } else if (fail_for_fk_cons || (!err_table_list.empty())) { if ((!fail_for_fk_cons) && drop_table_arg.if_exist_) { //should produce warning //todo } else { switch(drop_table_arg.table_type_) { case TMP_TABLE: case USER_TABLE: case MATERIALIZED_VIEW: case USER_VIEW: { if (fail_for_fk_cons) { ret = OB_ERR_TABLE_IS_REFERENCED; LOG_WARN("Cannot drop table that is referenced by foreign key", K(ret)); } else { ret = OB_ERR_BAD_TABLE; LOG_USER_ERROR(OB_ERR_BAD_TABLE, (int32_t)err_table_list.length() - 1, err_table_list.ptr()); LOG_WARN("failed to drop table/view ", K(ret), K(err_table_list)); } break; } case USER_INDEX: { ret = OB_TABLE_NOT_EXIST; LOG_WARN("failed to drop index table", K(err_table_list), K(ret)); break; } default: { ret = OB_ERR_UNEXPECTED; SQL_RESV_LOG(WARN, "Unknown table type", K(drop_table_arg), K(ret)); } } } } // to avoid drop table again in some scenarios like succeed to drop table but RPC timeout, // drop table and update ddl task status should be done in single trans. if (OB_FAIL(ret)) { } else if (drop_table_arg.task_id_ > 0 && OB_FAIL(ObDDLRetryTask::update_task_status_succ(trans, tenant_id, drop_table_arg.task_id_))) { LOG_WARN("update task status of drop table failed", K(ret)); } //no matter success or not, we should publish schema //drop table t1, __all_table, may be t1 has been dropped successfully //but __all_table failed, and we should refresh schema to see t1 dropped if (trans.is_started()) { int temp_ret = OB_SUCCESS; bool commit = OB_SUCC(ret); if (OB_SUCCESS != (temp_ret = trans.end(commit))) { ret = (OB_SUCC(ret)) ? temp_ret : ret; LOG_WARN("trans end failed", K(commit), K(temp_ret)); } } int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { ret = tmp_ret; LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::rebuild_index(const ObRebuildIndexArg &arg, const int64_t frozen_version, obrpc::ObAlterTableRes &res) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; const bool is_index = false; ObArenaAllocator allocator(ObModIds::OB_SCHEMA); const ObTableSchema *table_schema = NULL; ObSchemaGetterGuard schema_guard; // bool to_recyclebin = arg.to_recyclebin(); bool is_db_in_recyclebin = false; int64_t refreshed_schema_version = 0; schema_guard.set_session_id(arg.session_id_); if (check_inner_stat()) { ret = OB_INNER_STAT_ERROR; LOG_WARN("check_inner_stat error", K(is_inited()), KR(ret)); } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), KR(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("get_schema_guard failed", KR(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema( tenant_id, arg.database_name_, arg.table_name_, is_index, table_schema))) { LOG_WARN("failed to get data table schema", K(arg), KR(ret)); } else if (OB_ISNULL(table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(arg.database_name_), to_cstring(arg.table_name_)); LOG_WARN("table not found", K(arg), KR(ret)); } else if (table_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not truncate index of table in recyclebin.", KR(ret), K(arg)); } else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id, table_schema->get_database_id(), is_db_in_recyclebin))) { LOG_WARN("check database in recyclebin failed", KR(ret), K(tenant_id)); } else if (is_db_in_recyclebin) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("Can not truncate index of db in recyclebin", KR(ret), K(arg)); } else { const uint64_t table_id = table_schema->get_table_id(); const ObTableSchema *index_table_schema = NULL; ObString index_table_name; ObIndexBuilder index_builder(*this); if (OB_FAIL(ObTableSchema::build_index_table_name( // index name allocator, table_id, arg.index_name_, index_table_name))) { LOG_WARN("build_index_table_name failed", K(arg), K(table_id), KR(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_schema->get_database_id(), index_table_name, true, index_table_schema))) { LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), K(index_table_schema)); } else if (OB_ISNULL(index_table_schema)) { ret = OB_ERR_CANT_DROP_FIELD_OR_KEY; LOG_WARN("index table schema should not be null", K(arg.index_name_), KR(ret)); LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, arg.index_name_.length(), arg.index_name_.ptr()); } else { ObString ddl_stmt_str = arg.ddl_stmt_str_; ObTableSchema new_table_schema; SMART_VAR(ObCreateIndexArg, create_index_arg) { ObDDLTaskRecord task_record; ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(trans.start(&GCTX.root_service_->get_sql_proxy(), tenant_id, refreshed_schema_version))) { LOG_WARN("fail to start trans", K(ret)); } else if (OB_FAIL(new_table_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", KR(ret)); } else if (OB_FAIL(rebuild_index_in_trans(schema_guard, new_table_schema, frozen_version, &ddl_stmt_str, &trans))) { LOG_WARN("ddl_service_ rebuild_index failed", K(tenant_id), KR(ret)); } else if (OB_FAIL(index_builder.submit_build_index_task(trans, create_index_arg, table_schema, &new_table_schema, arg.parallelism_, allocator, task_record))) { LOG_WARN("fail to submit build global index task", KR(ret)); } else { res.index_table_id_ = new_table_schema.get_table_id(); res.schema_version_ = new_table_schema.get_schema_version(); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(ret), K(temp_ret)); ret = OB_SUCC(ret) ? temp_ret : ret; } } if (OB_SUCC(ret)) { int tmp_ret = OB_SUCCESS; if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("fail to publish schema", K(ret), K(tenant_id)); } else if (OB_TMP_FAIL(GCTX.root_service_->get_ddl_task_scheduler().schedule_ddl_task(task_record))) { LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record)); } } } } } ObSqlString err_table_list; if (OB_TABLE_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret) { int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = log_rebuild_warn_or_err_msg(arg, err_table_list))) { ret = tmp_ret; LOG_WARN("log_drop_warn_or_err_msg failed", KR(ret)); } else { ret = OB_TABLE_NOT_EXIST; LOG_WARN("failed to drop index table", K(err_table_list), KR(ret)); } } LOG_INFO("finish rebuild index", K(arg), KR(ret)); return ret; } // Rebuild index information is written to the internal table within a transaction // If sql_trans is NULL, you need to create a transaction inside the function int ObDDLService::rebuild_index_in_trans( ObSchemaGetterGuard &schema_guard, ObTableSchema &index_schema, const int64_t frozen_version, const ObString *ddl_stmt_str, ObMySQLTransaction *sql_trans) { int ret = OB_SUCCESS; uint64_t new_table_id = index_schema.get_table_id(); ObSchemaService *schema_service = schema_service_->get_schema_service(); ObDDLSQLTransaction tmp_trans(schema_service_); ObMySQLTransaction &trans = OB_ISNULL(sql_trans) ? tmp_trans : *sql_trans; const uint64_t tenant_id = index_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service must not null", KR(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_ISNULL(sql_trans) && OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(drop_table_in_trans( schema_guard, index_schema, true, true, false, ddl_stmt_str, &trans, NULL, NULL))) { LOG_WARN("drop_table failed", K(index_schema), KR(ret)); } else if (FALSE_IT(new_table_id = OB_INVALID_ID)) { } else if (OB_FAIL(schema_service->fetch_new_table_id( index_schema.get_tenant_id(), new_table_id))) { LOG_WARN("fail to fetch new table id", KR(ret)); } else { index_schema.set_index_status(INDEX_STATUS_UNAVAILABLE); index_schema.set_table_id(new_table_id); } if (OB_FAIL(ret)) { } else if (OB_FAIL(generate_tablet_id(index_schema))) { LOG_WARN("failed to generate tablet id", K(ret)); } else if (OB_FAIL(create_table_in_trans(index_schema, frozen_version, ddl_stmt_str, &trans, schema_guard))) { LOG_WARN("create_table_in_trans failed", K(index_schema), KR(ret), K(frozen_version), K(ddl_stmt_str)); } if (OB_ISNULL(sql_trans) && trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } return ret; } int ObDDLService::update_index_status(const obrpc::ObUpdateIndexStatusArg &arg) { int ret = OB_SUCCESS; const ObTableSchema *table = NULL; ObSchemaGetterGuard schema_guard; share::schema::ObIndexStatus new_status = arg.status_; const uint64_t table_id = arg.index_table_id_; const uint64_t tenant_id = arg.exec_tenant_id_; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == table_id || new_status <= INDEX_STATUS_NOT_FOUND || new_status >= INDEX_STATUS_MAX) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arugment", KT(table_id), K(new_status)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, table))) { LOG_WARN("get table schema failed", KR(ret), K(tenant_id), KT(table_id)); } else if (NULL == table) { // maybe table has already been deleted, do-nothing } else if (INDEX_STATUS_INDEX_ERROR == new_status && arg.convert_status_) { // Distinguish the failure of the physical recovery index rebuild const ObSimpleTenantSchema *tenant_schema = NULL; if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("fail to get tenant info", KR(ret), K(tenant_id)); } else if (OB_ISNULL(tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", KR(ret), K(tenant_id)); } else if (tenant_schema->is_restore()) { new_status = INDEX_STATUS_RESTORE_INDEX_ERROR; LOG_INFO("convert error index status", KR(ret), K(new_status)); } } if (OB_SUCC(ret) && NULL != table) { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.update_index_status( tenant_id, table->get_data_table_id(), table_id, new_status, arg.in_offline_ddl_white_list_, trans))) { } if (trans.is_started()) { int commit_ret = trans.end(OB_SUCC(ret)); if (OB_SUCCESS != commit_ret) { LOG_WARN("end transaction failed", KR(ret), K(commit_ret)); ret = OB_SUCC(ret) ? commit_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", KR(ret)); } } } return ret; } int ObDDLService::add_table_schema( ObTableSchema &table_schema, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_FAIL(create_table_in_trans(table_schema, 0, NULL, NULL, schema_guard))) { LOG_WARN("create_table_in_trans failed", KR(ret), K(table_schema)); } return ret; } int ObDDLService::drop_inner_table(const share::schema::ObTableSchema &table_schema) { int ret = OB_SUCCESS; ObString *stmt = NULL; ObSchemaGetterGuard schema_guard; const uint64_t tenant_id = table_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (!is_inner_table(table_schema.get_table_id())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("table not inner table", "table_id", table_schema.get_table_id(), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(drop_table_in_trans(schema_guard, table_schema, false, table_schema.is_index_table(), false, /* to recyclebin*/ stmt, NULL, NULL, NULL))) { LOG_WARN("drop table in transaction failed", K(ret), K(table_schema)); } return ret; } bool ObDDLService::is_zone_exist(const ObArray &zones, const ObZone &zone) { bool is_exist = false; for (int64_t i = 0; i < zones.count(); ++i) { if (zones[i] == zone) { is_exist = true; } } return is_exist; } int ObDDLService::create_sys_tenant( const obrpc::ObCreateTenantArg &arg, share::schema::ObTenantSchema &tenant_schema) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); ObSchemaService *schema_service = NULL; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); schema_service = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must not null", K(ret)); } else { ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_; ObRefreshSchemaStatus tenant_status(OB_SYS_TENANT_ID, OB_INVALID_TIMESTAMP, OB_INVALID_VERSION); ObSysVariableSchema sys_variable; tenant_schema.set_tenant_id(OB_SYS_TENANT_ID); const ObSchemaOperationType operation_type = OB_DDL_MAX_OP; // When the system tenant is created, the log_operation of the system variable is not recorded separately // The update of __all_core_table must be a single-partition transaction. // Failure to create a tenant will result in garbage data, but it will not affect int64_t refreshed_schema_version = 0; // won't lock if (OB_ISNULL(schema_status_proxy)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_status_proxy is null", K(ret)); } else if (OB_FAIL(schema_status_proxy->set_tenant_schema_status(tenant_status))) { LOG_WARN("init tenant create partition status failed", K(ret), K(tenant_status)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret)); } else if (OB_FAIL(init_system_variables(arg, tenant_schema, sys_variable))) { LOG_WARN("fail to init tenant sys params", K(ret), K(tenant_schema)); } else if (OB_FAIL(ddl_operator.create_tenant(tenant_schema, OB_DDL_ADD_TENANT, trans))) { LOG_WARN("create tenant failed", K(tenant_schema), K(ret)); } else if (OB_FAIL(ddl_operator.replace_sys_variable( sys_variable, tenant_schema.get_schema_version(), trans, operation_type))) { LOG_WARN("fail to replace sys variable", K(ret), K(sys_variable)); } else if (OB_FAIL(ddl_operator.init_tenant_env(tenant_schema, sys_variable, share::PRIMARY_TENANT_ROLE, trans))) { LOG_WARN("init tenant env failed", K(tenant_schema), K(ret)); } else if (OB_FAIL(ddl_operator.insert_tenant_merge_info(OB_DDL_ADD_TENANT, tenant_schema, trans))) { LOG_WARN("fail to insert tenant merge info", KR(ret)); } else if (OB_FAIL(ObServiceEpochProxy::init_service_epoch(trans, OB_SYS_TENANT_ID, 0/*freeze_service_epoch*/))) { LOG_WARN("fail to init service epoch", KR(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; LOG_INFO("end create tenant", "is_commit", OB_SUCC(ret), K(ret)); if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { ret = (OB_SUCC(ret)) ? temp_ret : ret; LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); } } } } return ret; } /* The reconstruction of zone_list follows the following rules: * 1. When creating tenant, the zone_list specified by the user will be ignored. Use the zone list of resource_pool. * We still retain the zone_list column in the all_tenant table and the zone_list field in the tenantSchema. * 2. When creating table, the zone_list specified by the user will be ignored. * If the create table does not specify locality, the locality of the table is not filled in, * and the zone_list is not filled in. * Both locality and zone_list are inherited from the tenant to which the table belongs. * 3. When creating table, the user specified zone_list and locality, the user-specified zone_list will be ignored; * a zone list will be calculated based on the user-specified locality and the zone list of the resource pool, * and the zone_list will be filled into the zone_list column of __all_table and in the zone_list field of TableSchema. * 4. When modifying locality, we are not allowed to modify F{3}@region_hz to F{2}@region_hz; * because the semantics of this modification is not clear enough, * it is impossible to clearly derive the change of zone */ int ObDDLService::create_tenant( const ObCreateTenantArg &arg, UInt64 &tenant_id) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObString &tenant_name = arg.tenant_schema_.get_tenant_name_str(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("fail to check inner stat", KR(ret)); } else if (!arg.is_valid()) { ret = OB_MISS_ARGUMENT; if (tenant_name.empty()) { LOG_USER_ERROR(OB_MISS_ARGUMENT, "tenant name"); } else if (arg.pool_list_.count() <= 0) { LOG_USER_ERROR(OB_MISS_ARGUMENT, "resource_pool_list"); } LOG_WARN("missing arg to create tenant", KR(ret), K(arg)); } else if (GCONF.in_upgrade_mode()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("create tenant when cluster is upgrading not allowed", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "create tenant when cluster is upgrading"); } else if (OB_ISNULL(sql_proxy_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard", KR(ret)); } else { // check tenant exist bool tenant_exist = false; if (OB_NOT_NULL(schema_guard.get_tenant_info(tenant_name))) { tenant_exist = true; } else if (!arg.is_restore_) { if (OB_FAIL(ObRestoreUtil::check_has_physical_restore_job(*sql_proxy_, tenant_name, tenant_exist))) { LOG_WARN("failed to check has physical restore job", KR(ret), K(tenant_name)); } } if (OB_FAIL(ret)) { } else if (tenant_exist) { if (arg.if_not_exist_) { ret = OB_SUCCESS; LOG_USER_NOTE(OB_TENANT_EXIST, to_cstring(tenant_name)); LOG_INFO("tenant already exists, not need to create", KR(ret), K(tenant_name)); } else { ret = OB_TENANT_EXIST; LOG_USER_ERROR(OB_TENANT_EXIST, to_cstring(tenant_name)); LOG_WARN("tenant already exists", KR(ret), K(tenant_name)); } } else if (OB_FAIL(create_tenant(arg, schema_guard, tenant_id))) { LOG_WARN("fail to create tenant", KR(ret), K(arg)); } } return ret; } int ObDDLService::generate_tenant_schema( const ObCreateTenantArg &arg, const share::ObTenantRole &tenant_role, share::schema::ObSchemaGetterGuard &schema_guard, ObTenantSchema &user_tenant_schema, ObSysVariableSchema &user_sys_variable, ObTenantSchema &meta_tenant_schema, ObSysVariableSchema &meta_sys_variable) { int ret = OB_SUCCESS; uint64_t user_tenant_id = arg.tenant_schema_.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("fail to check inner stat", KR(ret)); } else if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(schema_service)); } else if (OB_FAIL(schema_service_->get_schema_service()->fetch_new_tenant_id(user_tenant_id))) { LOG_WARN("fetch_new_tenant_id failed", KR(ret)); } else if (OB_INVALID_ID == user_tenant_id) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant id is invalid", KR(ret), K(user_tenant_id)); } else if (OB_FAIL(user_tenant_schema.assign(arg.tenant_schema_))) { LOG_WARN("fail to assign user tenant schema", KR(ret), K(arg)); } else if (OB_FAIL(meta_tenant_schema.assign(user_tenant_schema))) { LOG_WARN("fail to assign meta tenant schema", KR(ret), K(arg)); } else if (OB_FAIL(check_create_tenant_schema( arg.pool_list_, meta_tenant_schema, schema_guard))) { LOG_WARN("check tenant schema failed", KR(ret), K(meta_tenant_schema), K(arg)); } else if (OB_FAIL(check_create_tenant_schema( arg.pool_list_, user_tenant_schema, schema_guard))) { LOG_WARN("check tenant schema failed", KR(ret), K(user_tenant_schema), K(arg)); } else { // user tenant if (OB_SUCC(ret)) { user_tenant_schema.set_tenant_id(user_tenant_id); if (!tenant_role.is_primary()) { //standby cluster and restore tenant no need init user tenant system variables if (tenant_role.is_restore()) { user_tenant_schema.set_status(TENANT_STATUS_RESTORE); } } else if (OB_FAIL(init_system_variables(arg, user_tenant_schema, user_sys_variable))) { LOG_WARN("fail to init tenant sys params", KR(ret), K(user_tenant_schema), K(arg)); } else if (OB_FAIL(check_tenant_primary_zone_(schema_guard, user_tenant_schema))) { LOG_WARN("fail to check tenant primary zone", KR(ret), K(user_tenant_schema)); } } // meta tenant if (OB_SUCC(ret)) { const uint64_t meta_tenant_id = gen_meta_tenant_id(user_tenant_id); ObSqlString table_name; if (OB_FAIL(table_name.assign_fmt("META$%ld", user_tenant_id))) { LOG_WARN("fail to assign tenant name",KR(ret), K(user_tenant_id)); } else { meta_tenant_schema.set_tenant_id(meta_tenant_id); meta_tenant_schema.set_tenant_name(table_name.string()); meta_tenant_schema.set_compatibility_mode(ObCompatibilityMode::MYSQL_MODE); meta_tenant_schema.set_charset_type(ObCharset::get_default_charset()); meta_tenant_schema.set_collation_type(ObCharset::get_default_collation( ObCharset::get_default_charset())); if (OB_FAIL(init_system_variables(arg, meta_tenant_schema, meta_sys_variable))) { LOG_WARN("fail to init tenant sys params", KR(ret), K(meta_tenant_schema), K(arg)); } else if (OB_FAIL(check_tenant_primary_zone_(schema_guard, meta_tenant_schema))) { LOG_WARN("fail to check tenant primary zone", KR(ret), K(meta_tenant_schema)); } } } } return ret; } int ObDDLService::init_schema_status( const uint64_t tenant_id, const share::ObTenantRole &tenant_role) { int ret = OB_SUCCESS; ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_; if (OB_FAIL(check_inner_stat())) { LOG_WARN("fail to check inner stat", KR(ret)); } else if (is_meta_tenant(tenant_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id)); } else if (OB_ISNULL(schema_status_proxy)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to init schema status", KR(ret), K(tenant_id)); } else { // user tenant if (OB_SUCC(ret) && is_user_tenant(tenant_id)) { ObRefreshSchemaStatus partition_status(tenant_id, OB_INVALID_TIMESTAMP, OB_INVALID_VERSION); if (!tenant_role.is_primary()) { // reset tenant's schema status in standby cluster or in physical restore partition_status.snapshot_timestamp_ = 0; partition_status.readable_schema_version_ = 0; } if (FAILEDx(schema_status_proxy->set_tenant_schema_status(partition_status))) { LOG_WARN("fail to set refreshed schema status", KR(ret), K(partition_status)); } } // sys or meta tenant if (OB_SUCC(ret)) { // sys tenant's meta tenant is itself const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id); ObRefreshSchemaStatus meta_partition_status(meta_tenant_id, OB_INVALID_TIMESTAMP, OB_INVALID_VERSION); if (OB_FAIL(schema_status_proxy->set_tenant_schema_status(meta_partition_status))) { LOG_WARN("fail to set refreshed schema status", KR(ret), K(meta_partition_status)); } } } return ret; } int ObDDLService::create_tenant( const ObCreateTenantArg &arg, share::schema::ObSchemaGetterGuard &schema_guard, UInt64 &tenant_id) { int ret = OB_SUCCESS; bool is_standby = false; share::ObTenantRole tenant_role = share::PRIMARY_TENANT_ROLE; uint64_t user_tenant_id = OB_INVALID_TENANT_ID; uint64_t meta_tenant_id = OB_INVALID_TENANT_ID; palf::PalfBaseInfo user_palf_base_info; palf::PalfBaseInfo meta_palf_base_info; bool create_ls_with_palf = false; HEAP_VARS_4((ObTenantSchema, user_tenant_schema), (ObTenantSchema, meta_tenant_schema), (ObSysVariableSchema, user_sys_variable), (ObSysVariableSchema, meta_sys_variable)) { if (OB_FAIL(check_inner_stat())) { LOG_WARN("fail to check inner stat", KR(ret)); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("failed to get is standby cluster", KR(ret)); } else if (arg.is_restore_) { tenant_role = share::RESTORE_TENANT_ROLE; user_palf_base_info = arg.palf_base_info_; create_ls_with_palf = true; } else if (is_standby) { tenant_role = share::STANDBY_TENANT_ROLE; } else { tenant_role = share::PRIMARY_TENANT_ROLE; } if (OB_FAIL(ret)) { } else if (OB_FAIL(generate_tenant_schema(arg, tenant_role, schema_guard, user_tenant_schema, user_sys_variable, meta_tenant_schema, meta_sys_variable))) { LOG_WARN("fail to generate tenant schema", KR(ret), K(arg), K(tenant_role)); } else if (FALSE_IT(user_tenant_id = user_tenant_schema.get_tenant_id())) { } else if (FALSE_IT(meta_tenant_id = meta_tenant_schema.get_tenant_id())) { } else if (OB_FAIL(init_schema_status( user_tenant_schema.get_tenant_id(), tenant_role))) { LOG_WARN("fail to init schema status", KR(ret), K(user_tenant_id)); } else if (OB_FAIL(create_tenant_schema( arg, schema_guard, user_tenant_schema, meta_tenant_schema))) { LOG_WARN("fail to create tenant schema", KR(ret), K(arg)); } else { DEBUG_SYNC(BEFORE_CREATE_META_TENANT); // create ls/tablet/schema in tenant space ObArray pools; if (OB_FAIL(get_pools(arg.pool_list_, pools))) { LOG_WARN("get_pools failed", KR(ret), K(arg)); } else if (OB_FAIL(create_normal_tenant(meta_tenant_id, pools, meta_tenant_schema, tenant_role, meta_sys_variable, false/*create_ls_with_palf*/, meta_palf_base_info))) { LOG_WARN("fail to create meta tenant", KR(ret), K(meta_tenant_id), K(pools), K(meta_sys_variable), K(tenant_role), K(meta_palf_base_info)); } else { DEBUG_SYNC(BEFORE_CREATE_USER_TENANT); if (OB_FAIL(create_normal_tenant(user_tenant_id, pools, user_tenant_schema, tenant_role, user_sys_variable, create_ls_with_palf, user_palf_base_info))) { LOG_WARN("fail to create user tenant", KR(ret), K(user_tenant_id), K(pools), K(user_sys_variable), K(tenant_role), K(user_palf_base_info)); } } // drop tenant if create tenant failed. // meta tenant will be force dropped with its user tenant. if (OB_FAIL(ret) && tenant_role.is_primary()) { //tenant_id can not fallback, so can not drop tenant in standby cluster int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = try_force_drop_tenant(user_tenant_schema))) { LOG_WARN("fail to force drop tenant", KR(ret), KR(temp_ret), K(user_tenant_schema)); } } } } // end HEAP_VARS_4 if (FAILEDx(create_tenant_end(meta_tenant_id))) { LOG_WARN("failed to create tenant end", KR(ret), K(meta_tenant_id)); } else if (!tenant_role.is_primary()) { LOG_INFO("restore or standby user tenant cannot create end", K(tenant_role), K(user_tenant_id), K(arg), K(is_standby)); } else if (OB_FAIL(create_tenant_end(user_tenant_id))) { LOG_WARN("failed to create tenant end", KR(ret), K(user_tenant_id)); } if (OB_SUCC(ret)) { tenant_id = user_tenant_id; } return ret; } int ObDDLService::try_force_drop_tenant(const ObTenantSchema &tenant_schema) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("fail to check inner stat", KR(ret)); } else { obrpc::ObDropTenantArg arg; arg.tenant_name_ = tenant_schema.get_tenant_name(); arg.tenant_id_ = tenant_schema.get_tenant_id(); arg.if_exist_ = true; arg.delay_to_drop_ = false; ObSqlString sql; if (OB_FAIL(sql.append_fmt("DROP TENANT IF EXISTS %s FORCE", tenant_schema.get_tenant_name()))) { LOG_WARN("fail to generate sql", KR(ret), "tenant_id", tenant_schema.get_tenant_id()); } else if (FALSE_IT(arg.ddl_stmt_str_ = sql.string())) { } else if (OB_FAIL(drop_tenant(arg))) { LOG_WARN("fail to drop tenant", KR(ret), K(arg)); } } return ret; } // 1. create new tenant schema // 2. grant resource pool to new tenant int ObDDLService::create_tenant_schema( const ObCreateTenantArg &arg, share::schema::ObSchemaGetterGuard &schema_guard, ObTenantSchema &user_tenant_schema, ObTenantSchema &meta_tenant_schema) { const int64_t start_time = ObTimeUtility::fast_current_time(); LOG_INFO("[CREATE_TENANT] STEP 1. start create tenant schema", K(arg)); int ret = OB_SUCCESS; const uint64_t user_tenant_id = user_tenant_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("fail to check inner stat", KR(ret)); } else if (OB_ISNULL(schema_service_) || OB_ISNULL(sql_proxy_) || OB_ISNULL(unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(schema_service), KP_(sql_proxy), KP_(unit_mgr)); } else { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = OB_INVALID_VERSION; ObArray pools; common::ObArray new_ug_id_array; if (OB_FAIL(get_pools(arg.pool_list_, pools))) { LOG_WARN("get_pools failed", KR(ret), K(arg)); } else if (OB_FAIL(schema_guard.get_schema_version( OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("fail to get schema version", KR(ret), K(refreshed_schema_version)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed, ", KR(ret), K(refreshed_schema_version)); } // 1. create tenant schema if (OB_SUCC(ret)) { LOG_INFO("[CREATE_TENANT] STEP 1.1. start create tenant schema", K(arg)); const int64_t tmp_start_time = ObTimeUtility::fast_current_time(); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.create_tenant(meta_tenant_schema, OB_DDL_ADD_TENANT_START, trans))) { LOG_WARN("create tenant failed", KR(ret), K(meta_tenant_schema)); } else if (OB_FAIL(ddl_operator.create_tenant(user_tenant_schema, OB_DDL_ADD_TENANT_START, trans, &arg.ddl_stmt_str_))) { LOG_WARN("create tenant failed", KR(ret), K(user_tenant_schema)); } LOG_INFO("[CREATE_TENANT] STEP 1.2. finish create tenant schema", KR(ret), K(arg), "cost", ObTimeUtility::fast_current_time() - tmp_start_time); } // 2. grant pool if (OB_SUCC(ret)) { LOG_INFO("[CREATE_TENANT] STEP 1.2. start grant pools", K(user_tenant_id)); const int64_t tmp_start_time = ObTimeUtility::fast_current_time(); lib::Worker::CompatMode compat_mode = get_worker_compat_mode( user_tenant_schema.get_compatibility_mode()); if (OB_FAIL(unit_mgr_->grant_pools( trans, new_ug_id_array, compat_mode, pools, user_tenant_id, false, /*is_bootstrap*/ false, /*if not grant*/ false /*skip_offline_server*/))) { LOG_WARN("grant_pools_to_tenant failed", KR(ret), K(pools), K(user_tenant_id)); } LOG_INFO("[CREATE_TENANT] STEP 1.2. finish grant pools", KR(ret), K(user_tenant_id), "cost", ObTimeUtility::fast_current_time() - tmp_start_time); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; bool commit = OB_SUCC(ret); if (OB_SUCCESS != (temp_ret = trans.end(commit))) { ret = (OB_SUCC(ret)) ? temp_ret : ret; LOG_WARN("trans end failed", K(commit), K(temp_ret)); } } // If the transaction returns successfully, modify the unit_mgr memory data structure // If the transaction fails, the transaction may be submitted successfully. At this time, // the transaction is considered to have failed, and the unit_mgr memory state is not modified at this time, // and the transaction 1 is subsequently rolled back through drop tenant. if (OB_SUCC(ret)) { LOG_INFO("[CREATE_TENANT] STEP 1.3. start change pool owners", K(user_tenant_id)); const int64_t tmp_start_time = ObTimeUtility::fast_current_time(); const bool grant = true; if (OB_FAIL(unit_mgr_->commit_change_pool_owner(new_ug_id_array, grant, pools, user_tenant_id))) { LOG_WARN("commit change pool owner failed", K(grant), K(pools), K(user_tenant_id), KR(ret)); } LOG_INFO("[CREATE_TENANT] STEP 1.3. finish change pool owners", KR(ret), K(user_tenant_id), "cost", ObTimeUtility::fast_current_time() - tmp_start_time); } if (OB_SUCC(ret)) { ObArray addrs; ObZone zone; // empty means get all zone's servers if (OB_FAIL(unit_mgr_->get_tenant_unit_servers(user_tenant_id, zone, addrs))) { LOG_WARN("fail to get tenant's servers", KR(ret), K(user_tenant_id)); } else if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID, addrs))) { LOG_WARN("publish schema failed", KR(ret), K(addrs)); } } } LOG_INFO("[CREATE_TENANT] STEP 1. finish create tenant schema", KR(ret), K(user_tenant_id), "cost", ObTimeUtility::fast_current_time() - start_time); return ret; } // 1. create tenant's sys ls // 2. broadcast sys table schemas // 3. create tenant's sys tablets // 4. init tenant's schema(sys table schemas, database schemas, ...) int ObDDLService::create_normal_tenant( const uint64_t tenant_id, const ObIArray &pool_list, const share::schema::ObTenantSchema &tenant_schema, const share::ObTenantRole &tenant_role, ObSysVariableSchema &sys_variable, const bool create_ls_with_palf, const palf::PalfBaseInfo &palf_base_info) { const int64_t start_time = ObTimeUtility::fast_current_time(); LOG_INFO("[CREATE_TENANT] STEP 2. start create tenant", K(tenant_id), K(tenant_schema)); int ret = OB_SUCCESS; ObSArray tables; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (is_sys_tenant(tenant_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id)); } else if (OB_FAIL(insert_restore_tenant_job(tenant_id, tenant_schema.get_tenant_name(), tenant_role))) { LOG_WARN("failed to insert restore tenant job", KR(ret), K(tenant_id), K(tenant_role), K(tenant_schema)); } else if (OB_FAIL(create_tenant_sys_ls(tenant_schema, pool_list, create_ls_with_palf, palf_base_info))) { LOG_WARN("fail to create tenant sys log stream", KR(ret), K(tenant_schema), K(pool_list), K(palf_base_info)); } else if (is_user_tenant(tenant_id) && !tenant_role.is_primary()) { //standby cluster no need create sys tablet and init tenant schema } else if (OB_FAIL(ObSchemaUtils::construct_inner_table_schemas(tenant_id, tables))) { LOG_WARN("fail to get inner table schemas in tenant space", KR(ret), K(tenant_id)); } else if (OB_FAIL(broadcast_sys_table_schemas(tenant_id, tables))) { LOG_WARN("fail to broadcast sys table schemas", KR(ret), K(tenant_id)); } else if (OB_FAIL(create_tenant_sys_tablets(tenant_id, tables))) { LOG_WARN("fail to create tenant partitions", KR(ret), K(tenant_id)); } else if (OB_FAIL(init_tenant_schema(tenant_id, tenant_schema, tenant_role, tables, sys_variable))) { LOG_WARN("fail to init tenant schema", KR(ret), K(tenant_role), K(tenant_id), K(tenant_schema), K(sys_variable)); } LOG_INFO("[CREATE_TENANT] STEP 2. finish create tenant", KR(ret), K(tenant_id), "cost", ObTimeUtility::fast_current_time() - start_time); return ret; } int ObDDLService::insert_restore_tenant_job( const uint64_t tenant_id, const ObString &tenant_name, const share::ObTenantRole &tenant_role) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (is_sys_tenant(tenant_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id)); } else if (OB_ISNULL(sql_proxy_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy)); } else if (is_meta_tenant(tenant_id) || !tenant_role.is_restore()) { //no need to insert retore job; } else if (OB_FAIL(ObRestoreUtil::insert_user_tenant_restore_job(*sql_proxy_, tenant_name, tenant_id))) { LOG_WARN("failed to insert user tenant restore job", KR(ret), K(tenant_id), K(tenant_name)); } return ret; } int ObDDLService::create_tenant_sys_ls( const ObTenantSchema &tenant_schema, const ObIArray &pool_list, const bool create_ls_with_palf, const palf::PalfBaseInfo &palf_base_info) { const int64_t start_time = ObTimeUtility::fast_current_time(); LOG_INFO("[CREATE_TENANT] STEP 2.1. start create sys log stream", K(tenant_schema)); int ret = OB_SUCCESS; const uint64_t tenant_id = tenant_schema.get_tenant_id(); int64_t wait_leader = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (is_sys_tenant(tenant_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id)); } else if (OB_ISNULL(rpc_proxy_) || OB_ISNULL(sql_proxy_) || OB_ISNULL(lst_operator_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(rpc_proxy), KP_(sql_proxy), KP_(lst_operator)); } else { ObArray locality; ObArray primary_zone_list; ObSqlString zone_priority; share::schema::ObSchemaGetterGuard schema_guard; // not used int64_t paxos_replica_num = OB_INVALID_ID; ObLSCreator ls_creator(*rpc_proxy_, tenant_id, SYS_LS, sql_proxy_); if (OB_FAIL(tenant_schema.get_zone_replica_attr_array(locality))) { LOG_WARN("fail to get tenant's locality", KR(ret), K(locality)); } else if (OB_FAIL(tenant_schema.get_paxos_replica_num(schema_guard, paxos_replica_num))) { LOG_WARN("failed to get paxos replica num", KR(ret)); } else if (OB_FAIL(ObPrimaryZoneUtil::get_tenant_primary_zone_array(tenant_schema, primary_zone_list))) { LOG_WARN("failed to get tenant primary zone array", KR(ret)); } else if (OB_UNLIKELY(0 == primary_zone_list.count())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("primary zone is empty", KR(ret), K(tenant_schema)); } else if (OB_FAIL(ObTenantLSInfo::get_zone_priority( primary_zone_list.at(0), tenant_schema, zone_priority))) { LOG_WARN("failed to get zone priority", KR(ret), K(primary_zone_list), K(tenant_schema)); } else if (OB_FAIL(ls_creator.create_tenant_sys_ls( primary_zone_list.at(0), locality, pool_list, paxos_replica_num, tenant_schema.get_compatibility_mode(), zone_priority.string(), create_ls_with_palf, palf_base_info))) { LOG_WARN("fail to create tenant sys ls", KR(ret), K(pool_list), K(palf_base_info), K(locality), K(paxos_replica_num), K(tenant_schema), K(zone_priority)); } else { share::ObLSLeaderElectionWaiter ls_leader_waiter(*lst_operator_, stopped_); int64_t timeout = GCONF.rpc_timeout; if (INT64_MAX != THIS_WORKER.get_timeout_ts()) { timeout = max(timeout, THIS_WORKER.get_timeout_remain()); } int64_t wait_leader_start = ObTimeUtility::current_time(); if (OB_FAIL(ls_leader_waiter.wait(tenant_id, SYS_LS, timeout))) { LOG_WARN("fail to wait election leader", KR(ret), K(tenant_id), K(SYS_LS), K(timeout)); } int64_t wait_leader_end = ObTimeUtility::current_time(); wait_leader = wait_leader_end - wait_leader_end; } } if (is_meta_tenant(tenant_id)) { DEBUG_SYNC(AFTER_CREATE_META_TENANT_SYS_LOGSTREAM); } else { DEBUG_SYNC(AFTER_CREATE_USER_TENANT_SYS_LOGSTREAM); } LOG_INFO("[CREATE_TENANT] STEP 2.1. finish create sys log stream", KR(ret), K(tenant_schema), "cost", ObTimeUtility::fast_current_time() - start_time, "wait leader", wait_leader); return ret; } int ObDDLService::broadcast_sys_table_schemas( const uint64_t tenant_id, common::ObIArray &tables) { const int64_t start_time = ObTimeUtility::fast_current_time(); LOG_INFO("[CREATE_TENANT] STEP 2.2. start broadcast sys table schemas", K(tenant_id)); int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_ISNULL(lst_operator_) || OB_ISNULL(rpc_proxy_) || OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(lst_operator), KP_(rpc_proxy), KP_(schema_service)); } else { // Ensure observer which contains rs or tenant's sys ls leader has avaliable schemas. ObLSInfo ls_info; ObArray addrs; const ObLSReplica *leader = NULL; ObLSReplica::MemberList member_list; if (OB_FAIL(lst_operator_->get( GCONF.cluster_id, tenant_id, SYS_LS, ls_info))) { LOG_WARN("fail to get sys ls info", KR(ret), K(tenant_id)); } else if (OB_FAIL(ls_info.find_leader(leader))) { LOG_WARN("fail to get leader", KR(ret), K(tenant_id)); } else if (OB_ISNULL(leader)) { ret = OB_LEADER_NOT_EXIST; LOG_WARN("leader is null", KR(ret), K(tenant_id)); } else { member_list = leader->get_member_list(); ARRAY_FOREACH_N(member_list, idx, cnt) { const ObAddr &server = member_list.at(idx).get_server(); if (OB_UNLIKELY(!server.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid server", KR(ret), K(server), K(member_list)); } else if (OB_FAIL(addrs.push_back(server))) { LOG_WARN("fail to push back server", KR(ret), K(server), K(addrs)); } } if (OB_SUCC(ret) && !is_contain(addrs, GCONF.self_addr_) && OB_FAIL(addrs.push_back(GCONF.self_addr_))) { LOG_WARN("fail to push back rs addr", KR(ret)); } } if (OB_SUCC(ret)) { ObTimeoutCtx ctx; ObBatchBroadcastSchemaProxy proxy(*rpc_proxy_, &ObSrvRpcProxy::batch_broadcast_schema); obrpc::ObBatchBroadcastSchemaArg arg; int64_t sys_schema_version = OB_INVALID_VERSION; if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, GCONF.rpc_timeout))) { LOG_WARN("fail to set timeout ctx", KR(ret)); } else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version( OB_SYS_TENANT_ID, sys_schema_version))) { } else if (OB_FAIL(arg.init(tenant_id, sys_schema_version, tables))) { LOG_WARN("fail to init arg", KR(ret), K(tenant_id), K(sys_schema_version)); } const int64_t timeout_ts = ctx.get_timeout(0); for (int64_t i = 0; OB_SUCC(ret) && i < addrs.count(); i++) { const ObAddr &addr = addrs.at(i); if (OB_FAIL(proxy.call(addr, timeout_ts, arg))) { LOG_WARN("fail to send rpc", KR(ret), K(tenant_id), K(sys_schema_version), K(addr), K(timeout_ts)); } } // end for ObArray return_code_array; int tmp_ret = OB_SUCCESS; // always wait all if (OB_SUCCESS != (tmp_ret = proxy.wait_all(return_code_array))) { LOG_WARN("wait batch result failed", KR(tmp_ret), KR(ret)); ret = OB_SUCC(ret) ? tmp_ret : ret; } for (int64_t i = 0; OB_SUCC(ret) && i < return_code_array.count(); i++) { int res_ret = return_code_array.at(i); const ObAddr &addr = proxy.get_dests().at(i); if (OB_SUCCESS != res_ret && (addr == leader->get_server() || addr == GCONF.self_addr_)) { // leader and rs must succeed ret = res_ret; LOG_WARN("broadcast schema failed", KR(ret), K(addr), K(tenant_id)); } } // end for } } LOG_INFO("[CREATE_TENANT] STEP 2.2. finish broadcast sys table schemas", KR(ret), K(tenant_id), "cost", ObTimeUtility::fast_current_time() - start_time); return ret; } int ObDDLService::create_tenant_sys_tablets( const uint64_t tenant_id, common::ObIArray &tables) { const int64_t start_time = ObTimeUtility::fast_current_time(); LOG_INFO("[CREATE_TENANT] STEP 2.3. start create sys table tablets", K(tenant_id)); int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_ISNULL(rpc_proxy_) || OB_ISNULL(lst_operator_) || OB_ISNULL(sql_proxy_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(rpc_proxy), KP_(lst_operator)); } else { // FIXME: (yanmu.ztl) use actual trans later ObMySQLTransaction trans; share::schema::ObSchemaGetterGuard dummy_guard; ObTableCreator table_creator(tenant_id, ObFreezeInfoManager::ORIGIN_FROZEN_SCN, *lst_operator_, trans); ObNewTableTabletAllocator new_table_tablet_allocator( tenant_id, dummy_guard, sql_proxy_); common::ObArray ls_id_array; ObArray table_schemas; ObArray index_tids; if (OB_FAIL(trans.start(sql_proxy_, tenant_id))) { LOG_WARN("fail to start trans", KR(ret), K(tenant_id)); } else if (OB_FAIL(table_creator.init())) { LOG_WARN("fail to init tablet creator", KR(ret), K(tenant_id)); } else if (OB_FAIL(new_table_tablet_allocator.init())) { LOG_WARN("fail to init new table tablet allocator", KR(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < tables.count(); i++) { const ObTableSchema &data_table = tables.at(i); const uint64_t data_table_id = data_table.get_table_id(); if (data_table.has_partition()) { table_schemas.reset(); if (OB_FAIL(table_schemas.push_back(&data_table))) { LOG_WARN("fail to push back data table ptr", KR(ret), K(data_table_id)); } else if (ObSysTableChecker::is_sys_table_has_index(data_table_id)) { if (OB_FAIL(ObSysTableChecker::get_sys_table_index_tids(data_table_id, index_tids))) { LOG_WARN("fail to get sys index tids", KR(ret), K(data_table_id)); } else if (i + index_tids.count() >= tables.count() || index_tids.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys table's index should be next to its data table", KR(ret), K(i), "index_cnt", index_tids.count()); } else { for (int64_t j = 0; OB_SUCC(ret) && j < index_tids.count(); j++) { const ObTableSchema &index_schema = tables.at(i + j + 1); const int64_t index_id = index_schema.get_table_id(); if (index_id != index_tids.at(j) || data_table_id != index_schema.get_data_table_id()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys index schema order is not match", KR(ret), K(data_table_id), K(j), K(index_schema)); } else if (OB_FAIL(table_schemas.push_back(&index_schema))) { LOG_WARN("fail to push back index schema", KR(ret), K(index_id), K(data_table_id)); } } // end for } } if (OB_SUCC(ret) && is_system_table(data_table_id)) { uint64_t lob_meta_table_id = OB_INVALID_ID; uint64_t lob_piece_table_id = OB_INVALID_ID; if (OB_ALL_CORE_TABLE_TID == data_table_id) { // do nothing } else if (!get_sys_table_lob_aux_table_id(data_table_id, lob_meta_table_id, lob_piece_table_id)) { ret = OB_ENTRY_NOT_EXIST; LOG_WARN("fail to get_sys_table_lob_aux_table_id", KR(ret), K(data_table_id)); } else { int64_t meta_idx = -1; int64_t piece_idx = -1; for (int64_t k = i + 1; OB_SUCC(ret) && k < tables.count(); k++) { if (tables.at(k).get_table_id() == lob_meta_table_id) { meta_idx = k; } if (tables.at(k).get_table_id() == lob_piece_table_id) { piece_idx = k; } if (meta_idx != -1 && piece_idx != -1) { break; } } if (meta_idx == -1 || piece_idx == -1) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys table's lob table not matched", KR(ret), K(meta_idx), K(piece_idx), K(lob_piece_table_id), K(lob_meta_table_id), K(data_table_id)); } else { if (OB_FAIL(table_schemas.push_back(&tables.at(meta_idx)))) { LOG_WARN("fail to push back lob meta aux table ptr", KR(ret), K(meta_idx), K(data_table_id)); } else if (OB_FAIL(table_schemas.push_back(&tables.at(piece_idx)))) { LOG_WARN("fail to push back lob piece aux table ptr", KR(ret), K(piece_idx), K(data_table_id)); } } } } if (OB_FAIL(ret)) { // failed, bypass } else if (OB_FAIL(new_table_tablet_allocator.prepare(data_table))) { LOG_WARN("fail to prepare ls for index schema tablets"); } else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array( ls_id_array))) { LOG_WARN("fail to get ls id array", KR(ret)); } else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg( table_schemas, ls_id_array))) { LOG_WARN("fail to add create tablets of table", KR(ret), K(data_table), K(table_schemas)); } } } // end for if (FAILEDx(table_creator.execute())) { LOG_WARN("fail to execute creator", KR(ret), K(tenant_id)); } else { ALLOW_NEXT_LOG(); LOG_INFO("create tenant sys tables tablet", KR(ret), K(tenant_id)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; bool commit = OB_SUCC(ret); if (OB_SUCCESS != (temp_ret = trans.end(commit))) { ret = (OB_SUCC(ret)) ? temp_ret : ret; LOG_WARN("trans end failed", K(commit), K(temp_ret)); } } // finishing is always invoked for new table tablet allocator int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) { LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret)); } } LOG_INFO("[CREATE_TENANT] STEP 2.3. finish create sys table tablets", KR(ret), K(tenant_id), "cost", ObTimeUtility::fast_current_time() - start_time); return ret; } int ObDDLService::init_tenant_schema( const uint64_t tenant_id, const ObTenantSchema &tenant_schema, const share::ObTenantRole &tenant_role, common::ObIArray &tables, ObSysVariableSchema &sys_variable) { const int64_t start_time = ObTimeUtility::fast_current_time(); LOG_INFO("[CREATE_TENANT] STEP 2.4. start init tenant schemas", K(tenant_id)); int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_ISNULL(sql_proxy_) || OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service()) || OB_ISNULL(GCTX.lst_operator_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy), KP_(schema_service), KP(GCTX.lst_operator_)); } else { ObSchemaService *schema_service_impl = schema_service_->get_schema_service(); ObGlobalStatProxy global_stat_proxy(*sql_proxy_, tenant_id); // 1. init tenant global stat if (OB_SUCC(ret)) { const int64_t core_schema_version = OB_CORE_SCHEMA_VERSION + 1; const int64_t baseline_schema_version = OB_INVALID_VERSION; const int64_t snapshot_gc_ts = 0; if (OB_FAIL(global_stat_proxy.set_tenant_init_global_stat( core_schema_version, baseline_schema_version, snapshot_gc_ts))) { LOG_WARN("fail to set tenant init global stat", K(core_schema_version), K(baseline_schema_version)); } } // 2. init tenant schema if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_, true, true); const int64_t init_schema_version = tenant_schema.get_schema_version(); int64_t new_schema_version = OB_INVALID_VERSION; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); //FIXME:(yanmu.ztl) lock tenant's __all_core_table const int64_t refreshed_schema_version = 0; if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("fail to start trans", KR(ret), K(tenant_id)); } else if (OB_FAIL(create_sys_table_schemas(ddl_operator, trans, tables))) { LOG_WARN("fail to create sys tables", KR(ret), K(tenant_id)); } else if (is_user_tenant(tenant_id) && OB_FAIL(set_sys_ls_status(tenant_id))) { LOG_WARN("failed to set sys ls status", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_service_impl->gen_new_schema_version( tenant_id, init_schema_version, new_schema_version))) { } else if (OB_FAIL(ddl_operator.replace_sys_variable( sys_variable, new_schema_version, trans, OB_DDL_ALTER_SYS_VAR))) { LOG_WARN("fail to replace sys variable", KR(ret), K(sys_variable)); } else if (OB_FAIL(ddl_operator.init_tenant_env(tenant_schema, sys_variable, tenant_role, trans))) { LOG_WARN("init tenant env failed", KR(ret), K(tenant_role), K(tenant_schema)); } else if (OB_FAIL(ddl_operator.insert_tenant_merge_info(OB_DDL_ADD_TENANT_START, tenant_schema, trans))) { LOG_WARN("fail to insert tenant merge info", KR(ret), K(tenant_schema)); } else if (is_meta_tenant(tenant_id) && OB_FAIL(ObServiceEpochProxy::init_service_epoch(trans, tenant_id, 0/*freeze_service_epoch*/))) { LOG_WARN("fail to init service epoch", KR(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; bool commit = OB_SUCC(ret); if (OB_SUCCESS != (temp_ret = trans.end(commit))) { ret = (OB_SUCC(ret)) ? temp_ret : ret; LOG_WARN("trans end failed", K(commit), K(temp_ret)); } } ObLSInfo sys_ls_info; ObAddrArray addrs; if (FAILEDx(GCTX.lst_operator_->get( GCONF.cluster_id, tenant_id, SYS_LS, sys_ls_info))) { LOG_WARN("fail to get sys ls info by operator", KR(ret), K(tenant_id)); } else if (OB_FAIL(sys_ls_info.get_paxos_member_addrs(addrs))) { LOG_WARN("fail to get paxos member addrs", K(ret), K(tenant_id), K(sys_ls_info)); } else if (OB_FAIL(publish_schema(tenant_id, addrs))) { LOG_WARN("fail to publish schema", KR(ret), K(tenant_id), K(addrs)); } } // 3. set baseline schema version if (OB_SUCC(ret)) { ObRefreshSchemaStatus schema_status; schema_status.tenant_id_ = tenant_id; int64_t baseline_schema_version = OB_INVALID_VERSION; if (OB_FAIL(schema_service_->get_schema_version_in_inner_table( *sql_proxy_, schema_status, baseline_schema_version))) { LOG_WARN("fail to gen new schema version", KR(ret), K(schema_status)); } else if (OB_FAIL(global_stat_proxy.set_baseline_schema_version(baseline_schema_version))) { LOG_WARN("fail to set baseline schema version", KR(ret), K(tenant_id), K(baseline_schema_version)); } } } LOG_INFO("[CREATE_TENANT] STEP 2.4. finish init tenant schemas", KR(ret), K(tenant_id), "cost", ObTimeUtility::fast_current_time() - start_time); return ret; } int ObDDLService::create_sys_table_schemas( ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, common::ObIArray &tables) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_ISNULL(sql_proxy_) || OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy), KP_(schema_service)); } else { // persist __all_core_table's schema in inner table, which is only used for sys views. for (int64_t i = 0; OB_SUCC(ret) && i < tables.count(); i++) { ObTableSchema &table = tables.at(i); const int64_t table_id = table.get_table_id(); const ObString &table_name = table.get_table_name(); const ObString *ddl_stmt = NULL; bool need_sync_schema_version = !(ObSysTableChecker::is_sys_table_index_tid(table_id) || is_sys_lob_table(table_id)); if (OB_FAIL(ddl_operator.create_table(table, trans, ddl_stmt, need_sync_schema_version, false /*is_truncate_table*/))) { LOG_WARN("add table schema failed", KR(ret), K(table_id), K(table_name)); } else { LOG_INFO("add table schema succeed", K(i), K(table_id), K(table_name)); } } } return ret; } int ObDDLService::set_sys_ls_status(const uint64_t tenant_id) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id || !is_user_tenant(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id)); } else { share::ObLSAttr new_ls; share::ObLSFlag flag;//TODO int64_t ls_group_id = 0; const int64_t create_scn = OB_LS_MIN_SCN_VALUE; share::ObLSAttrOperator ls_operator(tenant_id, sql_proxy_); if (OB_FAIL(new_ls.init(SYS_LS, ls_group_id, flag, share::OB_LS_NORMAL, share::OB_LS_OP_CREATE_END, create_scn))) { LOG_WARN("failed to init new operation", KR(ret), K(flag), K(create_scn)); } else if (OB_FAIL(ls_operator.insert_ls(new_ls, ls_group_id))) { LOG_WARN("failed to insert new ls", KR(ret), K(new_ls), K(ls_group_id)); } } return ret; } int ObDDLService::create_tenant_end(const uint64_t tenant_id) { const int64_t start_time = ObTimeUtility::fast_current_time(); LOG_INFO("[CREATE_TENANT] STEP 3. start create tenant end", K(tenant_id)); int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTenantSchema *tenant_schema = NULL; ObAllTenantInfo tenant_info; int64_t sys_schema_version = OB_INVALID_VERSION; ObDDLSQLTransaction trans(schema_service_); DEBUG_SYNC(BEFORE_CREATE_TENANT_END); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_FAIL(ObAllTenantInfoProxy::load_tenant_info( tenant_id, sql_proxy_, false, tenant_info))) { LOG_WARN("failed to load tenant info", KR(ret), K(tenant_id)); } else if (OB_INVALID_TENANT_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant_id", K(ret), K(tenant_id)); } else if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service is null", K(ret), KP_(schema_service)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, sys_schema_version))) { LOG_WARN("fail to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, sys_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(sys_schema_version)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id)); } else if (OB_ISNULL(tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", K(ret), K(tenant_id)); } else if (tenant_schema->is_normal()) { // skip, Guaranteed reentrant } else if (!tenant_schema->is_creating() && !tenant_schema->is_restore()) { ret = OB_STATE_NOT_MATCH; LOG_WARN("state not match", K(ret), K(tenant_id)); } else { ObTenantSchema new_tenant_schema = *tenant_schema; ObDDLSQLTransaction tenant_trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = OB_INVALID_VERSION; if (!tenant_info.is_standby()) { // Push the system tenant schema_version, and the standalone cluster may fail due to unsynchronized heartbeat. // The standalone cluster uses the synchronized schema_version, // and there is no need to increase the system tenant schema_version. int64_t new_schema_version = OB_INVALID_VERSION; ObSchemaService *schema_service_impl = schema_service_->get_schema_service(); // Ensure that the schema_version monotonically increases among tenants' cross-tenant transactions // https://aone.alibaba-inc.com/issue/20970156 if (OB_ISNULL(schema_service_impl)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service_impl is null", K(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("fail to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(tenant_trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { refreshed_schema_version = sys_schema_version > refreshed_schema_version ? sys_schema_version : refreshed_schema_version; if (OB_FAIL(schema_service_impl->gen_new_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version, new_schema_version))) { LOG_WARN("fail to gen new schema_version", K(ret)); } } } if (OB_SUCC(ret)) { const ObString *ddl_stmt_str_ptr = NULL; const int64_t DDL_STR_BUF_SIZE = 128; char ddl_str_buf[DDL_STR_BUF_SIZE]; MEMSET(ddl_str_buf, 0, DDL_STR_BUF_SIZE); ObString ddl_stmt_str; if (tenant_schema->is_restore()) { int64_t gts_value = OB_INVALID_TIMESTAMP; int64_t pos = 0; if (OB_FAIL(get_tenant_external_consistent_ts(tenant_id, gts_value))) { SERVER_LOG(WARN, "failed to get_tenant_gts", KR(ret), K(tenant_id)); } else if (OB_FAIL(databuff_printf(ddl_str_buf, DDL_STR_BUF_SIZE, pos, "schema_version=%ld; tenant_gts=%ld", refreshed_schema_version, gts_value))) { SERVER_LOG(WARN, "failed to construct ddl_stmt_str", KR(ret), K(tenant_id), K(refreshed_schema_version), K(gts_value)); } else { ddl_stmt_str.assign_ptr(ddl_str_buf, pos); ddl_stmt_str_ptr = &ddl_stmt_str; } } if (OB_FAIL(ret)) { } else if (OB_FAIL(ddl_operator.create_tenant(new_tenant_schema, OB_DDL_ADD_TENANT_END, trans, ddl_stmt_str_ptr))) { LOG_WARN("create tenant failed", K(new_tenant_schema), K(ret)); } else {/*do nothing*/} } if (OB_SUCC(ret)) { ret = E(EventTable::EN_CREATE_TENANT_TRANS_THREE_FAILED) OB_SUCCESS; } int temp_ret = OB_SUCCESS; if (trans.is_started()) { LOG_INFO("end create tenant", "is_commit", OB_SUCC(ret), K(ret)); if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { ret = (OB_SUCC(ret)) ? temp_ret : ret; LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); } } if (tenant_trans.is_started()) { int temp_ret = OB_SUCCESS; const bool is_commit = false;//no need commit, only for check and lock if (OB_SUCCESS != (temp_ret = tenant_trans.end(is_commit))) { ret = (OB_SUCC(ret)) ? temp_ret : ret; LOG_WARN("trans end failed", KR(ret), KR(temp_ret), K(is_commit)); } } if (OB_SUCC(ret)) { if (OB_SUCCESS != (temp_ret = publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish schema failed", K(temp_ret)); } } } LOG_INFO("[CREATE_TENANT] STEP 3. finish create tenant end", KR(ret), K(tenant_id), "cost", ObTimeUtility::fast_current_time() - start_time); return ret; } int ObDDLService::commit_alter_tenant_locality( const rootserver::ObCommitAlterTenantLocalityArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTenantSchema *orig_tenant_schema = NULL; const ObTenantSchema *orig_meta_tenant_schema = NULL; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_UNLIKELY(!arg.is_valid())) { LOG_WARN("invalid argument", KR(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(arg.tenant_id_, orig_tenant_schema))) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", KR(ret), "tenant_id", arg.tenant_id_); } else if (OB_UNLIKELY(NULL == orig_tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", KR(ret), "tenant_id", arg.tenant_id_); } else if (OB_UNLIKELY(orig_tenant_schema->get_locality_str().empty()) || OB_UNLIKELY(orig_tenant_schema->get_previous_locality_str().empty())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant locality status error", KR(ret), "tenant_id", orig_tenant_schema->get_tenant_id(), "tenant locality", orig_tenant_schema->get_locality_str(), "tenant previous locality", orig_tenant_schema->get_previous_locality_str()); } else { // deal with meta tenant related to certain user tenant if (is_user_tenant(arg.tenant_id_)) { if (OB_FAIL(schema_guard.get_tenant_info(gen_meta_tenant_id(arg.tenant_id_), orig_meta_tenant_schema))) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("meta tenant not exist", KR(ret), "meta_tenant_id", gen_meta_tenant_id(arg.tenant_id_)); } else if (OB_UNLIKELY(NULL == orig_meta_tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("meta tenant not exist", KR(ret), "meta_tenant_id", gen_meta_tenant_id(arg.tenant_id_)); } else if (OB_UNLIKELY(orig_meta_tenant_schema->get_locality_str().empty()) || OB_UNLIKELY(orig_meta_tenant_schema->get_previous_locality_str().empty())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("meta tenant locality status error", KR(ret), "meta tenant_id", orig_meta_tenant_schema->get_tenant_id(), "meta tenant locality", orig_meta_tenant_schema->get_locality_str(), "meta tenant previous locality", orig_meta_tenant_schema->get_previous_locality_str()); } } if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("fail to start transaction", KR(ret), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObTenantSchema new_tenant_schema; ObTenantSchema new_meta_tenant_schema; // refresh sys/user tenant schema if (OB_FAIL(new_tenant_schema.assign(*orig_tenant_schema))) { LOG_WARN("fail to assign tenant schema", KR(ret), KPC(orig_tenant_schema)); } else if (OB_FAIL(new_tenant_schema.set_previous_locality(ObString::make_string("")))) { LOG_WARN("fail to set previous locality", KR(ret)); } else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant_schema, trans))) { LOG_WARN("fail to alter tenant", KR(ret), K(new_tenant_schema)); } else { // refresh meta tenant schema if (is_user_tenant(new_tenant_schema.get_tenant_id())) { if (OB_FAIL(new_meta_tenant_schema.assign(*orig_meta_tenant_schema))) { LOG_WARN("fail to assign meta tenant schema", KR(ret), KPC(orig_meta_tenant_schema)); } else if (OB_FAIL(new_meta_tenant_schema.set_previous_locality(ObString::make_string("")))) { LOG_WARN("fail to set meta tenant previous locality", KR(ret)); } else if (OB_FAIL(ddl_operator.alter_tenant(new_meta_tenant_schema, trans))) { LOG_WARN("fail to alter meta tenant", KR(ret)); } } if (OB_SUCC(ret)) { //do rs_job ObRsJobInfo job_info; if (OB_SUCC(RS_JOB_FIND(job_info, trans, "job_type", "ALTER_TENANT_LOCALITY", "job_status", "INPROGRESS", "tenant_id", arg.tenant_id_))) { // good, find job } else if (OB_SUCC(RS_JOB_FIND(job_info, trans, "job_type", "ROLLBACK_ALTER_TENANT_LOCALITY", "job_status", "INPROGRESS", "tenant_id", arg.tenant_id_))) { // good, find job } else { LOG_WARN("failed to find job", KR(ret), "tenant_id", arg.tenant_id_); } if (OB_SUCC(ret) && job_info.job_id_ > 0) { if (OB_FAIL(RS_JOB_COMPLETE(job_info.job_id_, 0, trans))) { LOG_WARN("do rs_job update failed", K(ret), K(job_info)); } } } } int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", K(temp_ret), "is_commit", OB_SUCC(ret)); ret = (OB_SUCCESS == ret ? temp_ret : ret); } else {} // ok if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) {// force return success LOG_WARN("fail to publish schema", KR(ret)); } } } } } return ret; } int ObDDLService::check_alter_tenant_locality_type( share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObTenantSchema &orig_tenant_schema, const share::schema::ObTenantSchema &new_tenant_schema, AlterLocalityType &alter_locality_type) { int ret = OB_SUCCESS; alter_locality_type = ALTER_LOCALITY_INVALID; const uint64_t tenant_id = orig_tenant_schema.get_tenant_id(); const common::ObString &locality = orig_tenant_schema.get_locality_str(); const common::ObString &previous_locality = orig_tenant_schema.get_previous_locality_str(); const bool is_restore = new_tenant_schema.is_restore(); if (OB_UNLIKELY(locality.empty())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error, tenant locality is empty", K(ret), K(locality), K(tenant_id)); } else if (previous_locality.empty()) { // previous locality is null, need check locality of tablegroup in tenant alter_locality_type = TO_NEW_LOCALITY; } else { // The previous locality is not empty, the tenant is undergoing locality changes // Currently, it is allowed to roll back the locality that is being changed. // Rollback is currently defined as being set to be exactly the same as the original locality. if (previous_locality != new_tenant_schema.get_locality_str()) { alter_locality_type = ALTER_LOCALITY_INVALID; } else { // locality from 1->2, then rollback 2->1, The bottom layer does not support it, i // it should be rejected // Check in try_rollback_modify_tenant_locality alter_locality_type = ROLLBACK_LOCALITY; } } return ret; } /* * The locality of tenant is changed in the following function. At present, * the locality settings of tenant and table have the following forms: * # describe * 1. The locality of the tenant must not be empty. The tenant locality upgraded from the version before 1.3 is empty * in the internal table, but when the schema is refreshed, the locality of the tenant will be filled in * as a full-featured replication of each zone. * 2. The locality of the table can be empty, which means that the locality of the tenant is inherited. * When the locality of the table is not empty, it means that it does not completely match the locality of the tenant; * # locality change semantics * 1. When the locality of a tenant changes, the distribution of replications of all tables whose locality is empty * under that tenant will change accordingly. When the locality of the tenant is changed for a table * whose locality is not empty, the distribution of the corresponding replication will not change. * 2. Alter table can change the distribution of replications of a table whose locality is not empty. * # Mutual restriction of tenant and table locality changes * 1. When the old round of tenant locality has not been changed, * the new round of tenant locality changes are not allowed to be executed. * 2. When the change of the table whose locality is not empty under tenant is not completed, * the change of tenant locality is not allowed to be executed. * 3. When the locality change of tenant is not initiated, the locality change of the table * whose locality is not empty is not allowed to be executed. * # Change rules * 1. One locality change is only allowed to do one of the operations of adding paxos, * subtracting paxos and paxos type conversion (paxos->paxos), paxos->non_paxos is regarded as subtracting paxos, * non_paxos->paxos is regarded as adding paxos; * 2. In a locality change: * 2.1. For adding paxos operation, orig_locality's paxos num >= majority(new_locality's paxos num); * 2.2. For subtracting paxos operation, new_locality's paxos num >= majority(orig_locality's paxos num); * 2.3. For converting paxos type operation, only one paxos type conversion is allowed for one locality change; * 3. For replication type conversion, the following constraints need to be met: * 3.1. For L-type replications, the replications other than F are not allowed to be converted to L, * and L is not allowed to be converted to other replication types; * 3.2. There will be no restrictions for the rest of the situation * 4. In particular, in a scenario where only one replication of paxos is added, * paxos num is allowed to go from 1 -> 2, but paxos num is not allowed to go from 2-> 1; * 5. Non_paxos replications can occur together with the above changes, and there is no limit to the number. * # after 1.4.7.1, the locality form of @region is no longer supported */ int ObDDLService::set_new_tenant_options( share::schema::ObSchemaGetterGuard &schema_guard, const ObModifyTenantArg &arg, share::schema::ObTenantSchema &new_tenant_schema, const share::schema::ObTenantSchema &orig_tenant_schema, AlterLocalityOp &alter_locality_op) { int ret = OB_SUCCESS; common::ObArray zones_in_pool; alter_locality_op = ALTER_LOCALITY_OP_INVALID; if (OB_FAIL(set_raw_tenant_options(arg, new_tenant_schema))) { LOG_WARN("fail to set raw tenant options", K(ret)); } else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) { common::ObArray zone_region_list; AlterLocalityType alter_locality_type = ALTER_LOCALITY_INVALID; bool tenant_pools_in_shrinking = false; common::ObArray resource_pool_names; if (new_tenant_schema.get_locality_str().empty()) { // It is not allowed to change the locality as an inherited attribute ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter locality to empty"); LOG_WARN("alter locality to empty is not allowed", K(ret)); } else if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr_ is null", K(ret), KP(unit_mgr_)); } else if (OB_FAIL(unit_mgr_->check_tenant_pools_in_shrinking( orig_tenant_schema.get_tenant_id(), tenant_pools_in_shrinking))) { LOG_WARN("fail to check tenant pools in shrinking", K(ret)); } else if (tenant_pools_in_shrinking) { ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter tenant locality when tenant pool is shrinking"); LOG_WARN("alter tenant locality not allowed", K(ret), K(orig_tenant_schema)); } else if (OB_FAIL(get_new_tenant_pool_zone_list( arg, new_tenant_schema, resource_pool_names, zones_in_pool, zone_region_list))) { LOG_WARN("fail to get new tenant pool zone list", K(ret)); } else if (OB_FAIL(new_tenant_schema.set_locality(arg.tenant_schema_.get_locality_str()))) { LOG_WARN("fail to set locality", K(ret)); } else if (OB_FAIL(parse_and_set_create_tenant_new_locality_options( schema_guard, new_tenant_schema, resource_pool_names, zones_in_pool, zone_region_list))) { LOG_WARN("fail to parse and set new locality option", K(ret)); } else if (OB_FAIL(check_alter_tenant_locality_type( schema_guard, orig_tenant_schema, new_tenant_schema, alter_locality_type))) { LOG_WARN("fail to check alter tenant locality allowed", K(ret)); } else if (ALTER_LOCALITY_INVALID == alter_locality_type) { ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter tenant locality when previous operation is in progress"); LOG_WARN("alter tenant locality not allowed", K(ret), K(orig_tenant_schema)); } else if (ROLLBACK_LOCALITY == alter_locality_type) { // Roll back the currently ongoing alter locality if (OB_FAIL(try_rollback_modify_tenant_locality( arg, new_tenant_schema, orig_tenant_schema, zones_in_pool, zone_region_list, alter_locality_op))) { LOG_WARN("fail to try rollback modify tenant locality", K(ret), K(new_tenant_schema), K(orig_tenant_schema)); } else {} // no more to do } else if (TO_NEW_LOCALITY == alter_locality_type) { if (OB_FAIL(try_modify_tenant_locality( arg, new_tenant_schema, orig_tenant_schema, zones_in_pool, zone_region_list, alter_locality_op))) { LOG_WARN("fail to try modify tenant locality", K(ret), K(new_tenant_schema), K(zones_in_pool)); } else {} // no more to do } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected alter locality type", K(ret), K(alter_locality_type)); } if (OB_SUCC(ret)) { common::ObArray pool_names; if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr_ is null", K(ret), KP(unit_mgr_)); } else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST)) { ret = get_pools(arg.pool_list_, pool_names); } else { ret = unit_mgr_->get_pool_names_of_tenant(new_tenant_schema.get_tenant_id(), pool_names); } if (OB_FAIL(ret)) { LOG_WARN("fail to get pool names", K(ret)); } else if (OB_FAIL(check_pools_unit_num_enough_for_schema_locality( pool_names, schema_guard, new_tenant_schema))) { LOG_WARN("pools unit num is not enough for locality", K(ret)); } else {} // no more to do } } else {} // locality do not changed, do nothing LOG_DEBUG("set new tenant options", K(arg), K(new_tenant_schema), K(orig_tenant_schema)); return ret; } int ObDDLService::try_alter_meta_tenant_schema( ObDDLOperator &ddl_operator, const obrpc::ObModifyTenantArg &arg, common::ObMySQLTransaction &trans, share::schema::ObSchemaGetterGuard &sys_schema_guard, const share::schema::ObTenantSchema &user_tenant_schema) { int ret = OB_SUCCESS; const uint64_t tenant_id = user_tenant_schema.get_tenant_id(); // only locality and primary_zone can be modified in meta_tenant bool meta_tenant_has_option_changed = arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY) || arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::PRIMARY_ZONE); if (is_meta_tenant(tenant_id) || is_sys_tenant(tenant_id)) { /* bypass, when this is a meta tenant, * alter meta tenant shall be invoked in the upper layer */ } else if (!meta_tenant_has_option_changed) { // do nothing LOG_INFO("nothing changed to this tenant", KR(ret), K(arg), K(tenant_id)); } else { const share::schema::ObTenantSchema *meta_tenant_schema = nullptr; const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id); share::schema::ObTenantSchema new_meta_tenant_schema; if (OB_FAIL(sys_schema_guard.get_tenant_info( meta_tenant_id, meta_tenant_schema))) { LOG_WARN("fail to get tenant schema", KR(ret), K(meta_tenant_id)); } else if (OB_UNLIKELY(nullptr == meta_tenant_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("meta_tenant_schema ptr is null", KR(ret), K(meta_tenant_id), KP(meta_tenant_schema)); } else if (OB_FAIL(new_meta_tenant_schema.assign( *meta_tenant_schema))) { LOG_WARN("fail to assign new meta tenant schema", KR(ret)); } else { if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) { common::ObArray user_zone_list; if (OB_FAIL(new_meta_tenant_schema.set_previous_locality( user_tenant_schema.get_previous_locality()))) { LOG_WARN("fail to set previous locality", KR(ret)); } else if (OB_FAIL(new_meta_tenant_schema.set_locality( user_tenant_schema.get_locality()))) { LOG_WARN("fail to set locality", KR(ret)); } else if (OB_FAIL(user_tenant_schema.get_zone_list(user_zone_list))) { LOG_WARN("fail to get zone list from user schema", KR(ret), K(user_zone_list)); } else if (OB_FAIL(new_meta_tenant_schema.set_zone_list(user_zone_list))) { LOG_WARN("fail to set zone list", KR(ret)); } else if (OB_FAIL(new_meta_tenant_schema.set_primary_zone(user_tenant_schema.get_primary_zone()))) { LOG_WARN("fail to set primary zone", KR(ret), "primary_zone", user_tenant_schema.get_primary_zone()); } } else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::PRIMARY_ZONE)) { if (OB_FAIL(new_meta_tenant_schema.set_primary_zone(user_tenant_schema.get_primary_zone()))) { LOG_WARN("fail to set primary zone", KR(ret), "primary_zone", user_tenant_schema.get_primary_zone()); } } } if (FAILEDx(ddl_operator.alter_tenant( new_meta_tenant_schema, trans, nullptr /* do not record ddl stmt str */))) { LOG_WARN("fail to alter meta tenant locality", KR(ret), K(meta_tenant_id)); } } return ret; } int ObDDLService::try_rollback_modify_tenant_locality( const obrpc::ObModifyTenantArg &arg, share::schema::ObTenantSchema &new_schema, const share::schema::ObTenantSchema &orig_schema, const common::ObIArray &zones_in_pool, const common::ObIArray &zone_region_list, AlterLocalityOp &alter_locality_op) { int ret = OB_SUCCESS; UNUSED(arg); alter_locality_op = ALTER_LOCALITY_OP_INVALID; ObArray alter_paxos_tasks; ObArray pre_zone_locality; common::ObArray cur_zone_locality; const ObString &previous_locality = orig_schema.get_locality_str(); if (new_schema.get_locality_str().empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant locality", K(ret)); } else if (orig_schema.get_previous_locality_str() != new_schema.get_locality_str()) { MODIFY_LOCALITY_NOT_ALLOWED(); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter tenant locality when the previous operation is in progress"); } else if (OB_FAIL(new_schema.get_zone_replica_attr_array(cur_zone_locality))) { LOG_WARN("fail to get zone replica attr array", K(ret)); } else { // In the following two cases, locality rollback will not succeed, so check is needed: // 1.Since the implementation is not yet supported, the two-way conversion between types is not supported, // for example, currently supports F->L, but does not support L->F // 2. Support paxos member number 1->2, but does not support paxos member number 2->1 ObLocalityDistribution locality_dist; int64_t pre_paxos_num = 0; // not used int64_t cur_paxos_num = 0; // not used bool non_paxos_locality_modified = false; if (OB_FAIL(locality_dist.init())) { LOG_WARN("fail to init locality dist", K(ret)); } else if (OB_FAIL(locality_dist.parse_locality( previous_locality, zones_in_pool, &zone_region_list))) { LOG_WARN("fail to parse locality", K(ret)); } else if (OB_FAIL(locality_dist.get_zone_replica_attr_array(pre_zone_locality))) { LOG_WARN("fail to get zone region replica num array", K(ret)); } else if (OB_FAIL(ObLocalityCheckHelp::check_alter_locality( pre_zone_locality, cur_zone_locality, alter_paxos_tasks, non_paxos_locality_modified, pre_paxos_num, cur_paxos_num))) { LOG_WARN("fail to check and get paxos replica task", K(ret), K(pre_zone_locality), K(cur_zone_locality)); } else if (0 < alter_paxos_tasks.count() || non_paxos_locality_modified) { if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::FORCE_LOCALITY)) { if (OB_FAIL(new_schema.set_previous_locality(""))) { LOG_WARN("fail to set previous locality", K(ret)); } } else { if (OB_FAIL(new_schema.set_previous_locality(orig_schema.get_locality_str()))) { LOG_WARN("fail to set previous locality", K(ret)); } } if (OB_SUCC(ret)) { alter_locality_op = ROLLBACK_ALTER_LOCALITY; } } } return ret; } int ObDDLService::generate_zone_list_by_locality( const ZoneLocalityIArray &zone_locality, const common::ObIArray &zone_region_list, common::ObArray &zone_list) const { int ret = OB_SUCCESS; zone_list.reset(); UNUSED(zone_region_list); common::ObArray tmp_zone_list; for (int64_t i = 0; OB_SUCC(ret) && i < zone_locality.count(); ++i) { const ObZoneReplicaAttrSet &zone_num_set = zone_locality.at(i); const ObIArray &zone_set = zone_num_set.zone_set_; if (OB_FAIL(append(tmp_zone_list, zone_set))) { LOG_WARN("fail to append zone set", K(ret)); } else {} // ok, go on next } if (OB_SUCC(ret)) { std::sort(tmp_zone_list.begin(), tmp_zone_list.end()); for (int64_t i = 0; OB_SUCC(ret) && i < tmp_zone_list.count(); ++i) { common::ObZone &this_zone = tmp_zone_list.at(i); if (0 == zone_list.count() || zone_list.at(zone_list.count() - 1) != this_zone) { if (OB_FAIL(zone_list.push_back(this_zone))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } else {} // duplicated zone, no need to push into. } } return ret; } /* * The locality change currently allows the following types of transformations: * 1. Increase the locality of @zone; for example "F@zone1"-->"F@zone1,F@zone2" * 2. Delete the locality of @zone; for example "F@zone1,F@zone2"-->"F@zone1" * 3. Modify the locality of @zone; for example "F@zone1"-->"L@zone1" * * The change of locality needs to meet the following restrictions at the same time: * 1. One locality change is only allowed to do one of the operations of * adding paxos, subtracting paxos and paxos type conversion (paxos->paxos), * 2. In a locality change: * 2.1. for adding paxos operation, orig_locality's paxos num >= majority(new_locality's paxos num); * 2.2. for subtracting paxos operation, new_locality's paxos num >= majority(orig_locality's paxos num); * 2.3. for converting operation, only one paxos type conversion is allowed for one locality change; * 3. For replication type conversion, the following constraints need to be met: * 3.1. For L-type replications, the replications other than F are not allowed to be converted to L, * and L is not allowed to be converted to other replication types; * 3.2 There will be no restrictions for the rest of the situation * In particular, in a scenario where only one replicaiton of paxos is added, * paxos num is allowed to go from 1 -> 2, but paxos num is not allowed to go from 2-> 1; * * for example * 1. F@z1,F@z2,F@z3 -> F@z1,L@z3,F@z4 : z3 has done a paxos type conversion, adding F to z4, and subtracting F from z2, * which does not meet condition 1; * 2. F@z1,F@z2,R@z3 -> F@z1,F@z2,F@z3,F@z4 : z3 and z4 plus F, does not meet condition 2.1 * 3. F@z1,F@z2,F@z3,F@z4 -> F@z1,F@z2,R@z3 : As z3 and z4 minus F, condition 2.2 is not met * 4. F@z1,F@z2,F@z3,F@z4,F@z5 -> F@z1,F@z2,F@z3,L@z4,L@z5 : Both z4 and z5 have done paxos type conversion * and do not meet condition 2.3 * 5. F@z1,F@z2,R@z3 -> F@z1,F@z2,L@z3 : do not meet condition 3.1 * 6. F@z1 -> F@z1,F@z2 : Meet special rules * 7. F@z1 -> F@z2,F@z3 : Subtract F, add two F, does not meet special rules * */ int ObDDLService::try_modify_tenant_locality( const ObModifyTenantArg &arg, share::schema::ObTenantSchema &new_tenant_schema, const share::schema::ObTenantSchema &orig_tenant_schema, const common::ObIArray &zones_in_pool, const common::ObIArray &zone_region_list, AlterLocalityOp &alter_locality_op) { int ret = OB_SUCCESS; UNUSED(zones_in_pool); UNUSED(arg); alter_locality_op = ALTER_LOCALITY_OP_INVALID; // after 1.4.7.1, The locality writing method of @region is not supported, only the scenario of @zone is considered here ObArray alter_paxos_tasks; ObArray pre_zone_locality; common::ObArray cur_zone_locality; const ObString &previous_locality = orig_tenant_schema.get_locality_str(); if (!orig_tenant_schema.get_previous_locality_str().empty()) { // Defensive check, go to this branch, orig_tenant_schema previous locality should be empty ret = OB_ERR_UNEXPECTED; LOG_WARN("previous locality is not empty", K(ret), "pre_locality", orig_tenant_schema.get_previous_locality_str()); } else if (OB_FAIL(new_tenant_schema.get_zone_replica_attr_array(cur_zone_locality))) { LOG_WARN("fail to get zone replica attr array", K(ret)); } else { ObLocalityDistribution locality_dist; int64_t pre_paxos_num = 0; // not used int64_t cur_paxos_num = 0; // not used bool non_paxos_locality_modified = false; if (OB_FAIL(locality_dist.init())) { LOG_WARN("fail to init locality dist", K(ret)); } else if (OB_FAIL(locality_dist.parse_locality( previous_locality, zones_in_pool, &zone_region_list))) { LOG_WARN("fail to parse locality", K(ret)); } else if (OB_FAIL(locality_dist.get_zone_replica_attr_array(pre_zone_locality))) { LOG_WARN("fail to get zone region replica num array", K(ret)); } else if (OB_FAIL(ObLocalityCheckHelp::check_alter_locality( pre_zone_locality, cur_zone_locality, alter_paxos_tasks, non_paxos_locality_modified, pre_paxos_num, cur_paxos_num))) { LOG_WARN("fail to check and get paxos replica task", K(ret), K(pre_zone_locality), K(cur_zone_locality)); } else if (0 < alter_paxos_tasks.count() || non_paxos_locality_modified) { if (OB_FAIL(new_tenant_schema.set_previous_locality( orig_tenant_schema.get_locality_str()))) { LOG_WARN("fail to set previous locality", K(ret)); } else { alter_locality_op = ALTER_LOCALITY; } } else { alter_locality_op = NOP_LOCALITY_OP; } } return ret; } int ObDDLService::get_zones_of_pools( const common::ObIArray &resource_pool_names, common::ObIArray &zones_in_pool) { int ret = OB_SUCCESS; common::ObArray temp_zones; zones_in_pool.reset(); if (OB_UNLIKELY(resource_pool_names.count() <= 0)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), "resource pool count", resource_pool_names.count()); } else if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit mgr is null", K(ret), KP(unit_mgr_)); } else if (OB_FAIL(unit_mgr_->get_zones_of_pools(resource_pool_names, temp_zones))) { LOG_WARN("get zones of pools failed", K(ret), K(resource_pool_names)); } else if (temp_zones.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("empty zone array", K(ret)); } else { std::sort(temp_zones.begin(), temp_zones.end()); FOREACH_X(zone, temp_zones, OB_SUCC(ret)) { if (OB_ISNULL(zone)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("zone is null", K(ret)); } else if (0 == zones_in_pool.count() || zones_in_pool.at(zones_in_pool.count() - 1) != *zone) { if (OB_FAIL(zones_in_pool.push_back(*zone))) { LOG_WARN("fail to push back", K(ret)); } else {} } else {} // duplicated zone, no need to push into } } return ret; } int ObDDLService::get_tenant_pool_zone_list( const share::schema::ObTenantSchema &tenant_schema, common::ObIArray &zones_in_pool) { int ret = OB_SUCCESS; common::ObArray resource_pool_names; zones_in_pool.reset(); uint64_t tenant_id = tenant_schema.get_tenant_id(); if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr_ is null", K(ret), KP(unit_mgr_)); } else if (OB_FAIL(unit_mgr_->get_pool_names_of_tenant(tenant_id, resource_pool_names))) { LOG_WARN("fail to get pools of tenant", K(ret)); } else if (OB_FAIL(get_zones_of_pools(resource_pool_names, zones_in_pool))) { LOG_WARN("fail to get zones of pools", K(ret)); } else {} // no more to do return ret; } // What we need to retrieve is the zone of all resource_pools under the tenant's name, // not just the zone_list of the tenant itself int ObDDLService::get_new_tenant_pool_zone_list( const ObModifyTenantArg &arg, const share::schema::ObTenantSchema &tenant_schema, common::ObIArray &resource_pool_names, common::ObIArray &zones_in_pool, common::ObIArray &zone_region_list) { int ret = OB_SUCCESS; zones_in_pool.reset(); zone_region_list.reset(); if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST)) { if (OB_FAIL(get_pools(arg.pool_list_, resource_pool_names))) { LOG_WARN("fail to get pools", K(ret), "pool_list", arg.pool_list_); } else {} // got pool names, ok } else { uint64_t tenant_id = tenant_schema.get_tenant_id(); if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr_ is null", K(ret), KP(unit_mgr_)); } else if (OB_FAIL(unit_mgr_->get_pool_names_of_tenant(tenant_id, resource_pool_names))) { LOG_WARN("fail to get pools of tenant", K(ret)); } else {} // got pool names, ok } if (OB_FAIL(ret)) { } else if (OB_FAIL(get_zones_of_pools(resource_pool_names, zones_in_pool))) { LOG_WARN("fail to get zones of pools", K(ret)); } else if (OB_FAIL(construct_zone_region_list(zone_region_list, zones_in_pool))) { LOG_WARN("fail to construct zone region list", K(ret)); } else {} // no more to do return ret; } int ObDDLService::set_raw_tenant_options( const ObModifyTenantArg &arg, ObTenantSchema &new_tenant_schema) { int ret = OB_SUCCESS; const ObTenantSchema &alter_tenant_schema = arg.tenant_schema_; //replace alter options for (int32_t i = ObModifyTenantArg::REPLICA_NUM; ret == OB_SUCCESS && i < ObModifyTenantArg::MAX_OPTION; ++i) { if (arg.alter_option_bitset_.has_member(i)) { switch (i) { case ObModifyTenantArg::REPLICA_NUM: { ret = OB_NOT_SUPPORTED; LOG_WARN("modify replica num is not supported!", K(i), K(ret)); break; } case ObModifyTenantArg::CHARSET_TYPE: { ret = OB_NOT_SUPPORTED; LOG_WARN("modify replica num is not supported!", K(i), K(ret)); break; } case ObModifyTenantArg::COLLATION_TYPE: { ret = OB_NOT_SUPPORTED; LOG_WARN("modify replica num is not supported!", K(i), K(ret)); break; } case ObModifyTenantArg::PRIMARY_ZONE: { new_tenant_schema.set_primary_zone(alter_tenant_schema.get_primary_zone()); break; } case ObModifyTenantArg::ZONE_LIST: { ret = OB_NOT_SUPPORTED; LOG_WARN("modify zone list is not supported!", K(i), K(ret)); break; } case ObModifyTenantArg::RESOURCE_POOL_LIST: { break; } case ObModifyTenantArg::READ_ONLY: { ret = OB_NOT_SUPPORTED; LOG_WARN("modify tenant readonly option not supported", K(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "modify tenant readonly option"); break; } case ObModifyTenantArg::COMMENT: { new_tenant_schema.set_comment(alter_tenant_schema.get_comment()); break; } case ObModifyTenantArg::LOCALITY: { // locality change is processed in try_modify_tenant_locality, skip break; } case ObModifyTenantArg::DEFAULT_TABLEGROUP: { if (OB_FAIL(new_tenant_schema.set_default_tablegroup_name( alter_tenant_schema.get_default_tablegroup_name()))) { LOG_WARN("failed to set default tablegroup name", K(ret)); } else if (OB_FAIL(set_default_tablegroup_id(new_tenant_schema))) { LOG_WARN("failed to set default tablegroup id", K(ret)); } break; } case ObModifyTenantArg::FORCE_LOCALITY: { // do nothing break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unknown option!", K(i)); } } } } return ret; } /* Modify the internal table related to the resource pool, and calculate the transformation of * the resource pool list of the alter tenant at the same time. Currently, only one is allowed to be added, * one resource pool is reduced or the resource pool remains unchanged. * input: * tenant_id: tenant_id corresponding to alter tenant * new_pool_list: The new resource pool list passed in by alter tenant * output: * grant: subtract resource pool: false; add resource pool: true * diff_pools: the diff from newresource pool list and old resource pool list. */ int ObDDLService::modify_and_cal_resource_pool_diff( common::ObISQLClient &client, common::ObIArray &new_ug_id_array, share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObTenantSchema &new_tenant_schema, const common::ObIArray &new_pool_list, bool &grant, common::ObIArray &diff_pools) { int ret = OB_SUCCESS; lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID; common::ObArray new_pool_name_list; common::ObArray old_pool_name_list; const uint64_t tenant_id = new_tenant_schema.get_tenant_id(); if (OB_UNLIKELY(OB_INVALID_ID == tenant_id) || OB_UNLIKELY(new_pool_list.count() <= 0)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(tenant_id), K(new_pool_list)); } else if (OB_FAIL(unit_mgr_->get_pool_names_of_tenant(tenant_id, old_pool_name_list))) { LOG_WARN("fail to get pool names of tenant", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_tenant_compat_mode(tenant_id, compat_mode))) { LOG_WARN("fail to get compat mode", K(ret)); } else if (OB_UNLIKELY(old_pool_name_list.count() <= 0)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("old pool name list null", K(ret), K(old_pool_name_list)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < new_pool_list.count(); ++i) { if (OB_FAIL(new_pool_name_list.push_back(new_pool_list.at(i).ptr()))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } if (OB_SUCC(ret)) { std::sort(new_pool_name_list.begin(), new_pool_name_list.end()); std::sort(old_pool_name_list.begin(), old_pool_name_list.end()); bool is_permitted = false; if (new_pool_name_list.count() == old_pool_name_list.count() + 1) { grant = true; if (OB_FAIL(cal_resource_pool_list_diff( new_pool_name_list, old_pool_name_list, diff_pools))) { LOG_WARN("fail to cal resource pool list diff", K(ret)); } else if (OB_FAIL(check_grant_pools_permitted( schema_guard, diff_pools, new_tenant_schema, is_permitted))) { LOG_WARN("fail to check grant pools permitted", K(ret)); } else if (!is_permitted) { ret = OB_NOT_SUPPORTED; LOG_WARN("fail to grant pool", K(ret), K(diff_pools)); } else if (OB_FAIL(unit_mgr_->grant_pools( client, new_ug_id_array, compat_mode, diff_pools, tenant_id))) { LOG_WARN("fail to grant pools", K(ret)); } } else if (new_pool_name_list.count() + 1 == old_pool_name_list.count()) { grant = false; if (OB_FAIL(cal_resource_pool_list_diff( old_pool_name_list, new_pool_name_list, diff_pools))) { LOG_WARN("fail to cal resource pool list diff", K(ret)); } else if (OB_FAIL(check_revoke_pools_permitted( schema_guard, new_pool_name_list, new_tenant_schema, is_permitted))) { LOG_WARN("fail to check revoke pools permitted", K(ret)); } else if (!is_permitted) { ret = OB_OP_NOT_ALLOW; LOG_WARN("revoking resource pools is not allowed", K(ret), K(diff_pools)); } else if (OB_FAIL(unit_mgr_->revoke_pools( client, new_ug_id_array, diff_pools, tenant_id))) { LOG_WARN("fail to revoke pools", K(ret)); } else {} // no more to do } else if (new_pool_name_list.count() == old_pool_name_list.count()) { for (int64_t i = 0; OB_SUCC(ret) && i < new_pool_name_list.count(); ++i) { if (new_pool_name_list.at(i) != old_pool_name_list.at(i)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(new_pool_name_list), K(old_pool_name_list)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list"); } } } else { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(new_pool_name_list), K(old_pool_name_list)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list"); } } LOG_INFO("cal resource pool list result", K(new_pool_name_list), K(old_pool_name_list), K(diff_pools), K(grant)); } return ret; } int ObDDLService::check_grant_pools_permitted( share::schema::ObSchemaGetterGuard &schema_guard, const common::ObIArray &to_be_grant_pools, const share::schema::ObTenantSchema &tenant_schema, bool &is_permitted) { int ret = OB_SUCCESS; UNUSED(schema_guard); const uint64_t tenant_id = tenant_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_GTS_TENANT_ID == tenant_id) { // gts tenant, pass is_permitted = true; } else { if (OB_UNLIKELY(nullptr == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit mgr ptr is null", K(ret)); } else if (OB_FAIL(unit_mgr_->check_locality_for_logonly_unit( tenant_schema, to_be_grant_pools, is_permitted))) { LOG_WARN("fail to check locality for logonly unit", K(ret)); } } return ret; } int ObDDLService::check_revoke_pools_permitted( share::schema::ObSchemaGetterGuard &schema_guard, const common::ObIArray &new_pool_name_list, const share::schema::ObTenantSchema &tenant_schema, bool &is_permitted) { int ret = OB_SUCCESS; const uint64_t tenant_id = tenant_schema.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_GTS_TENANT_ID == tenant_id) { if (OB_FAIL(check_gts_tenant_revoke_pools_permitted( schema_guard, new_pool_name_list, tenant_schema, is_permitted))) { LOG_WARN("fail to check gts tenant revoke pools permitted", K(ret)); } } else { if (OB_FAIL(check_normal_tenant_revoke_pools_permitted( schema_guard, new_pool_name_list, tenant_schema, is_permitted))) { LOG_WARN("fail to check normal tenant revoke pools permitted", K(ret)); } } return ret; } int ObDDLService::check_gts_tenant_revoke_pools_permitted( share::schema::ObSchemaGetterGuard &schema_guard, const common::ObIArray &new_pool_name_list, const share::schema::ObTenantSchema &tenant_schema, bool &is_permitted) { int ret = OB_SUCCESS; UNUSED(schema_guard); UNUSED(new_pool_name_list); UNUSED(tenant_schema); UNUSED(is_permitted); is_permitted = false; // TODO: wenduo return ret; } int ObDDLService::check_normal_tenant_revoke_pools_permitted( share::schema::ObSchemaGetterGuard &schema_guard, const common::ObIArray &new_pool_name_list, const share::schema::ObTenantSchema &tenant_schema, bool &is_permitted) { int ret = OB_SUCCESS; is_permitted = true; common::ObArray zone_list; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr_ ptr is null", K(ret)); } else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) { LOG_WARN("fail to get zones of pools", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < zone_list.count() && is_permitted; ++i) { const common::ObZone &zone = zone_list.at(i); int64_t total_unit_num = 0; int64_t full_unit_num = 0; int64_t logonly_unit_num = 0; bool enough = false; if (OB_FAIL(unit_mgr_->get_zone_pools_unit_num( zone, new_pool_name_list, total_unit_num, full_unit_num, logonly_unit_num))) { LOG_WARN("fail to get pools unit num", K(ret)); } else if (total_unit_num != full_unit_num + logonly_unit_num) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit num value not match", K(ret), K(total_unit_num), K(full_unit_num), K(logonly_unit_num)); } else if (!tenant_schema.get_previous_locality_str().empty()) { is_permitted = false; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "revoking resource pools when tenant in locality modification"); } else if (OB_FAIL(unit_mgr_->check_schema_zone_unit_enough( zone, total_unit_num, full_unit_num, logonly_unit_num, tenant_schema, schema_guard, enough))) { LOG_WARN("fail to check schema zone unit enough", K(ret)); } else if (!enough) { is_permitted = false; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "revoking resource pools with tenant locality on"); } else { /* good */ } } } return ret; } /* long_pool_name_list and short_pool_name_list has sorted * in parameter condition: * The length of long_pool_name_list is 1 larger than the length of short_pool_name_list * This function has two functions: * 1 check whether long_pool_name_list is only one more resource_pool_name than short_pool_name_list * 2 Put this extra resource_pool_name into the diff_pools array. */ int ObDDLService::cal_resource_pool_list_diff( const common::ObIArray &long_pool_name_list, const common::ObIArray &short_pool_name_list, common::ObIArray &diff_pools) { int ret = OB_SUCCESS; if (long_pool_name_list.count() != short_pool_name_list.count() + 1) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(long_pool_name_list), K(short_pool_name_list)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list"); } else { diff_pools.reset(); int64_t index = 0; for (; OB_SUCC(ret) && index < short_pool_name_list.count(); ++index) { if (short_pool_name_list.at(index) != long_pool_name_list.at(index)) { if (OB_FAIL(diff_pools.push_back(long_pool_name_list.at(index)))) { LOG_WARN("fail to push back", K(ret)); } else { break; // got it, exit loop } } else {} // still the same, go on next } if (OB_FAIL(ret)) { } else if (index >= short_pool_name_list.count()) { // The pool of diff is the last element of long_pool_name_list if (index >= long_pool_name_list.count()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid resource pool list", K(ret)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list"); } else if (OB_FAIL(diff_pools.push_back(long_pool_name_list.at(index)))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } else { // The pool of diff is not the last element of long_pool_name_list. The diff has been found in the previous for loop. // It is necessary to further check whether short_pool_name_list and long_pool_name_list are consistent after index. for (; OB_SUCC(ret) && index < short_pool_name_list.count(); ++index) { if (index + 1 >= long_pool_name_list.count()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid resource pool list", K(ret)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list"); } else if (short_pool_name_list.at(index) != long_pool_name_list.at(index + 1)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid resource pool list", K(ret), K(short_pool_name_list), K(long_pool_name_list)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list"); } else {} // go on next } } } return ret; } /* * After the schema is split, there are two scenarios for cross-tenant transactions involved in modify_tenant: * * Scenario 1: Modify tenant option and system variable at the same time through alter_tenant. * For this scenario, the following restrictions are introduced: * 1. It is not allowed to modify tenant option and system variable at the same time. * 2. For redundant system variables in the tenant schema and system variable schema, * the synchronization of the two will no longer be guaranteed in the future * - read only: For the read only attribute, in order to avoid the failure of inner sql to write user tenant system tables, * inner sql skips the read only check. For external SQL, the read only attribute is subject to the system variable; * - name_case_mode: This value is specified when the tenant is created. It is a read only system variable * (lower_case_table_names), and subsequent modifications are not allowed; * - ob_compatibility_mode: This value needs to be specified when the tenant is created. * It is a read only system variable and cannot be modified later. * Scenario 2: * When the tenant locality is modified, the primary_zone is set in database/tablegroup/table * and the locality of the tablegroup/table adopts inherited semantics, there will be a scenario * where the primary_zone does not match the locality. In this case, it need to modify the primary_zone * of each database object under the tenant through DDL. * * After the schema is split, in order to avoid cross-tenant transactions, the process is split into two transactions. * The first transaction modifies the primary_zone of the database object under the tenant, and the second transaction * modifies the tenant schema. DDL failed, manual intervention to modify the schema. */ int ObDDLService::modify_tenant(const ObModifyTenantArg &arg) { LOG_INFO("receive modify tenant request", K(arg)); int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTenantSchema *orig_tenant_schema = NULL; const ObString &tenant_name = arg.tenant_schema_.get_tenant_name(); bool is_restore = false; bool is_standby = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("failed to get is standby", K(ret)); } else if (0 != arg.sys_var_list_.count() && !arg.alter_option_bitset_.is_empty()) { // After the schema is split, because __all_sys_variable is split under the tenant, in order to avoid // cross-tenant transactions, it is forbidden to modify the tenant option and the system variable at the same time. // For this reason, the read only column of the tenant option is no longer maintained, // and it is subject to system variables. ret = OB_OP_NOT_ALLOW; LOG_WARN("modify tenant option and system variable at the same time", K(ret), K(arg)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify tenant option and system variable at the same time"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_name, orig_tenant_schema))) { ret = OB_TENANT_NOT_EXIST; LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr()); LOG_WARN("tenant not exists", K(arg), K(ret)); } else if (OB_UNLIKELY(NULL == orig_tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr()); LOG_WARN("tenant not exists", K(arg), K(ret)); } else if (FALSE_IT(is_restore = orig_tenant_schema->is_restore())) { } else if (!is_restore) { // The physical recovery may be in the system table recovery stage, and it is necessary to avoid // the situation where SQL cannot be executed and hang if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table( orig_tenant_schema->get_tenant_id(), schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), "tenant_id", orig_tenant_schema->get_tenant_id()); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_name, orig_tenant_schema))) { ret = OB_TENANT_NOT_EXIST; LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr()); LOG_WARN("tenant not exists", K(arg), K(ret)); } else if (OB_UNLIKELY(NULL == orig_tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr()); LOG_WARN("tenant not exists", K(arg), K(ret)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(modify_tenant_inner_phase(arg, orig_tenant_schema, schema_guard, is_standby, is_restore))) { LOG_WARN("modify_tenant_inner_phase fail", K(ret)); } return ret; } int ObDDLService::modify_tenant_inner_phase(const ObModifyTenantArg &arg, const ObTenantSchema *orig_tenant_schema, ObSchemaGetterGuard &schema_guard, bool is_standby, bool is_restore) { int ret = OB_SUCCESS; if (OB_GTS_TENANT_ID == orig_tenant_schema->get_tenant_id()) { ret = OB_NOT_SUPPORTED; LOG_WARN("modify gts tenant not supported", KR(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter gts tenant"); } else if (0 != arg.sys_var_list_.count()) { // modify system variable const ObSysVariableSchema *orig_sys_variable = NULL; const uint64_t tenant_id = orig_tenant_schema->get_tenant_id(); int64_t schema_version = OB_INVALID_VERSION; ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); bool value_changed = false; if ((is_standby || is_restore) && is_user_tenant(tenant_id)) { ret = OB_OP_NOT_ALLOW; LOG_WARN("ddl operation is not allowed in standby cluster", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "ddl operation in standby cluster"); } else if (OB_FAIL(schema_guard.get_sys_variable_schema( orig_tenant_schema->get_tenant_id(), orig_sys_variable))) { LOG_WARN("get sys variable schema failed", K(ret)); } else if (OB_ISNULL(orig_sys_variable)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys variable schema is null", K(ret)); } else { ObSysVariableSchema new_sys_variable = *orig_sys_variable; new_sys_variable.reset_sysvars(); if (OB_FAIL(update_sys_variables(arg.sys_var_list_, *orig_sys_variable, new_sys_variable, value_changed))) { LOG_WARN("failed to update_sys_variables", K(ret)); } else if (value_changed == true) { int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, schema_version))) { LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id)); } else { const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR; if (OB_FAIL(ddl_operator.replace_sys_variable(new_sys_variable, schema_version, trans, operation_type, &arg.ddl_stmt_str_))) { LOG_WARN("failed to replace sys variable", K(ret), K(new_sys_variable)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (OB_SUCC(ret) && OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed, ", K(ret)); } } } } else if (!arg.alter_option_bitset_.is_empty()) { // modify tenant option const uint64_t tenant_id = orig_tenant_schema->get_tenant_id(); bool grant = true; ObArray diff_pools; AlterLocalityOp alter_locality_op = ALTER_LOCALITY_OP_INVALID; ObTenantSchema new_tenant_schema = *orig_tenant_schema; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (is_meta_tenant(tenant_id)) { ret = OB_NOT_SUPPORTED; LOG_WARN("not allowed to modify meta tenant's options manually", KR(ret), K(tenant_id)); } else if (OB_FAIL(set_new_tenant_options(schema_guard, arg, new_tenant_schema, *orig_tenant_schema, alter_locality_op))) { LOG_WARN("failed to set new tenant options", K(ret)); } else if (OB_FAIL(check_alter_tenant_replica_options( arg, new_tenant_schema, *orig_tenant_schema, schema_guard))) { LOG_WARN("check tenant replica options failed", K(new_tenant_schema), K(ret)); } // modify tenant option if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; common::ObArray new_ug_id_array; if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version)); } else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST) && OB_FAIL(modify_and_cal_resource_pool_diff( trans, new_ug_id_array, schema_guard, new_tenant_schema, arg.pool_list_, grant, diff_pools))) { LOG_WARN("fail to grant_pools", K(ret)); } if (OB_SUCC(ret) && !is_restore) { if (OB_FAIL(check_tenant_primary_zone_(schema_guard, new_tenant_schema))) { LOG_WARN("fail to check tenant primary zone", KR(ret), K(new_tenant_schema)); } } if (OB_FAIL(ret)) { } else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant_schema, trans, &arg.ddl_stmt_str_))) { LOG_WARN("failed to alter tenant", K(ret)); } else if (OB_FAIL(try_alter_meta_tenant_schema( ddl_operator, arg, trans, schema_guard, new_tenant_schema))) { LOG_WARN("failed to try alter meta tenant schema", KR(ret)); } if (OB_SUCC(ret) && arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) { if (OB_FAIL(record_tenant_locality_event_history(alter_locality_op, arg, new_tenant_schema, trans))) { LOG_WARN("fail to record tenant locality event history", K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (OB_SUCC(ret) && OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish schema failed, ", K(ret)); } // When the new and old resource pool lists are consistent, no diff is generated, diff_pools is empty, // and there is no need to call the following function if (OB_SUCC(ret) && arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST) && diff_pools.count() > 0) { if (OB_FAIL(unit_mgr_->commit_change_pool_owner( new_ug_id_array, grant, diff_pools, tenant_id))) { LOG_WARN("commit change pool owner failed", K(grant), K(diff_pools), K(tenant_id), K(ret)); } } } } else if (!arg.new_tenant_name_.empty()) { // rename tenant const uint64_t tenant_id = orig_tenant_schema->get_tenant_id(); const ObString new_tenant_name = arg.new_tenant_name_; ObTenantSchema new_tenant_schema = *orig_tenant_schema; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (is_meta_tenant(tenant_id)) { ret = OB_NOT_SUPPORTED; LOG_WARN("not allowed to modify meta tenant's options manually", KR(ret), K(tenant_id)); } else if (orig_tenant_schema->is_restore()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("rename tenant while tenant is in physical restore status is not allowed", KR(ret), KPC(orig_tenant_schema)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "rename tenant while tenant is in physical restore status is"); } else if (is_standby && is_user_tenant(orig_tenant_schema->get_tenant_id()) && !arg.is_sync_from_primary()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("rename user tenant in standby is not allowed", KR(ret), K(arg)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "rename user tenant in standby cluster"); } else if (orig_tenant_schema->get_tenant_id() <= OB_MAX_RESERVED_TENANT_ID) { ret = OB_NOT_SUPPORTED; LOG_WARN("rename special tenant not supported", K(ret), K(orig_tenant_schema->get_tenant_id())); LOG_USER_ERROR(OB_NOT_SUPPORTED, "rename special tenant"); } else if (NULL != schema_guard.get_tenant_info(new_tenant_name)) { ret = OB_TENANT_EXIST; LOG_USER_ERROR(OB_TENANT_EXIST, to_cstring(new_tenant_name)); LOG_WARN("tenant already exists", K(ret), K(new_tenant_name)); } else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version)); } else if (OB_FAIL(new_tenant_schema.set_tenant_name(new_tenant_name))) { LOG_WARN("failed to rename tenant", K(ret), K(new_tenant_name), K(new_tenant_schema)); } else if (OB_FAIL(ddl_operator.rename_tenant(new_tenant_schema, trans, &arg.ddl_stmt_str_))) { LOG_WARN("failed to rename tenant", K(ret), K(new_tenant_schema)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (OB_SUCC(ret) && OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish schema failed, ", K(ret)); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys variable or tenant option should changed", K(ret), K(arg)); } return ret; } // not used // When alter tenant, tenant option and sys variable are both set to readonly, // the current implementation is based on sys variable int ObDDLService::update_sys_variables(const common::ObIArray &sys_var_list, const share::schema::ObSysVariableSchema &orig_sys_variable, share::schema::ObSysVariableSchema &new_sys_variable, bool &value_changed) { int ret = OB_SUCCESS; bool found = false; value_changed = false; if (!sys_var_list.empty()) { const int64_t set_sys_var_count = sys_var_list.count(); const ObSysVarSchema *sysvar = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < set_sys_var_count; ++i) { const obrpc::ObSysVarIdValue &sysvar_value = sys_var_list.at(i); /* look ahead to find same variable, if found, jump this action. After doing so, only the rightmost set action can be accepted. */ found = false; for (int64_t j = i + 1; OB_SUCC(ret) && j < set_sys_var_count && (!found); ++j) { const obrpc::ObSysVarIdValue &tmp_var = sys_var_list.at(j); if (sysvar_value.sys_id_ == tmp_var.sys_id_) { found = true; } } if (OB_SUCC(ret) && !found) { if (OB_FAIL(orig_sys_variable.get_sysvar_schema(sysvar_value.sys_id_, sysvar))) { LOG_WARN("failed to get sysvar schema", K(sysvar_value), K(ret)); } else if (OB_ISNULL(sysvar)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sysvar is null", K(sysvar_value), K(ret)); } else { ObSysVarSchema new_sysvar; new_sysvar = *sysvar; if (SYS_VAR_OB_COMPATIBILITY_MODE == ObSysVarFactory::find_sys_var_id_by_name(new_sysvar.get_name())) { ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "change tenant compatibility mode"); } else if (new_sysvar.is_read_only()) { ret = OB_ERR_INCORRECT_GLOBAL_LOCAL_VAR; LOG_USER_ERROR(OB_ERR_INCORRECT_GLOBAL_LOCAL_VAR, new_sysvar.get_name().length(), new_sysvar.get_name().ptr(), (int)strlen("read only"), "read only"); } else if (new_sysvar.get_value() != sysvar_value.value_) { value_changed = true; if (OB_FAIL(new_sysvar.set_value(sysvar_value.value_))) { LOG_WARN("failed to set_value", K(ret)); } else if (OB_FAIL(new_sys_variable.add_sysvar_schema(new_sysvar))) { LOG_WARN("failed to add sysvar schema", K(ret)); } else { LOG_DEBUG("succ to update sys value", K(sysvar_value)); sysvar = NULL; } } } } } } return ret; } int ObDDLService::check_rename_object_type( share::schema::ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, const uint64_t database_id, const ObString &object_name, const ObTableSchema *&table_schema, const ObSynonymInfo *&synonym_info, const ObSequenceSchema *&sequence_schema, RenameOracleObjectType &obj_type) { int ret = OB_SUCCESS; uint64_t sequence_id; bool sequence_exist = false; bool is_system_generated = false; obj_type = RENAME_TYPE_INVALID; if (OB_FAIL(schema_guard.get_table_schema(tenant_id, database_id, object_name, false, table_schema))) { LOG_WARN("fail to get table schema", K(ret)); } else if (NULL != table_schema) { obj_type = RENAME_TYPE_TABLE_OR_VIEW; } if (FAILEDx(schema_guard.get_synonym_info(tenant_id, database_id, object_name, synonym_info))) { LOG_WARN("fail to get synonym info", K(ret)); } else if (NULL != synonym_info) { if (RENAME_TYPE_INVALID != obj_type) { ret = OB_ERR_UNEXPECTED; LOG_WARN("multi types of objects have the same object name", K(ret), K(object_name), K(obj_type)); } else { obj_type = RENAME_TYPE_SYNONYM; } } if (FAILEDx(schema_guard.check_sequence_exist_with_name(tenant_id, database_id, object_name, sequence_exist, sequence_id, is_system_generated))) { LOG_WARN("fail to check sequence exist", K(ret)); } else if (sequence_exist) { if (is_system_generated) { ret = OB_ERR_CANNOT_RENAME_SYSTEM_GENERATED_SEQUENCE; LOG_WARN("cannot rename system generated sequence", K(sequence_id), K(ret)); LOG_USER_ERROR(OB_ERR_CANNOT_RENAME_SYSTEM_GENERATED_SEQUENCE); } else if (OB_FAIL(schema_guard.get_sequence_schema(tenant_id, sequence_id, sequence_schema))) { LOG_WARN("fail get sequence schema", K(ret)); } else if (RENAME_TYPE_INVALID != obj_type) { ret = OB_ERR_UNEXPECTED; LOG_WARN("multi types of objects have the same object name", K(ret), K(object_name), K(obj_type)); } else { obj_type = RENAME_TYPE_SEQUENCE; } } if (OB_SUCC(ret) && RENAME_TYPE_INVALID == obj_type) { ret = OB_OBJECT_NAME_NOT_EXIST; LOG_WARN("object does not exist", K(ret)); } return ret; } int ObDDLService::record_tenant_locality_event_history( const AlterLocalityOp &alter_locality_op, const obrpc::ObModifyTenantArg &arg, const share::schema::ObTenantSchema &tenant_schema, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; if (ALTER_LOCALITY == alter_locality_op) { int64_t job_id = RS_JOB_CREATE(ALTER_TENANT_LOCALITY, trans, "tenant_name", tenant_schema.get_tenant_name(), "tenant_id", tenant_schema.get_tenant_id(), "sql_text", ObHexEscapeSqlStr(arg.ddl_stmt_str_), "extra_info", tenant_schema.get_previous_locality_str()); if (job_id < 1) { ret = OB_SQL_OPT_ERROR; LOG_WARN("insert into all_rootservice_job failed", K(ret), "tenant_id", tenant_schema.get_tenant_id()); } } else if (ROLLBACK_ALTER_LOCALITY == alter_locality_op) { ObRsJobInfo job_info; if (OB_SUCC(RS_JOB_FIND(job_info, trans, "job_type", "ALTER_TENANT_LOCALITY", "job_status", "INPROGRESS", "tenant_id", tenant_schema.get_tenant_id()))) { //good find job } else if (OB_SUCC(RS_JOB_FIND(job_info, trans, "job_type", "ROLLBACK_ALTER_TENANT_LOCALITY", "job_status", "INPROGRESS", "tenant_id", tenant_schema.get_tenant_id()))) { //good find job } else { LOG_WARN("failed to find job need rollback", K(ret), K(tenant_schema.get_tenant_id())); } if (OB_SUCC(ret) && job_info.job_id_ > 0) { if (OB_FAIL(RS_JOB_COMPLETE(job_info.job_id_, -1, trans))) {// The change task is rolled back, this change failed LOG_WARN("update rs_job failed", K(ret), "tenant_id", tenant_schema.get_tenant_id()); } } else { LOG_WARN("failed to find rs job", K(ret), "tenant_id", tenant_schema.get_tenant_id()); } if (OB_SUCC(ret)) { int64_t job_id = RS_JOB_CREATE(ROLLBACK_ALTER_TENANT_LOCALITY, trans, "tenant_name", tenant_schema.get_tenant_name(), "tenant_id", tenant_schema.get_tenant_id(), "sql_text", ObHexEscapeSqlStr(arg.ddl_stmt_str_), "extra_info", tenant_schema.get_locality_str()); if (job_id < 1) { ret = OB_SQL_OPT_ERROR; LOG_WARN("insert into all_rootservice_job failed", K(ret), "tenant_id", tenant_schema.get_tenant_id()); } } } else if (NOP_LOCALITY_OP == alter_locality_op) { int64_t job_id = RS_JOB_CREATE(ALTER_TENANT_LOCALITY, trans, "tenant_name", tenant_schema.get_tenant_name(), "tenant_id", tenant_schema.get_tenant_id(), "sql_text", ObHexEscapeSqlStr(arg.ddl_stmt_str_), "extra_info", tenant_schema.get_previous_locality_str()); if (job_id < 1) { ret = OB_SQL_OPT_ERROR; LOG_WARN("insert into all_rootservice_job failed", K(ret), "tenant_id", tenant_schema.get_tenant_id()); } else if (OB_FAIL(RS_JOB_COMPLETE(job_id, 0, trans))) {// The change task is rolled back, this change failed LOG_WARN("complete rs_job failed", K(ret), "tenant_id", tenant_schema.get_tenant_id()); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid alter locality op", K(ret), K(alter_locality_op)); } return ret; } /* * This interface includes 4 situations of primary and standalone cluster in total * primary cluster * drop tenant force: The tenant is forced to be deleted, with the highest priority. Variable identification: drop_force * drop tenant and recyclebin is enable: put tenant into recyclebin. Variable identification: to_recyclebin * drop tenant and recyclebin is disable or drop tenant purge: Both cases take the path of delayed deletion * Variable identification: delay_to_drop * The priority of the 3 variables is reduced, and there can only be one state at the same time * * standalone cluster * It is not allowed to initiate related drop tenant operations, all need to be synchronized from the primary cluster, * the following is the synchronized operation * * drop tenant force is consistent with the behavior of the primary cluster * drop tenant When the primary cluster recycle bin is opened, that is, the primary cluster puts the tenant * into the recycle bin, the standalone cluster will put the tenant into the recycle bin regardless of * whether the recycle bin is opened or not. * drop tenant: When the primary cluster closes the recycle bin or drop tenant purge, * the standalone cluster must take the path of delayed deletion regardless of whether the recycle bin is opened or not. * * meta tenant can only be force dropped with its user tenant. */ int ObDDLService::drop_tenant(const ObDropTenantArg &arg) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); const bool if_exist = arg.if_exist_; const bool drop_force = !arg.delay_to_drop_; const ObTenantSchema *tenant_schema = NULL; ObSchemaGetterGuard schema_guard; ObArray pool_names; ObArray pools; ObRootService *rootservice = GCTX.root_service_; ret = E(EventTable::EN_DROP_TENANT_FAILED) OB_SUCCESS; bool is_standby = false; uint64_t user_tenant_id = common::OB_INVALID_ID; int64_t refreshed_schema_version = 0; common::ObArray drop_ug_id_array; bool specify_tenant_id = OB_INVALID_TENANT_ID != arg.tenant_id_; if (OB_FAIL(ret)) { } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("failed to get is standby", K(ret)); } else if (OB_UNLIKELY(nullptr == rootservice)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("rootservice is null", K(ret), KP(rootservice)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else if (specify_tenant_id && OB_FAIL(schema_guard.get_tenant_info(arg.tenant_id_, tenant_schema))) { LOG_WARN("fail to gt tenant info", KR(ret), K(arg)); } else if (!specify_tenant_id && OB_FAIL(schema_guard.get_tenant_info(arg.tenant_name_, tenant_schema))) { LOG_WARN("fail to gt tenant info", KR(ret), K(arg)); } else if (OB_ISNULL(tenant_schema)) { if (if_exist) { LOG_USER_NOTE(OB_TENANT_NOT_EXIST, arg.tenant_name_.length(), arg.tenant_name_.ptr()); LOG_INFO("tenant not exist, no need to delete it", K(arg)); } else { ret = OB_TENANT_NOT_EXIST; LOG_USER_ERROR(OB_TENANT_NOT_EXIST, arg.tenant_name_.length(), arg.tenant_name_.ptr()); LOG_WARN("tenant not exist, can't delete it", K(arg), KR(ret)); } } else if (FALSE_IT(user_tenant_id = tenant_schema->get_tenant_id())) { } else if (!is_user_tenant(user_tenant_id)) { ret = OB_NOT_SUPPORTED; LOG_WARN("can't drop sys or meta tenant", KR(ret), K(user_tenant_id)); } else if (tenant_schema->is_in_recyclebin() && !drop_force) { ret = OB_TENANT_NOT_EXIST; LOG_USER_ERROR(OB_TENANT_NOT_EXIST, arg.tenant_name_.length(), arg.tenant_name_.ptr()); LOG_WARN("tenant in recyclebin, can't delete it", K(arg), KR(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(user_tenant_id), K(refreshed_schema_version)); } else { /* * drop tenant force: delay_to_drop_ is false * delay_to_drop_ is true in rest the situation * drop tenant && recyclebin enable: in recyclebin * (drop tenant && recyclebin disable) || drop tenant purge: delay delete */ bool open_recyclebin = arg.open_recyclebin_; if (is_standby && arg.object_name_.empty()) { // The delayed deletion of the standalone cluster synchronization is not affected by // whether the standalone cluster recycle bin is opened, but is controlled by the primary cluster // When the primary cluster is opened, it will enter the recycle bin, and use rpc // when the standalone cluster is synchronized. // When the primary cluster is closed, it will be deleted after a delay, and the recycle bin of // the standalone cluster is set to be closed here, keeping it consistent with the primary cluster // However, the operation of entering the recycle bin synchronized by the standalone cluster // does not need to change open_recyclebin, and distinguish whether to enter the recycle bin // through arg.object_name_.empty() open_recyclebin = false; } else { open_recyclebin = arg.open_recyclebin_; } const bool to_recyclebin = (arg.delay_to_drop_ && open_recyclebin); const bool delay_to_drop = (arg.delay_to_drop_ && !open_recyclebin); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); //1.drop tenant force if (drop_force) { const uint64_t meta_tenant_id = gen_meta_tenant_id(user_tenant_id); if (OB_FAIL(drop_resource_pool_pre( user_tenant_id, drop_ug_id_array, pool_names, is_standby, trans))) { LOG_WARN("fail to drop resource pool pre", KR(ret)); } else if (OB_FAIL(ddl_operator.drop_tenant(user_tenant_id, trans, &arg.ddl_stmt_str_))) { LOG_WARN("ddl_operator drop_tenant failed", K(user_tenant_id), KR(ret)); } else if (OB_FAIL(ddl_operator.drop_tenant(meta_tenant_id, trans))) { LOG_WARN("ddl_operator drop_tenant failed", K(meta_tenant_id), KR(ret)); } else if (OB_FAIL(try_drop_sys_ls_(meta_tenant_id, trans))) { LOG_WARN("failed to drop sys ls", KR(ret), K(meta_tenant_id)); } else if (tenant_schema->is_in_recyclebin()) { // try recycle record from __all_recyclebin ObArray recycle_objs; ObSchemaService *schema_service_impl = NULL; if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", KR(ret), KP_(schema_service)); } else if (FALSE_IT(schema_service_impl = schema_service_->get_schema_service())) { } else if (OB_FAIL(schema_service_impl->fetch_recycle_object( OB_SYS_TENANT_ID, tenant_schema->get_tenant_name_str(), ObRecycleObject::TENANT, trans, recycle_objs))) { LOG_WARN("get_recycle_object failed", KR(ret), KPC(tenant_schema)); } else if (0 == recycle_objs.size()) { // skip } else if (1 < recycle_objs.size()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("records should not be more than 1", KR(ret), KPC(tenant_schema), K(recycle_objs)); } else if (OB_FAIL(schema_service_impl->delete_recycle_object( OB_SYS_TENANT_ID, recycle_objs.at(0), trans))) { LOG_WARN("delete_recycle_object failed", KR(ret), KPC(tenant_schema)); } } } else {// put tenant into recyclebin ObTenantSchema new_tenant_schema = *tenant_schema; ObSqlString new_tenant_name; if (tenant_schema->is_restore() || tenant_schema->is_creating() || tenant_schema->is_dropping()) { // Due to the particularity of restore tenants, in order to avoid abnormal behavior of the cluster, // restore tenants cannot be placed in the recycle bin. // The creating state is the intermediate state of tenant creation, and it will become the normal state // if it is successfully created // The dropping state is the previous delayed deletion state. The two states are managed by the gc thread, // responsible for deletion and cannot be placed in the recycle bin. ret = OB_NOT_SUPPORTED; LOG_WARN("drop tenant to recyclebin is not supported", KR(ret), K(arg)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "should drop tenant force, delay drop tenant"); } else { if (!arg.object_name_.empty()) { //arg.object_name_ is not empty, it is the case that the standalone cluster synchronizes the primary cluster, //and the recycle bin name is synchronized from the primary cluster if (!is_standby) { ret = OB_ERR_UNEXPECTED; LOG_WARN("is not standby", K(ret)); } else if (OB_FAIL(new_tenant_name.assign(arg.object_name_))) { LOG_WARN("fail to assign", K(ret)); } } else { // Otherwise, the primary cluster generates the name of the recycle bin by itself if (OB_FAIL(ddl_operator.construct_new_name_for_recyclebin( new_tenant_schema, new_tenant_name))) { LOG_WARN("fail to construct new name", K(ret)); } } if (OB_SUCC(ret)) { if (to_recyclebin) { //2. tenant in recyclebin if (new_tenant_name.empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant name is null", K(ret)); } else if (OB_FAIL(ddl_operator.drop_tenant_to_recyclebin( new_tenant_name, new_tenant_schema, trans, &arg.ddl_stmt_str_))) { LOG_WARN("fail to drop tenant in recyclebin", KR(ret), K(user_tenant_id)); } } else if (delay_to_drop) { //3. tenant delay delete if (OB_FAIL(ddl_operator.delay_to_drop_tenant(new_tenant_schema, trans, &arg.ddl_stmt_str_))) { LOG_WARN("fail to delay_to drop tenant", K(ret)); } else { // ObLSManager will process force_drop_tenant() logic each 100ms. } } } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (drop_force) { if (OB_SUCC(ret) && OB_NOT_NULL(tenant_schema)) { if (OB_FAIL(drop_resource_pool_final( tenant_schema->get_tenant_id(), drop_ug_id_array, is_standby, pool_names))) { LOG_WARN("fail to drop resource pool finsl", KR(ret)); } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish schema failed", KR(ret)); } } LOG_INFO("drop tenant", K(arg), KR(ret)); return ret; } int ObDDLService::try_drop_sys_ls_(const uint64_t meta_tenant_id, common::ObMySQLTransaction &trans) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_UNLIKELY(!is_meta_tenant(meta_tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("not meta tenant", KR(ret), K(meta_tenant_id)); } else if (OB_ISNULL(sql_proxy_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sql proxy is null", KR(ret)); } else { //check ls exist in status ObLSLifeAgentManager life_agent(*sql_proxy_); ObLSStatusOperator ls_status; ObLSStatusInfo sys_ls_info; if (OB_FAIL(ls_status.get_ls_status_info(meta_tenant_id, SYS_LS, sys_ls_info, trans))) { if (OB_ENTRY_NOT_EXIST == ret) { ret = OB_SUCCESS; LOG_INFO("sys ls not exist, no need to drop", KR(ret), K(meta_tenant_id)); } else { LOG_WARN("failed to get ls status info", KR(ret), K(meta_tenant_id)); } } else if (OB_FAIL(life_agent.drop_ls_in_trans(meta_tenant_id, SYS_LS, trans))) { LOG_WARN("failed to drop ls in trans", KR(ret), K(meta_tenant_id)); } } return ret; } int ObDDLService::drop_resource_pool_pre(const uint64_t tenant_id, common::ObIArray &drop_ug_id_array, ObIArray &pool_names, const bool is_standby, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(unit_mgr_->get_pool_names_of_tenant(tenant_id, pool_names))) { LOG_WARN("get_pool_names_of_tenant failed", K(tenant_id), KR(ret)); } else if (OB_FAIL(unit_mgr_->revoke_pools(trans, drop_ug_id_array, pool_names, tenant_id))) { LOG_WARN("revoke_pools failed", K(pool_names), K(tenant_id), KR(ret)); } else if (is_standby) { if (OB_FAIL(unit_mgr_->drop_standby_resource_pool(pool_names, trans))) { LOG_WARN("failed to drop standby resource pool", KR(ret), K(pool_names)); } } return ret; } int ObDDLService::drop_resource_pool_final(const uint64_t tenant_id, common::ObIArray &drop_ug_id_array, const bool is_standby, ObIArray &pool_names) { int ret = OB_SUCCESS; const bool grant = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(unit_mgr_->commit_change_pool_owner( drop_ug_id_array, grant, pool_names, tenant_id))) { LOG_WARN("commit change pool owner failed", K(grant), K(pool_names), K(tenant_id), KR(ret)); } else if (is_standby) { // delete resource pool memery strut if (OB_FAIL(unit_mgr_->commit_drop_standby_resource_pool(pool_names))) { LOG_WARN("failed to drop standby resource pool", KR(ret), K(pool_names)); } } // delete from __all_schema_status // The update of __all_core_table must be guaranteed to be a single partition transaction, // so a separate transaction is required here. // The failure of the transaction will not affect it, but there is garbage data in __all_core_table. if (OB_SUCC(ret)) { int temp_ret = OB_SUCCESS; ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_; if (OB_ISNULL(schema_status_proxy)) { temp_ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_status_proxy is null", K(temp_ret)); } else if (OB_SUCCESS != (temp_ret = schema_status_proxy->del_tenant_schema_status(tenant_id))) { LOG_ERROR("del tenant schema status failed", KR(temp_ret), "tenant_id", tenant_id); } } return ret; } int ObDDLService::get_tenant_object_name_with_origin_name_in_recyclebin( const ObString &origin_tenant_name, ObString &object_name, common::ObIAllocator *allocator, const bool is_flashback) { int ret = OB_SUCCESS; ObSqlString sql; SMART_VAR(ObMySQLProxy::MySQLResult, res) { common::sqlclient::ObMySQLResult *result = NULL; const char *desc_or_asc = (is_flashback ? "desc" : "asc"); if (OB_FAIL(sql.append_fmt( "select object_name from oceanbase.__all_recyclebin where " "original_name = '%.*s' and TYPE = 7 order by gmt_create %s limit 1", origin_tenant_name.length(), origin_tenant_name.ptr(), desc_or_asc))) { LOG_WARN("failed to append sql", K(ret), K(origin_tenant_name), K(*desc_or_asc)); } else if (OB_FAIL(sql_proxy_->read(res, OB_SYS_TENANT_ID, sql.ptr()))) { LOG_WARN("failed to execute sql", K(sql), K(ret)); } else if (OB_ISNULL(result = res.get_result())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get result", K(ret)); } else if (OB_FAIL(result->next())) { if (OB_ITER_END == ret) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("origin tenant_name not exist in recyclebin", K(ret), K(sql)); } else { LOG_WARN("iterate next result fail", K(ret), K(sql)); } } else { ObString tmp_object_name; EXTRACT_VARCHAR_FIELD_MYSQL(*result, "object_name", tmp_object_name); if (OB_FAIL(deep_copy_ob_string(*allocator, tmp_object_name, object_name))) { LOG_WARN("failed to deep copy member list", K(ret), K(object_name)); } } if (OB_FAIL(ret)) { } else if (OB_ITER_END != result->next()) { // The result will not exceed one line ret = OB_ERR_UNEXPECTED; LOG_WARN("result failed", K(ret), K(sql)); } } return ret; } int ObDDLService::flashback_tenant(const obrpc::ObFlashBackTenantArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTenantSchema *tenant_schema = NULL; ObArenaAllocator allocator(ObModIds::OB_TENANT_INFO); ObString final_tenant_name; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (arg.tenant_id_ != OB_SYS_TENANT_ID) { ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "falshback tenant must in sys tenant"); LOG_WARN("falshback tenant must in sys tenant", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table( OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(arg.origin_tenant_name_, tenant_schema))) { LOG_WARN("failt to get tenant info", K(ret)); } else if (OB_ISNULL(tenant_schema)) { const bool is_flashback = true; ObString new_tenant_name; if (OB_FAIL(get_tenant_object_name_with_origin_name_in_recyclebin(arg.origin_tenant_name_, new_tenant_name, &allocator, is_flashback))) { LOG_WARN("fail to get tenant obfect name", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(new_tenant_name, tenant_schema))) { LOG_WARN("fail to get tenant info", K(ret)); } else if (OB_ISNULL(tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant name is not exist", K(ret)); } } if (OB_SUCC(ret)) { if (!tenant_schema->is_in_recyclebin()) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("tenant schema is not in recyclebin", K(ret), K(arg), K(*tenant_schema)); } else if (!arg.new_tenant_name_.empty()) { final_tenant_name = arg.new_tenant_name_; } else {}//nothing todo } if (OB_SUCC(ret)) { if (OB_FAIL(flashback_tenant_in_trans(*tenant_schema, final_tenant_name, schema_guard, arg.ddl_stmt_str_))) { LOG_WARN("flashback tenant in trans failed", K(ret)); } else if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish_schema failed", K(ret)); } } LOG_INFO("finish flashback tenant", K(arg), K(ret)); return ret; } int ObDDLService::flashback_tenant_in_trans(const share::schema::ObTenantSchema &tenant_schema, const ObString &new_tenant_name, share::schema::ObSchemaGetterGuard &schema_guard, const ObString &ddl_stmt_str) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.flashback_tenant_from_recyclebin(tenant_schema, trans, new_tenant_name, schema_guard, ddl_stmt_str))) { LOG_WARN("flashback tenant from recyclebin failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } return ret; } int ObDDLService::purge_tenant( const obrpc::ObPurgeTenantArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTenantSchema *tenant_schema = NULL; ObArenaAllocator allocator(ObModIds::OB_TENANT_INFO); ObArray pool_names; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check_inner_stat failed", K(ret)); } else if (arg.tenant_id_ != OB_SYS_TENANT_ID) { ret = OB_OP_NOT_ALLOW; LOG_USER_ERROR(OB_OP_NOT_ALLOW, "purge tenant must in sys tenant"); LOG_WARN("purge tenant must in sys tenant", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table( OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(arg.tenant_name_, tenant_schema))) { LOG_WARN("fail to get tenant info", K(ret)); } else if (OB_ISNULL(tenant_schema)) { const bool is_flashback = false; ObString new_tenant_name; if (OB_FAIL(get_tenant_object_name_with_origin_name_in_recyclebin(arg.tenant_name_, new_tenant_name, &allocator, is_flashback))) { LOG_WARN("fail to get tenant obfect name", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(new_tenant_name, tenant_schema))) { LOG_WARN("fail to get tenant info", K(ret)); } else if (OB_ISNULL(tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant name is not exist", K(ret)); } } if (OB_SUCC(ret)) { if (!tenant_schema->is_in_recyclebin()) { ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN; LOG_WARN("tenant not in recyclebin, can not be purge", K(arg), K(*tenant_schema), K(ret)); } } if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); const uint64_t tenant_id = tenant_schema->get_tenant_id(); bool is_standby = false; int64_t refreshed_schema_version = 0; if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("fail to get is standby", K(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.purge_tenant_in_recyclebin( *tenant_schema, trans, &arg.ddl_stmt_str_))) { LOG_WARN("purge tenant failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish_schema failed", K(ret)); } } } LOG_INFO("finish purge tenant", K(arg), K(ret)); return ret; } int ObDDLService::lock_tenant(const ObString &tenant_name, const bool is_lock) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; const ObTenantSchema *tenant_schema = NULL; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (tenant_name.length() <= 0) { ret = OB_INVALID_TENANT_NAME; LOG_WARN("invalid tenant name", K(tenant_name), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_name, tenant_schema)) || NULL == tenant_schema) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist, can't lock it", K(tenant_name), K(ret)); } else if (tenant_schema->get_locked() == is_lock) { ret = OB_SUCCESS; } else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObTenantSchema new_tenant_schema = *tenant_schema; new_tenant_schema.set_locked(is_lock); if (OB_FAIL(ddl_operator.alter_tenant(new_tenant_schema, trans))) { LOG_WARN("ddl_operator alter tenant failed", K(new_tenant_schema), K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::add_system_variable(const ObAddSysVarArg &arg) { LOG_INFO("receive add system variable request", K(arg)); DEBUG_SYNC(BEFORE_UPRADE_SYSTEM_VARIABLE); int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; const ObSysVarSchema *old_schema = NULL; const ObTenantSchema *tenant_info = NULL; const ObSysVariableSchema *sys_variable_schema = NULL; const ObString var_name = arg.sysvar_.get_name(); const uint64_t tenant_id = arg.sysvar_.get_tenant_id(); bool execute = true; ObSysVarSchema new_sys_var; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", KR(ret), K(arg)); } else if (OB_FAIL(new_sys_var.assign(arg.sysvar_))) { LOG_WARN("fail to assign sysvar", KR(ret), K(arg)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_info))) { LOG_WARN("get tenant info failed", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) { LOG_WARN("get sys variable schema failed", KR(ret)); } else if (OB_ISNULL(sys_variable_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys variable schema is null", KR(ret)); } // check sys var schema if (FAILEDx(sys_variable_schema->get_sysvar_schema(var_name, old_schema))) { if (!arg.update_sys_var_ && OB_ERR_SYS_VARIABLE_UNKNOWN == ret) { // add sys var, sys var should not exist ret = OB_SUCCESS; } else { LOG_WARN("get tenant system variable failed", KR(ret), K(arg)); } } else if (OB_ISNULL(old_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys var schema is null", KR(ret), K(arg)); } else if (!arg.update_sys_var_) { // case 1. add sys var, and sys var exist if (arg.if_not_exist_) { execute = false; } else { ret = OB_ERR_PARAM_DUPLICATE; LOG_WARN("system variable duplicated", KR(ret), K(var_name)); } } else { // upate sys var if (new_sys_var.is_equal_except_value(*old_schema)) { // case 2. new sys var is same with existed schema(except value), do nothing execute = false; } else if (OB_FAIL(new_sys_var.set_value(old_schema->get_value()))) { LOG_WARN("fail to set old value", KR(ret), K(new_sys_var), KPC(old_schema)); } } // execute add/update sys var if (OB_FAIL(ret)) { } else if (!execute) { // Reentrant update, do nothing in the following scenarios: // case 1. add sys var, sys var exist // case 2. update sys var, sys var is same with existed schema(except value) } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t schema_version = OB_INVALID_VERSION; ObSysVariableSchema new_sys_variable_schema; if (OB_UNLIKELY(!arg.is_valid())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("add sysvar argument is invalid", K(arg)); } else if (OB_FAIL(new_sys_variable_schema.assign(*sys_variable_schema))) { LOG_WARN("fail to ass new sys variable schema", KR(ret), KPC(sys_variable_schema)); } else if (FALSE_IT(new_sys_variable_schema.reset_sysvars())) { } else if (OB_FAIL(new_sys_variable_schema.add_sysvar_schema(new_sys_var))) { LOG_WARN("add sysvar schema to new tenant info failed", KR(ret)); } else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, schema_version))) { LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id)); } else { const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR; if (OB_FAIL(ddl_operator.replace_sys_variable(new_sys_variable_schema, schema_version, trans, operation_type, &arg.ddl_stmt_str_))) { LOG_WARN("alter tenant info failed", KR(ret)); } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", KR(ret)); } } return ret; } int ObDDLService::modify_system_variable(const ObModifySysVarArg &arg) { LOG_INFO("receive modify system variable request", K(arg)); int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; const ObTenantSchema *tenant_schema = NULL; const ObSysVariableSchema *sys_variable_schema = NULL; const uint64_t tenant_id = arg.tenant_id_; bool value_changed = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema)) || NULL == tenant_schema) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", K(tenant_id), K(ret)); } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) { LOG_WARN("get sys variable schema failed", K(ret)); } else if (OB_ISNULL(sys_variable_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys variable schema is null", K(ret)); } else { const ObSysVarSchema *old_schema = NULL; ObSysVarSchema new_schema; ObSysVariableSchema new_sys_variable_schema = *sys_variable_schema; if (OB_UNLIKELY(!new_sys_variable_schema.is_valid())) { ret = new_sys_variable_schema.get_err_ret(); LOG_WARN("new sys variable schema is invalid", K(ret)); } else { new_sys_variable_schema.reset_sysvars(); } for (int64_t i = 0; OB_SUCC(ret) && i < arg.sys_var_list_.count(); ++i) { const ObSysVarSchema &modify_var = arg.sys_var_list_.at(i); bool found = false; for (int64_t j = i + 1; !found && OB_SUCC(ret) && j < arg.sys_var_list_.count(); ++j) { const ObSysVarSchema &tmp_var = arg.sys_var_list_.at(j); if (modify_var.get_name() == tmp_var.get_name()) { found = true; } } if (OB_SUCC(ret) && !found) { if (OB_FAIL(sys_variable_schema->get_sysvar_schema(modify_var.get_name(), old_schema))) { LOG_WARN("get sysvar schema failed", K(ret), K(modify_var)); } else { new_schema = *old_schema; if (OB_UNLIKELY(!new_schema.is_valid())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("new schema is invalid", K(new_schema)); } else if(new_schema.get_value() != modify_var.get_value()) { value_changed = true; if(OB_FAIL(new_schema.set_value(modify_var.get_value()))) { LOG_WARN("set new schema value failed", K(ret)); } else if (OB_FAIL(new_sys_variable_schema.add_sysvar_schema(new_schema))) { LOG_WARN("add sysvar schema to new sys variable schema failed", K(ret)); } } else{ //new value == old value, no need to update sys var schema //do nothing } } } } if (OB_SUCC(ret) && value_changed == true) { bool is_oracle_mode = true; int64_t schema_version = OB_INVALID_VERSION; const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR; ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, schema_version))) { LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id)); } else if (OB_FAIL(ddl_operator.replace_sys_variable(new_sys_variable_schema, schema_version, trans, operation_type, &arg.ddl_stmt_str_))) { LOG_WARN("alter tenant info failed", K(ret)); } else if (OB_FAIL(sys_variable_schema->get_oracle_mode(is_oracle_mode))) { LOG_WARN("failed to get oracle mode", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } if (OB_SUCC(ret) && value_changed) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::create_database(const bool if_not_exist, share::schema::ObDatabaseSchema &database_schema, const ObString *ddl_stmt_str, ObMySQLTransaction *ora_user_trans) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); bool is_exist = false; uint64_t database_id = OB_INVALID_ID; const uint64_t tenant_id = database_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; ObCollationType tenant_collation_type = CS_TYPE_INVALID; ObCharsetType tenant_charset_type = CHARSET_INVALID; int64_t refreshed_schema_version = 0; LOG_INFO("database schema received", K(database_schema)); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_service_->check_database_exist(database_schema.get_tenant_id(), database_schema.get_database_name_str(), database_id, is_exist))) { LOG_WARN("check database exist failed", "database", database_schema.get_database_name(), K(ret)); } else { const ObString database_name = database_schema.get_database_name_str(); if (is_exist) { if (if_not_exist) { ret = OB_SUCCESS; database_schema.set_database_id(database_id); LOG_USER_NOTE(OB_DATABASE_EXIST, database_name.length(), database_name.ptr()); LOG_WARN("database already exists, not need to create", "tenant_id", database_schema.get_tenant_id(), "database_name", database_schema.get_database_name()); } else { ret = OB_DATABASE_EXIST; LOG_USER_ERROR(OB_DATABASE_EXIST, database_name.length(), database_name.ptr()); LOG_WARN("database already exists, ", "tenant_id", database_schema.get_tenant_id(), "database_name", database_schema.get_database_name(), K(ret)); } } else if (OB_FAIL(check_create_with_db_id(database_schema))) { LOG_WARN("not supported operator", K(ret), K(database_schema)); } else if (OB_FAIL(set_default_tablegroup_id(database_schema))) { LOG_WARN("set_tablegroup_id failed", "tablegroup name", database_schema.get_default_tablegroup_name(), K(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_ISNULL(ora_user_trans) && OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { if (OB_SUCC(ret)) { const ObSysVarSchema *collation_var = NULL; ObObj collation_obj; ObMalloc alloc(ObModIds::OB_TEMP_VARIABLES); if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id, SYS_VAR_COLLATION_SERVER, collation_var))) { LOG_WARN("fail to get tenant var schema", K(ret)); } else if (OB_ISNULL(collation_var)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to get charset_var or collation_var", K(ret)); } else if (OB_FAIL(collation_var->get_value(&alloc, NULL, collation_obj))) { LOG_WARN("fail to get charset var value", K(ret)); } else { tenant_collation_type = static_cast(collation_obj.get_int()); tenant_charset_type = ObCharset::charset_type_by_coll(tenant_collation_type); } } if (OB_SUCC(ret)) { // if zone_list, primary_zone not set, copy from tenant_schema const ObTenantSchema *tenant_schema = NULL; if (OB_FAIL(schema_guard.get_tenant_info( database_schema.get_tenant_id(), tenant_schema))) { LOG_WARN("tenant not exist in schema manager", "tenant id", database_schema.get_tenant_id(), K(ret)); } else if (OB_ISNULL(tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant is not exist", KR(ret), "tenant_id", database_schema.get_tenant_id()); } else if (OB_FAIL(ObSchema::set_charset_and_collation_options(tenant_charset_type, tenant_collation_type, database_schema))) { LOG_WARN("set charset and collation options failed", K(ret)); } else {} // ok if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.create_database(database_schema, OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans, ddl_stmt_str))) { LOG_WARN("create database failed", K(ret), K(database_schema)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } } } // publish schema if (OB_SUCC(ret)) { // Specify ora_user_trans, create user ddl, publish schema when create_user is completed if (NULL == ora_user_trans) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed, ", K(ret)); } } } return ret; } //set new database options to new database schema int ObDDLService::set_new_database_options(const ObAlterDatabaseArg &arg, ObDatabaseSchema &new_database_schema) { int ret = OB_SUCCESS; const ObDatabaseSchema &alter_database_schema = arg.database_schema_; //replace alter options for (int32_t i = ObAlterDatabaseArg::REPLICA_NUM; ret == OB_SUCCESS && i < ObAlterDatabaseArg::MAX_OPTION; ++i) { if (arg.alter_option_bitset_.has_member(i)) { switch (i) { case ObAlterDatabaseArg::REPLICA_NUM: { // ignore alter replica num break; } case ObAlterDatabaseArg::CHARSET_TYPE: case ObAlterDatabaseArg::COLLATION_TYPE: { new_database_schema.set_charset_type(alter_database_schema.get_charset_type()); new_database_schema.set_collation_type(alter_database_schema.get_collation_type()); break; } case ObAlterDatabaseArg::PRIMARY_ZONE: { LOG_INFO("changing database's primary_zone does not take effect"); ret = OB_SUCCESS; // do nothing break; } case ObAlterDatabaseArg::READ_ONLY: { new_database_schema.set_read_only(alter_database_schema.is_read_only()); break; } case ObAlterDatabaseArg::DEFAULT_TABLEGROUP: { if (OB_FAIL(new_database_schema.set_default_tablegroup_name( alter_database_schema.get_default_tablegroup_name()))) { LOG_WARN("failed to set default tablegroup name", K(ret)); } else if (OB_FAIL(set_default_tablegroup_id(new_database_schema))) { LOG_WARN("failed to set default tablegroup id", K(ret)); } break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unknown option!", K(i)); } } } } return ret; } int ObDDLService::alter_database(const ObAlterDatabaseArg &arg) { int ret = OB_SUCCESS; LOG_DEBUG("zixu alter database arg", K(arg)); const ObDatabaseSchema *origin_database_schema = NULL; ObSchemaGetterGuard schema_guard; uint64_t tenant_id = arg.database_schema_.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { if (OB_FAIL(schema_guard.get_database_schema( tenant_id, arg.database_schema_.get_database_name_str(), origin_database_schema)) || NULL == origin_database_schema) { ret = OB_ERR_BAD_DATABASE; LOG_WARN("database not exists", K(arg), K(ret)); } else if (origin_database_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("Can not perform alter operation on object in recyclebin", K(ret), K(*origin_database_schema)); } } if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDatabaseSchema new_database_schema = *origin_database_schema; if (OB_FAIL(set_new_database_options(arg, new_database_schema))) { LOG_WARN("failed to set new database options", K(ret)); } if (OB_SUCC(ret)) { ret = ddl_operator.alter_database(new_database_schema, trans, OB_DDL_ALTER_DATABASE, &arg.ddl_stmt_str_); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (OB_SUCC(ret) && OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed, ", K(ret)); } } return ret; } int ObDDLService::drop_database(const ObDropDatabaseArg &arg, obrpc::ObDropDatabaseRes &res, ObMySQLTransaction *ora_user_trans) { int ret = OB_SUCCESS; const bool if_exist = arg.if_exist_; const uint64_t tenant_id = arg.tenant_id_; const ObString &database_name = arg.database_name_; UInt64 &affected_row = res.affected_row_; ObDDLSQLTransaction trans(schema_service_); uint64_t table_count = 0; ObArray table_ids; ObSchemaGetterGuard schema_guard; const ObDatabaseSchema *db_schema = NULL; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_INVALID_ID == tenant_id || database_name.length() <= 0) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(tenant_id), K(database_name), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, database_name, db_schema))) { LOG_WARN("get_database schema failed", K(ret)); } else if (OB_ISNULL(db_schema)) { if (if_exist) { LOG_USER_NOTE(OB_ERR_DB_DROP_EXISTS, database_name.length(), database_name.ptr()); LOG_INFO("database not exist, no need to delete it", K(tenant_id), K(database_name)); } else { ret = OB_ERR_DB_DROP_EXISTS; LOG_USER_ERROR(OB_ERR_DB_DROP_EXISTS, database_name.length(), database_name.ptr()); LOG_WARN("database not exist, can't delete it",K(tenant_id), K(database_name), K(ret)); } } else if (db_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("can not drop database in recyclebin", K(ret), K(*db_schema)); } else { uint64_t database_id = db_schema->get_database_id(); int64_t refreshed_schema_version = 0; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(schema_guard.get_table_ids_in_database(tenant_id, database_id, table_ids))) { LOG_WARN("fail to get table ids in database", K(tenant_id), K(database_id), K(ret)); } else if (FALSE_IT(table_count = table_ids.count())) { //nothing to do } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_ISNULL(ora_user_trans) && OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { const ObTableSchema *schema = NULL; // lock table when drop data table for (int64_t i = 0; OB_SUCC(ret) && i < table_count; i++) { if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_ids.at(i), schema))) { LOG_WARN("fail to get table schema", K(ret), "table_id", table_ids.at(i)); } else if (OB_ISNULL(schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema should not be null", K(ret)); } else if (!schema->check_can_do_ddl()) { ret = OB_NOT_SUPPORTED; LOG_WARN("offline ddl is being executed, other ddl operations are not allowed", K(schema), K(ret)); } else if (OB_FAIL(lock_table(OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans, *schema))) { LOG_WARN("fail to lock_table", KR(ret), KPC(schema)); // for ddl retry task, upper layer only focus on `OB_TRY_LOCK_ROW_CONFLICT`, and then retry it. const bool is_ddl_scheduled_task = arg.task_id_ > 0 ? true : false; ret = is_ddl_scheduled_task && ObDDLUtil::is_table_lock_retry_ret_code(ret) ? OB_TRY_LOCK_ROW_CONFLICT : ret; } } // drop mv force for (int64_t i = 0; OB_SUCC(ret) && i < table_count; i++) { if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_ids.at(i), schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), "table_id", table_ids.at(i)); } else if (!schema->check_can_do_ddl()) { ret = OB_NOT_SUPPORTED; LOG_WARN("offline ddl is being executed, other ddl operations are not allowed", K(schema), K(ret)); } else if (schema && schema->is_materialized_view()) { if (OB_FAIL(ddl_operator.drop_table(*schema, OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans))) { LOG_WARN("fail to drop mv", K(ret), K(*schema)); } } } if (OB_SUCC(ret) && arg.to_recyclebin_ && !is_inner_db(db_schema->get_database_id())) { if (OB_FAIL(ddl_operator.drop_database_to_recyclebin(*db_schema, OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans, schema_guard, &arg.ddl_stmt_str_))) { LOG_WARN("drop database to recyclebin failed", K(arg), K(ret)); } } else { if (OB_FAIL(ret)) { // FAIL } else if (OB_FAIL(ddl_operator.drop_database(*db_schema, OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans, schema_guard, &arg.ddl_stmt_str_))) { LOG_WARN("ddl_operator drop_database failed", K(tenant_id), KT(database_id), K(ret)); } } if (OB_FAIL(ret)) { } else if (arg.task_id_ > 0 && OB_FAIL(ObDDLRetryTask::update_task_status_succ( OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans, tenant_id, arg.task_id_))) { LOG_WARN("update ddl task status to success failed", K(ret)); } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { affected_row = table_count; // Specify ora_user_trans, create user ddl, publish schema when create_user is completed if (OB_ISNULL(ora_user_trans)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } LOG_INFO("finish drop database", K(tenant_id), K(arg), K(ret)); return ret; } int ObDDLService::create_tablegroup(const bool if_not_exist, share::schema::ObTablegroupSchema &tablegroup_schema, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; RS_TRACE(create_tablegroup_begin); ObDDLSQLTransaction trans(schema_service_); bool is_exist = false; uint64_t tablegroup_id = OB_INVALID_ID; const uint64_t tenant_id = tablegroup_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(try_format_partition_schema(tablegroup_schema))) { LOG_WARN("fail to try_format_partition_schema", K(tablegroup_schema), KR(ret)); } else if (OB_FAIL(generate_object_id_for_partition_schema(tablegroup_schema))) { LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(tablegroup_schema)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_service_->check_tablegroup_exist( tenant_id, tablegroup_schema.get_tablegroup_name(), tablegroup_id, is_exist))) { LOG_WARN("check tablegroup exist failed", "tenant_id", tenant_id, "tablegroup_name", tablegroup_schema.get_tablegroup_name_str(), K(ret)); } else { if (is_exist) { if (if_not_exist) { ret = OB_SUCCESS; tablegroup_schema.set_tablegroup_id(tablegroup_id); LOG_USER_NOTE(OB_TABLEGROUP_EXIST); LOG_INFO("tablegroup already exists, not need to create", "tenant_id", tablegroup_schema.get_tenant_id(), "tablegroup_name", tablegroup_schema.get_tablegroup_name_str()); } else { ret = OB_TABLEGROUP_EXIST; LOG_WARN("tablegroup already exists", "tenant_id", tablegroup_schema.get_tenant_id(), "tablegroup_name", tablegroup_schema.get_tablegroup_name_str(), K(ret)); } } else { int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { // do nothing } RS_TRACE(write_tablegroup_schema_begin); if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.create_tablegroup(tablegroup_schema, trans, ddl_stmt_str))) { LOG_WARN("ddl operator failed to create tablegroup", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } } // publish schema if (OB_SUCC(ret)) { RS_TRACE(publish_tablegroup_schema_begin); if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("fail to public schema", KR(ret), K(tenant_id)); } else { RS_TRACE(publish_tablegroup_schema_end); } } FORCE_PRINT_TRACE(THE_RS_TRACE, "[create tablegroup]"); return ret; } int ObDDLService::drop_tablegroup(const ObDropTablegroupArg &arg) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); const bool if_exist = arg.if_exist_; const uint64_t tenant_id = arg.tenant_id_; const ObString &tablegroup_name = arg.tablegroup_name_; uint64_t tablegroup_id = OB_INVALID_ID; bool is_exist = false; ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || tablegroup_name.length() <= 0) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(tenant_id), K(tablegroup_name), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); const ObTablegroupSchema *tablegroup_schema = NULL; int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_service_->check_tablegroup_exist(tenant_id, tablegroup_name, tablegroup_id, is_exist))) { LOG_WARN("check_tablegroup_exist failed", K(tenant_id), K(tablegroup_name), K(ret)); } else if (!is_exist) { if (if_exist) { LOG_USER_NOTE(OB_TABLEGROUP_NOT_EXIST); LOG_INFO("tablegroup not exist, no need to delete it", K(tenant_id), K(tablegroup_name)); } else { ret = OB_TABLEGROUP_NOT_EXIST; LOG_WARN("tablegroup not exist, can't delete it", K(tenant_id), K(tablegroup_name), K(ret)); } } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(schema_guard.get_tablegroup_schema(tenant_id, tablegroup_id, tablegroup_schema))) { LOG_WARN("fail to get tablegroup schema", K(ret), K(tenant_id), KT(tablegroup_id)); } else if (OB_ISNULL(tablegroup_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tablegroup schema is null", K(ret), KT(tablegroup_id)); } else { if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.drop_tablegroup(*tablegroup_schema, trans, &arg.ddl_stmt_str_))) { LOG_WARN("ddl_operator drop_tablegroup failed", K(tenant_id), KT(tablegroup_id), K(ret)); } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish drop tablegroup", K(tenant_id), KT(tablegroup_id), K(ret)); return ret; } // alter tablegroup usage: // 1. Add the specified table_list to the tablegroup, it need to check whether the newly added table // meets the constraints of the tablegroup; // 2. Modify the attributes of the tablegroup: including partition; // it can be regarded as an atomic operation to modify the attributes of all tables in the tablegroup; // when all the table information is modified successfully, the user will return to success, // otherwise the rollback operation will fail; int ObDDLService::alter_tablegroup(const ObAlterTablegroupArg &arg) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; const uint64_t tenant_id = arg.tenant_id_; const ObString &tablegroup_name = arg.tablegroup_name_; uint64_t tablegroup_id = OB_INVALID_ID; const ObTablegroupSchema *tablegroup_schema = NULL; // after publish_schema, timeout will be chenged to max value const int64_t trans_timeout = THIS_WORKER.get_timeout_remain(); const int64_t abs_timeout = trans_timeout + ObTimeUtility::current_time(); ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (arg.alter_option_bitset_.has_member(obrpc::ObAlterTablegroupArg::DROP_PARTITION) || arg.alter_option_bitset_.has_member(obrpc::ObAlterTablegroupArg::ADD_PARTITION)) { ret = OB_OP_NOT_ALLOW; LOG_WARN("modify partition for tablegroup in 4.0 not allowed", K(ret), K(tablegroup_id)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify partition for tablegroup in 4.0"); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(tenant_id), K(ret)); } else if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service should not be null", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_tablegroup_id(tenant_id, tablegroup_name, tablegroup_id))) { LOG_WARN("fail to get tablegroup id", K(ret), K(tenant_id), K(arg)); } else if (OB_INVALID_ID == tablegroup_id) { ret = OB_TABLEGROUP_NOT_EXIST; LOG_WARN("tablegroup not exist", K(ret), K(ret), K(tablegroup_name)); } else if (OB_FAIL(schema_guard.get_tablegroup_schema(tenant_id, tablegroup_id, tablegroup_schema))) { LOG_WARN("fail to get tablegroup schema", K(ret), K(tenant_id), K(tablegroup_id)); } else { ObSchemaGetterGuard new_schema_guard; const ObTablegroupSchema *orig_tablegroup = NULL; ObTablegroupSchema new_tablegroup; // Since tablegroup involves the update of all the following tables, it takes a long time // Set the transaction timeout time to the remaining time of the query // abs_timeout = timeout + current_time ObTimeoutCtx timeout_ctx; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, new_schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(timeout_ctx.set_trx_timeout_us(trans_timeout))) { LOG_WARN("failed to set trans timeout", K(ret), K(trans_timeout)); } else if (OB_FAIL(timeout_ctx.set_abs_timeout(abs_timeout))) { LOG_WARN("failed to set stmt timeout", K(ret), K(abs_timeout)); } else if (OB_FAIL(new_schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(new_schema_guard.get_tablegroup_schema(tenant_id, tablegroup_id, orig_tablegroup))) { LOG_WARN("fail to get tablegroup schema", K(ret), K(tenant_id), K(tablegroup_id)); } else if (OB_ISNULL(orig_tablegroup)) { ret = OB_TABLEGROUP_NOT_EXIST; LOG_WARN("get invalid tablegroup schema", K(ret), K(orig_tablegroup)); } else if (OB_FAIL(helper.add_tables_to_tablegroup(trans, new_schema_guard, *orig_tablegroup, arg))) { LOG_WARN("fail to add tables to tablegroup", K(ret)); } else if (OB_FAIL(helper.modify_partition_option(trans, new_schema_guard, new_tablegroup, arg))) { LOG_WARN("fail to modify partition option", K(ret), K(new_tablegroup), K(arg)); } // however, end the trans int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_FAIL(ret)) { } else if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("fail to publish schema", KR(ret), K(tenant_id)); } LOG_INFO("finish alter tablegroup", K(tenant_id), KT(tablegroup_id), K(ret)); return ret; } int ObDDLService::refresh_schema(uint64_t tenant_id) { int ret = OB_SUCCESS; int64_t refresh_count = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { int64_t original_timeout_us = THIS_WORKER.get_timeout_ts(); // refresh schema will retry to success, so ignore the DDL request timeout. THIS_WORKER.set_timeout_ts(INT64_MAX); ObArray tenant_ids; if (OB_INVALID_TENANT_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant_id", K(ret)); } else if (OB_FAIL(tenant_ids.push_back(tenant_id))) { LOG_WARN("fail to push back tenant_id", K(ret), K(tenant_id)); } while (!stopped_) { common::ObTimeoutCtx ctx; if (OB_FAIL(schema_service_->set_timeout_ctx(ctx))) { LOG_ERROR("fail to set timeout_ctx, refresh schema failed", K(ret)); break; } else { ret = schema_service_->refresh_and_add_schema(tenant_ids); } if (OB_SUCC(ret)) { break; } else { ++refresh_count; if (refresh_count > 2) { LOG_ERROR("refresh schema failed", K(refresh_count), "refresh_schema_interval", static_cast(REFRESH_SCHEMA_INTERVAL_US), K(ret)); } else { LOG_WARN("refresh schema failed", K(refresh_count), "refresh_schema_interval", static_cast(REFRESH_SCHEMA_INTERVAL_US), K(ret)); } ob_usleep(REFRESH_SCHEMA_INTERVAL_US); } } if (OB_SUCC(ret) && !stopped_) { int64_t schema_version = OB_INVALID_VERSION; if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version( tenant_id, schema_version))) { LOG_WARN("fail to get tenant refreshed schema version", K(ret), K(tenant_id)); } else { ObSchemaService *schema_service = schema_service_->get_schema_service(); ObRefreshSchemaInfo schema_info; schema_info.set_tenant_id(tenant_id); schema_info.set_schema_version(schema_version); if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service is null", K(ret)); } else if (OB_FAIL(schema_service->inc_sequence_id())) { LOG_WARN("increase sequence_id failed", K(ret)); } else if (OB_FAIL(schema_service->set_refresh_schema_info(schema_info))) { LOG_WARN("fail to set refresh schema info", K(ret), K(schema_info)); } } } if (OB_FAIL(ret) && stopped_) { ret = OB_CANCELED; LOG_WARN("rs is stopped"); } THIS_WORKER.set_timeout_ts(original_timeout_us); } return ret; } int ObDDLService::notify_refresh_schema(const ObAddrIArray &addrs) { int ret = OB_SUCCESS; const ObZone zone; ObServerManager::ObServerArray server_list; ObSwitchSchemaProxy proxy(*rpc_proxy_, &ObSrvRpcProxy::switch_schema); ObSwitchSchemaArg arg; ObRefreshSchemaInfo local_schema_info; ObRefreshSchemaInfo &schema_info = arg.schema_info_; int64_t schema_version = OB_INVALID_VERSION; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(server_mgr_->get_alive_servers(zone, server_list))) { LOG_WARN("get alive server failed", KR(ret)); } else if (OB_ISNULL(schema_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema_service is null", KR(ret)); } else if (OB_FAIL(schema_service_->get_refresh_schema_info(local_schema_info))) { LOG_WARN("fail to get schema info", KR(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_version(OB_SYS_TENANT_ID, schema_version))) { LOG_WARN("fail to get sys schema version", KR(ret)); } else { // For compatibility reasons, local_schema_info is invalid before the schema is split, // and sequence_id is not broadcast if (local_schema_info.is_valid()) { if (OB_FAIL(schema_info.assign(local_schema_info))) { LOG_WARN("fail to assign schema_info", KR(ret), K(local_schema_info)); } } else { schema_info.set_schema_version(schema_version); } LOG_INFO("try to notify refresh schema", K(schema_version), K(local_schema_info), K(schema_info)); const int64_t rpc_timeout = GCONF.rpc_timeout; int64_t timeout = 0; FOREACH_X(s, server_list, OB_SUCC(ret)) { if (OB_ISNULL(s)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("s is null", K(ret)); } else if (server_mgr_->get_rs_addr() == *s) { continue; } else { bool found = false; for (int64_t i = 0; !found && i < addrs.count(); i++) { if (addrs.at(i) == *s) { found = true; } } if (found) { // refresh schema sync and report error timeout = THIS_WORKER.get_timeout_remain(); } else { // refresh schema async and ignore error timeout = std::min(THIS_WORKER.get_timeout_remain(), rpc_timeout); } arg.force_refresh_ = found; // overwrite ret if (OB_FAIL(proxy.call(*s, timeout, arg))) { LOG_WARN("send switch schema rpc failed", KR(ret), K(timeout), K(schema_version), K(schema_info), K(arg), "server", *s); if (!found) { // ignore servers that are not in addrs ret = OB_SUCCESS; } } } } ObArray return_code_array; int tmp_ret = OB_SUCCESS; // always wait all if (OB_SUCCESS != (tmp_ret = proxy.wait_all(return_code_array))) { LOG_WARN("wait result failed", KR(tmp_ret), KR(ret)); ret = OB_SUCC(ret) ? tmp_ret : ret; } ARRAY_FOREACH_N(return_code_array, i, cnt) { int res_ret = return_code_array.at(i); const ObAddr &addr = proxy.get_dests().at(i); const obrpc::ObSwitchSchemaArg &tmp_arg = proxy.get_args().at(i); if (OB_SUCCESS != res_ret && tmp_arg.force_refresh_) { ret = res_ret; LOG_WARN("switch schema failed", KR(ret), K(addr)); } } // end for } LOG_INFO("notify switch schema finished", KR(ret), K(schema_version), K(schema_info), K(arg), K(addrs)); return ret; } // force return OB_SUCCESS // this function is only used in ddl service, the caller don't care // the result of publish_schema in whole procedure of DDL process. int ObDDLService::publish_schema(uint64_t tenant_id /*=OB_INVALID_TENANT_ID*/) { int ret = OB_SUCCESS; ObAddrArray addrs; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(publish_schema(tenant_id, addrs))) { LOG_WARN("fail to pubish schema", K(ret), K(tenant_id)); } return OB_SUCCESS; } int ObDDLService::publish_schema(uint64_t tenant_id, const ObAddrIArray &addrs) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(refresh_schema(tenant_id))) { LOG_WARN("refresh schema failed", K(ret)); } else if (OB_FAIL(notify_refresh_schema(addrs))) { LOG_WARN("notify refresh schema failed", K(ret)); } return ret; } int ObDDLService::check_create_tenant_schema( const ObIArray &pool_list, ObTenantSchema &tenant_schema, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (tenant_schema.get_tenant_name_str().length() > OB_MAX_TENANT_NAME_LENGTH) { ret = OB_INVALID_TENANT_NAME; LOG_USER_ERROR(OB_INVALID_TENANT_NAME, to_cstring(tenant_schema.get_tenant_name_str()), OB_MAX_TENANT_NAME_LENGTH); LOG_WARN("tenant name can't over max_tenant_name_length", KR(ret), K(OB_MAX_TENANT_NAME_LENGTH)); } else if (OB_FAIL(check_create_tenant_locality(pool_list, tenant_schema, schema_guard))) { LOG_WARN("fail to check create tenant locality", KR(ret), K(pool_list), K(tenant_schema)); } else if (OB_FAIL(check_create_tenant_replica_options(tenant_schema, schema_guard))) { LOG_WARN("check replica options failed", KR(ret), K(tenant_schema)); } return ret; } int ObDDLService::check_create_tenant_locality( const ObIArray &pool_list, ObTenantSchema &tenant_schema, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObArray pools; ObArray pool_zones; ObArray temp_zones; ObArray zone_region_list; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (0 == pool_list.count()) { ret = OB_EMPTY_RESOURCE_POOL_LIST; LOG_WARN("pool list can not be empty", K(pool_list), K(ret)); } else if (OB_FAIL(get_pools(pool_list, pools))) { LOG_WARN("get_pools failed", K(pool_list), K(ret)); } else if (OB_FAIL(unit_mgr_->get_zones_of_pools(pools, temp_zones))) { LOG_WARN("get_zones_of_pools failed", K(pools), K(ret)); } else if (temp_zones.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get_zones_of_pools return empty zone array", K(ret)); } else { // get zones of resource pools, remove duplicated zone std::sort(temp_zones.begin(), temp_zones.end()); FOREACH_X(zone, temp_zones, OB_SUCC(ret)) { if (OB_ISNULL(zone)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("zone is null", K(ret)); } else if (0 == pool_zones.count() || pool_zones.at(pool_zones.count() - 1) != *zone) { if (OB_FAIL(pool_zones.push_back(*zone))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } else {} // duplicated zone, no need to push into. } if (OB_FAIL(ret)) { } else if (OB_FAIL(construct_zone_region_list(zone_region_list, pool_zones))) { LOG_WARN("fail to construct zone_region list", K(ret)); } else if (OB_FAIL(parse_and_set_create_tenant_new_locality_options( schema_guard, tenant_schema, pools, pool_zones, zone_region_list))) { LOG_WARN("fail to parse and set new locality option", K(ret)); } else if (OB_FAIL(check_pools_unit_num_enough_for_schema_locality( pools, schema_guard, tenant_schema))) { LOG_WARN("pools unit num is not enough for locality", K(ret)); } else {} // no more to do } return ret; } template int ObDDLService::check_pools_unit_num_enough_for_schema_locality( const common::ObIArray &pools, share::schema::ObSchemaGetterGuard &schema_guard, const SCHEMA &schema) { int ret = OB_SUCCESS; common::ObArray zone_list; common::ObArray zone_locality; bool is_legal = true; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr ptr is null", K(ret), KP(unit_mgr_)); } else if (OB_FAIL(schema.get_zone_list(schema_guard, zone_list))) { LOG_WARN("fail to get zone list", K(ret)); } else if (OB_FAIL(schema.get_zone_replica_attr_array_inherit(schema_guard, zone_locality))) { LOG_WARN("fail to get zone replica num array", K(ret)); } else if (OB_FAIL(unit_mgr_->check_pools_unit_legality_for_locality( pools, zone_list, zone_locality, is_legal))) { LOG_WARN("fail to check", K(ret)); } else if (!is_legal) { ret = OB_OP_NOT_ALLOW; LOG_WARN("pool unit num is not enough for tenant locality", K(pools)); } else {} // no more return ret; } int ObDDLService::handle_security_audit_in_trans(const ObSAuditSchema &audit_schema, const ObSAuditModifyType modify_type, const bool need_update, const ObString &ddl_str, common::ObSqlString &public_sql_string, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); const uint64_t tenant_id = audit_schema.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.handle_audit_metainfo(audit_schema, modify_type, need_update, &ddl_str, trans, public_sql_string))) { LOG_WARN("Failed to handle audit meta info", K(ret)); } else { LOG_INFO("Succeed in inserting audit in audit table", K(audit_schema), K(ddl_str)); } if (trans.is_started()) { int tmp_ret = ret; if (OB_UNLIKELY(OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret))))) { LOG_WARN("Failed to commit/rollback trans", K(tmp_ret), K(ret)); ret = (OB_LIKELY(OB_SUCCESS == tmp_ret) ? ret : tmp_ret); } } } return ret; } int ObDDLService::handle_security_audit(const obrpc::ObSecurityAuditArg &arg) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!arg.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("arg is invalid", K(arg), K(ret)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObSAuditOperTimingType in_success = AUDIT_OT_INVALID;//no operate ObSAuditOperTimingType in_failure = AUDIT_OT_INVALID;//no operate if (AUDIT_MT_ADD == arg.modify_type_) { if (AUDIT_WHEN_NOT_SET == arg.when_type_) { in_success = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION); in_failure = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION); } else if (AUDIT_WHEN_SUCCESS == arg.when_type_) { in_success = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION); } else { in_failure = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION); } } else { if (AUDIT_WHEN_NOT_SET == arg.when_type_) { in_success = AUDIT_OT_NOT_SET; in_failure = AUDIT_OT_NOT_SET; } else if (AUDIT_WHEN_SUCCESS == arg.when_type_) { in_success = AUDIT_OT_NOT_SET; } else { in_failure = AUDIT_OT_NOT_SET; } } ObSAuditSchema audit_schema; audit_schema.set_tenant_id(arg.tenant_id_); audit_schema.set_audit_type(arg.audit_type_); audit_schema.set_in_success(in_success); audit_schema.set_in_failure(in_failure); if (AUDIT_STMT_ALL_USER == arg.audit_type_ || AUDIT_STMT == arg.audit_type_) {//stmt if (OB_FAIL(handle_security_audit_for_stmt(arg, audit_schema))) { LOG_WARN("failed to handle_security_audit_for_object", K(arg), K(ret)); } } else if (AUDIT_OBJ_DEFAULT <= arg.audit_type_ && arg.audit_type_ <= AUDIT_PROCEDURE) {//object if (OB_FAIL(handle_security_audit_for_object(arg, audit_schema))) { LOG_WARN("failed to handle_security_audit_for_object", K(arg), K(ret)); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("audit_type is unexpected", K(arg.audit_type_), K(ret)); } } return ret; } int ObDDLService::handle_security_audit_for_stmt(const obrpc::ObSecurityAuditArg &arg, ObSAuditSchema &audit_schema) { int ret = OB_SUCCESS; ObSqlString ddl_stmt_str; ObSqlString ddl_operation_sql_string; share::schema::ObSchemaService *schema_service_impl = NULL; if (OB_UNLIKELY(AUDIT_STMT_ALL_USER != arg.audit_type_ && AUDIT_STMT != arg.audit_type_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected audit_type", K(arg.audit_type_), K(ret)); } else if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_impl = schema_service_->get_schema_service())) { ret = OB_ERR_SYS; LOG_ERROR("schema_service_impl must not be null", K(ret)); } else { uint64_t mock_user_id[1] = { OB_AUDIT_MOCK_USER_ID }; ObArrayHelper mock_user_ids(1, mock_user_id, 1); const common::ObIArray *stmt_user_ids = &arg.stmt_user_ids_; if (AUDIT_STMT_ALL_USER == arg.audit_type_) { stmt_user_ids = &mock_user_ids; } const ObSAuditOperTimingType target_in_success = audit_schema.get_in_success(); const ObSAuditOperTimingType target_in_failure = audit_schema.get_in_failure(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(arg.tenant_id_, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(arg)); } for (int64_t i = 0; OB_SUCC(ret) && i < stmt_user_ids->count(); ++i) { audit_schema.set_owner_id(stmt_user_ids->at(i)); ObString username; const ObUserInfo *user_info = NULL; if (AUDIT_STMT == arg.audit_type_) { if (OB_FAIL(schema_guard.get_user_info(arg.tenant_id_,stmt_user_ids->at(i),user_info))) { LOG_WARN("fail to get get_user_info", K(ret), K(arg), "user_id", stmt_user_ids->at(i)); } else if (OB_ISNULL(user_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("user_info is null", K(ret), K(arg), "user_id", stmt_user_ids->at(i)); } else { username = user_info->get_user_name_str(); } } for (int64_t j = 0; OB_SUCC(ret) && j < arg.operation_types_.count(); ++j) { audit_schema.set_operation_type(arg.operation_types_.at(j)); audit_schema.set_audit_id(OB_INVALID_ID); audit_schema.set_in_success(target_in_success); audit_schema.set_in_failure(target_in_failure); bool need_update = false; bool need_continue = false; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table( arg.tenant_id_, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(arg)); } else if (OB_FAIL(check_and_update_audit_schema(schema_guard, schema_service_impl, audit_schema, arg.modify_type_, need_update, need_continue))) { LOG_WARN("failed to check_and_update_audit_schema", K(arg), K(ret)); } else if (need_continue) { continue; } else { LOG_DEBUG("before gen_audit_stmt_sql", K(arg), K(audit_schema)); ddl_stmt_str.reuse(); ddl_operation_sql_string.reuse(); if (OB_FAIL(ObDDLSqlGenerator::gen_audit_stmt_sql(username, arg.modify_type_, audit_schema, arg.by_type_, arg.when_type_, ddl_stmt_str))) { LOG_WARN("fail to gen_audit_stmt_sql", K(audit_schema), K(ret)); } else if (OB_FAIL(handle_security_audit_in_trans(audit_schema, arg.modify_type_, need_update, ddl_stmt_str.string(), ddl_operation_sql_string, schema_guard))) { LOG_WARN("failed to handle_security_audit_in_trans", K(ret)); } else if (OB_FAIL(publish_schema(audit_schema.get_tenant_id()))) { LOG_WARN("publish schema failed", K(ret)); } } }//end of for j }//end of for i } return ret; } int ObDDLService::check_and_get_object_name(share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObSAuditType audit_type, const uint64_t tenant_id, const uint64_t obj_object_id, common::ObString &schema_name, common::ObString &object_name) { int ret = OB_SUCCESS; switch (audit_type) { case AUDIT_OBJ_DEFAULT: { //do nothing break; } case AUDIT_TABLE: { const ObSimpleTableSchemaV2 *table_schema = NULL; const ObSimpleDatabaseSchema *database_schema = NULL; if (OB_FAIL(schema_guard.get_simple_table_schema( tenant_id, obj_object_id, table_schema))) { LOG_WARN("failed to get_table_schema", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table_schema is null", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, table_schema->get_database_id(), database_schema))) { LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret)); } else { schema_name = database_schema->get_database_name_str(); object_name = table_schema->get_table_name_str(); } break; } case AUDIT_SEQUENCE: { const ObSequenceSchema *sequence_schema = NULL; const ObSimpleDatabaseSchema *database_schema = NULL; if (OB_FAIL(schema_guard.get_sequence_schema(tenant_id, obj_object_id, sequence_schema))) { LOG_WARN("failed to get_sequence_schema", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(sequence_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sequence_schema is null", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, sequence_schema->get_database_id(), database_schema))) { LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret)); } else { schema_name = database_schema->get_database_name_str(); object_name = sequence_schema->get_sequence_name(); } break; } case AUDIT_PACKAGE: { const ObPackageInfo *package_info = NULL; const ObSimpleDatabaseSchema *database_schema = NULL; if (OB_FAIL(schema_guard.get_package_info(tenant_id, obj_object_id, package_info))) { LOG_WARN("failed to get_package_info", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(package_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("package_info is null", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, package_info->get_database_id(), database_schema))) { LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret)); } else { schema_name = database_schema->get_database_name_str(); object_name = package_info->get_package_name(); } break; } case AUDIT_PROCEDURE: { const ObRoutineInfo *routine_info = NULL; const ObSimpleDatabaseSchema *database_schema = NULL; if (OB_FAIL(schema_guard.get_routine_info(tenant_id, obj_object_id, routine_info))) { LOG_WARN("failed to get_routine_schema", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(routine_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("routine_info is null", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, routine_info->get_database_id(), database_schema))) { LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret)); } else if (OB_ISNULL(database_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret)); } else { schema_name = database_schema->get_database_name_str(); object_name = routine_info->get_routine_name(); } break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected type", K(audit_type), K(obj_object_id), K(ret)); } } return ret; } int ObDDLService::handle_security_audit_for_object(const obrpc::ObSecurityAuditArg &arg, ObSAuditSchema &audit_schema) { int ret = OB_SUCCESS; share::schema::ObSchemaService *schema_service_impl = schema_service_->get_schema_service(); if (OB_UNLIKELY(arg.audit_type_ < AUDIT_OBJ_DEFAULT || arg.audit_type_ > AUDIT_PROCEDURE)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected audit_type", K(arg.audit_type_), K(ret)); } else if (OB_ISNULL(schema_service_impl)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service_impl must not be null", K(ret)); } else { ObSqlString ddl_stmt_str; ObSqlString ddl_operation_sql_string; ObString schema_name; ObString object_name; const ObSAuditOperTimingType target_in_success = audit_schema.get_in_success(); const ObSAuditOperTimingType target_in_failure = audit_schema.get_in_failure(); const uint64_t object_id = (AUDIT_OBJ_DEFAULT == arg.audit_type_ ? OB_AUDIT_MOCK_USER_ID : arg.obj_object_id_); audit_schema.set_owner_id(object_id); ObSchemaGetterGuard schema_guard; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(arg.tenant_id_, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(arg.tenant_id_)); } else if (OB_FAIL(check_and_get_object_name(schema_guard, arg.audit_type_, arg.tenant_id_, arg.obj_object_id_, schema_name, object_name))) { LOG_WARN("fail to check_and_get_object_name", K(arg), K(ret)); } for (int64_t j = 0; OB_SUCC(ret) && j < arg.operation_types_.count(); ++j) { audit_schema.set_operation_type(arg.operation_types_.at(j)); audit_schema.set_audit_id(OB_INVALID_ID); audit_schema.set_in_success(target_in_success); audit_schema.set_in_failure(target_in_failure); bool need_update = false; bool need_continue = false; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table( arg.tenant_id_, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(arg.tenant_id_)); } else if (OB_FAIL(check_and_update_audit_schema(schema_guard, schema_service_impl, audit_schema, arg.modify_type_, need_update, need_continue))) { LOG_WARN("failed to check_and_update_audit_schema", K(arg), K(ret)); } else if (need_continue) { continue; } else { ddl_stmt_str.reuse(); ddl_operation_sql_string.reuse(); LOG_DEBUG("before gen_audit_object_sql", K(arg), K(audit_schema)); if (OB_FAIL(ObDDLSqlGenerator::gen_audit_object_sql(schema_name, object_name, arg.modify_type_, audit_schema, arg.by_type_, arg.when_type_, ddl_stmt_str))) { LOG_WARN("fail to gen_audit_stmt_sql", K(schema_name), K(object_name), K(ret)); } else if (OB_FAIL(handle_security_audit_in_trans(audit_schema, arg.modify_type_, need_update, ddl_stmt_str.string(), ddl_operation_sql_string, schema_guard))) { LOG_WARN("failed to handle_security_audit_in_trans", KR(ret), K(arg)); } else if (OB_FAIL(publish_schema(audit_schema.get_tenant_id()))) { LOG_WARN("publish schema failed", K(ret)); } } } } return ret; } int ObDDLService::check_and_update_audit_schema(ObSchemaGetterGuard &schema_guard, share::schema::ObSchemaService *ss_impl, ObSAuditSchema &audit_schema, const ObSAuditModifyType modify_type, bool &need_update, bool &need_continue) { int ret = OB_SUCCESS; const ObSAuditSchema *tmp_audit_schema = NULL; need_update = false; need_continue = false; if (OB_FAIL(schema_guard.get_audit_schema_in_tenant(audit_schema.get_tenant_id(), audit_schema.get_audit_type(), audit_schema.get_owner_id(), audit_schema.get_operation_type(), tmp_audit_schema))) { LOG_WARN("failed to get_audit_schema", K(audit_schema), K(ret)); } else if (NULL == tmp_audit_schema) { if (AUDIT_MT_ADD == modify_type) { //add new audit uint64_t new_audit_id = common::OB_INVALID_ID; if (OB_FAIL(ss_impl->fetch_new_audit_id(audit_schema.get_tenant_id(), new_audit_id))) { LOG_WARN("Failed to fetch new_audit_id", K(ret)); } else { audit_schema.set_audit_id(new_audit_id); if (AUDIT_OT_INVALID == audit_schema.get_in_success()) { audit_schema.set_in_success(AUDIT_OT_NOT_SET); } if (AUDIT_OT_INVALID == audit_schema.get_in_failure()) { audit_schema.set_in_failure(AUDIT_OT_NOT_SET); } } } else { //mark do nothing about schema_table need_continue = true; } } else { audit_schema.set_audit_id(tmp_audit_schema->get_audit_id()); if (AUDIT_MT_ADD == modify_type) { need_update = true; //use orig one if (AUDIT_OT_INVALID == audit_schema.get_in_success()) { audit_schema.set_in_success(tmp_audit_schema->get_in_success()); } if (AUDIT_OT_INVALID == audit_schema.get_in_failure()) { audit_schema.set_in_failure(tmp_audit_schema->get_in_failure()); } } else { const int64_t MAX_ATTR_COUNT = 2;//in_success, in_failure const bool effect_in_succ = (tmp_audit_schema->get_in_success() != AUDIT_OT_NOT_SET); const bool effect_in_fail = (tmp_audit_schema->get_in_failure() != AUDIT_OT_NOT_SET); const bool del_in_succ = (audit_schema.get_in_success() != AUDIT_OT_INVALID); const bool del_in_fail = (audit_schema.get_in_failure() != AUDIT_OT_INVALID); if ((!effect_in_succ || del_in_succ) + (!effect_in_fail || del_in_fail) != MAX_ATTR_COUNT) { need_update = true; //use orig one if (effect_in_succ && !del_in_succ) { audit_schema.set_in_success(tmp_audit_schema->get_in_success()); } if (effect_in_fail && !del_in_fail) { audit_schema.set_in_failure(tmp_audit_schema->get_in_failure()); } } } } LOG_DEBUG("finish check_and_update_audit_schema", KPC(tmp_audit_schema), K(audit_schema), K(need_update), K(need_continue)); return ret; } int ObDDLService::create_user(ObCreateUserArg &arg, ObIArray &failed_index) { int ret = OB_SUCCESS; bool is_oracle_mode = false; uint64_t creator_id = arg.creator_id_; if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(arg.tenant_id_, is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < arg.user_infos_.count(); ++i) { ObUserInfo &user_info = arg.user_infos_.at(i); uint64_t user_id = OB_INVALID_ID; if (OB_FAIL(create_user(user_info, creator_id, user_id))) { if (is_oracle_mode) { // in oracle mode, if creating a user failed, just return the error code directly LOG_WARN("create user failed", K(ret), K(user_info), K(creator_id)); } else { // mysql mode if (OB_ERR_USER_EXIST == ret && true == arg.if_not_exist_) { ret = OB_SUCCESS; LOG_WARN("user already exist", K(ret), K(user_info)); LOG_USER_WARN(OB_ERR_USER_EXIST); } else { // in mysql mode, if creating a user failed, try next one and recover error code LOG_WARN("create_user failed", K(ret), K(user_info)); ret = OB_SUCCESS; if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } } } } return ret; } int ObDDLService::create_user(ObUserInfo &user_info, uint64_t creator_id, uint64_t &user_id) { int ret = OB_SUCCESS; ObSchemaService *schema_service_impl = NULL; const uint64_t tenant_id = user_info.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == user_info.get_tenant_id()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant_id is valid", K(user_info), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { schema_service_impl = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service_impl)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service_impl must not be null", K(schema_service_impl), K(ret)); } else if (OB_FAIL(check_user_exist(user_info))) { if (OB_ERR_USER_EXIST == ret) { LOG_WARN("User is exist, cannot create it twice,", "tenant_id", user_info.get_tenant_id(), "user_id", user_info.get_user_id(), "user_name", user_info.get_user_name_str(), K(ret)); } else { LOG_WARN("check_user_exist failed", K(user_info), K(ret)); } } } if (OB_SUCC(ret)) { // if user_id is valid, don't fetch new table id, in case of create user with user_id uint64_t new_user_id = user_info.get_user_id(); if (OB_FAIL(schema_service_impl->fetch_new_user_id( user_info.get_tenant_id(), new_user_id))) { LOG_WARN("Failed to fetch new_user_id", K(ret)); } else { user_info.set_user_id(new_user_id); } if (OB_SUCC(ret)) { if (OB_FAIL(create_user_in_trans(user_info, creator_id, user_id, schema_guard))) { LOG_WARN("create_user_in_trans failed", K(user_info), K(ret), K(creator_id)); } } } else if (OB_ERR_USER_EXIST == ret) { } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::drop_user(const ObDropUserArg &arg, ObIArray &failed_index) { int ret = OB_SUCCESS; ObDropUserArg arg_tmp = arg; const uint64_t tenant_id = arg.tenant_id_; uint64_t user_id = OB_INVALID_ID; ObSqlString ddl_stmt_str; ObAccountArg account; ObString ddl_sql; for (int64_t i = 0; OB_SUCC(ret) && i < arg_tmp.users_.count(); ++i) { ObSchemaGetterGuard schema_guard; ddl_stmt_str.reuse(); ddl_sql.reset(); account.user_name_ = arg_tmp.users_.at(i); account.host_name_ = arg_tmp.hosts_.at(i); const bool is_role = arg_tmp.is_role_; account.is_role_ = is_role; const ObUserInfo *user_info = NULL; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { ret = OB_ERR_SYS; LOG_WARN("Get schema manager failed", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_user_info(tenant_id, account.user_name_, account.host_name_, user_info))) { LOG_WARN("get_user_id failed", K(ret), K(ret), K(account)); } else if (NULL == user_info) { if (is_role) { ret = OB_ROLE_NOT_EXIST; LOG_WARN("drop non-exist user or role", K(ret), K(tenant_id), K(account.user_name_)); LOG_USER_ERROR(OB_ROLE_NOT_EXIST, account.user_name_.length(), account.user_name_.ptr()); } else { ret = OB_SUCCESS; //no such user, recover LOG_WARN("Try to drop non-exist user or role", K(tenant_id), K(account)); if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } } else if (is_role != user_info->is_role()) { if (is_role) { // Try to drop role, but the current name is user ret = OB_ROLE_NOT_EXIST; LOG_WARN("this is an user name", K(ret), K(tenant_id), K(account.user_name_)); LOG_USER_ERROR(OB_ROLE_NOT_EXIST, account.user_name_.length(), account.user_name_.ptr()); } else { // Try to drop user, but the current name is essentially a role ret = OB_USER_NOT_EXIST; //no such user LOG_WARN("Try to drop user", K(ret), K(tenant_id), K(account.user_name_)); } } else if (OB_FAIL(ObDDLSqlGenerator::gen_drop_user_sql(account, ddl_stmt_str))) { LOG_WARN("gen drop_user sql failed", K(ret), K(account)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(drop_user_in_trans(tenant_id, user_info->get_user_id(), &ddl_sql))) { LOG_WARN("Drop one user failed", K(account), K(tenant_id), K(user_id), K(ret)); ret = OB_SUCCESS; //drop fail, try next, recover if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } } return ret; } int ObDDLService::drop_user_in_trans(const uint64_t tenant_id, const uint64_t user_id, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant id is invalid", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.drop_user(tenant_id, user_id, ddl_stmt_str, trans))) { LOG_WARN("failed to drop user", K(ret), K(tenant_id), K(user_id)); } else { const ObTenantSchema *tenant_schema = NULL; const ObSysVariableSchema *sys_variable_schema = NULL; bool is_oracle_mode = false; const ObUserInfo *user_info = NULL; if (OB_FAIL(ret)) { // do-nothing } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("tenant not exists", K(ret), K(tenant_id)); } else if (OB_UNLIKELY(NULL == tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) { LOG_WARN("get sys variable schema failed", K(ret)); } else if (OB_ISNULL(sys_variable_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys variable schema is null", K(ret)); } else if (OB_FAIL(sys_variable_schema->get_oracle_mode(is_oracle_mode))) { LOG_WARN("failed to get oracle mode", K(ret), K(tenant_id)); } else if (is_oracle_mode) { if (OB_FAIL(schema_guard.get_user_info(tenant_id, user_id, user_info))) { LOG_WARN("failed to get user info", K(ret), K(tenant_id), K(user_id)); } else if (NULL == user_info) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get user info", K(ret), K(tenant_id), K(user_id)); } else if (!user_info->is_role()) { // For Oracle mode, drop user will drop the same name db // role does not contain the same name db obrpc::ObDropDatabaseArg arg; arg.tenant_id_ = tenant_id; arg.database_name_ = user_info->get_user_name(); arg.if_exist_ = false; arg.to_recyclebin_ = false; obrpc::ObDropDatabaseRes res; if (OB_FAIL(drop_database(arg, res, &trans))) { LOG_WARN("failed to create oracle user database", K(ret), K(tenant_id), K(user_info)); } } } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::alter_role(const ObAlterRoleArg &arg) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; uint64_t role_id = OB_INVALID_ID; const ObString &role_name = arg.role_name_; const ObString &host_name = arg.host_name_; const ObString &passwd = arg.pwd_enc_; const ObUserInfo *role_info = NULL; ObSchemaGetterGuard schema_guard; if(OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("failed to get schema guard", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_user_info(tenant_id, role_name, host_name, role_info))) { LOG_WARN("get_user_info failed", K(ret), K(tenant_id), K(role_name), K(host_name)); } else if (OB_ISNULL(role_info) || OB_UNLIKELY(false == role_info->is_role())) { ret = OB_ROLE_NOT_EXIST; LOG_WARN("alter non-exist role", K(ret), K(tenant_id), K(role_name), K(host_name)); LOG_USER_ERROR(OB_ROLE_NOT_EXIST, role_name.length(), role_name.ptr()); } else { ObSqlString ddl_stmt_str; ObString ddl_sql; role_id = role_info->get_user_id(); if (OB_UNLIKELY(OB_INVALID_ID == role_id)) { ret = OB_ROLE_NOT_EXIST; LOG_WARN("Try to alter a not-exist role", K(tenant_id), K(role_name), K(host_name), K(ret)); LOG_USER_ERROR(OB_ROLE_NOT_EXIST, role_name.length(), role_name.ptr()); } else if (OB_FAIL(ObDDLSqlGenerator::gen_alter_role_sql(ObAccountArg(role_name, host_name), passwd, ddl_stmt_str))) { LOG_WARN("gen_alter_role_sql failed", K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(alter_role_in_trans(tenant_id, role_id, passwd, &ddl_sql, schema_guard))) { LOG_WARN("alter_role_in_trans failed", K(tenant_id), K(role_id), K(arg), K(ret)); } } return ret; } int ObDDLService::alter_role_in_trans(const uint64_t tenant_id, const uint64_t role_id, const common::ObString &new_passwd, const common::ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variale is not init", K(ret)); } else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id) || OB_UNLIKELY(OB_INVALID_ID == role_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant_id or role_id is invalid", K(ret), K(tenant_id), K(role_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else { if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.alter_role(tenant_id, role_id, new_passwd, ddl_stmt_str, trans))) { LOG_WARN("fail to alter role", K(ret), K(tenant_id), K(role_id), K(new_passwd)); } } if (trans.is_started()) { int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(tmp_ret)); ret = (OB_SUCC(ret)) ? tmp_ret : ret; } } } // publish schema if(OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } //FIXME: For oracle user, database should be renamed too. int ObDDLService::rename_user(const ObRenameUserArg &arg, ObIArray &failed_index) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; uint64_t user_id = OB_INVALID_ID; ObSqlString ddl_stmt_str; ObString ddl_sql; ObAccountArg old_account; ObAccountArg new_account; const ObUserInfo *user_info = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < arg.old_users_.count(); ++i) { ObSchemaGetterGuard schema_guard; ddl_stmt_str.reuse(); ddl_sql.reset(); user_info = NULL; old_account.user_name_ = arg.old_users_.at(i); old_account.host_name_ = arg.old_hosts_.at(i); new_account.user_name_ = arg.new_users_.at(i); new_account.host_name_ = arg.new_hosts_.at(i); if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_user_info(tenant_id, new_account.user_name_, new_account.host_name_, user_info))) { LOG_WARN("Check user exist failed", K(tenant_id), K(new_account), K(ret)); } else if (NULL != user_info) { // don't need to set ret LOG_WARN("target user name already in use", K(tenant_id), K(new_account)); if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } else if (OB_FAIL(schema_guard.get_user_id(tenant_id, old_account.user_name_, old_account.host_name_, user_id))) { LOG_WARN("get user id failed", K(tenant_id), K(old_account), K(ret)); } else if (OB_INVALID_ID == user_id) { ret = OB_SUCCESS; //no such user, recover LOG_WARN("Try to rename non-exist user", K(tenant_id), K(old_account)); if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } else if (OB_FAIL(ObDDLSqlGenerator::gen_rename_user_sql(old_account, new_account, ddl_stmt_str))) { LOG_WARN("failed to gen rename user sql", K(ret), K(old_account), K(new_account)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(rename_user_in_trans(tenant_id, user_id, new_account, &ddl_sql, schema_guard))) { LOG_WARN("Rename one user failed", K(tenant_id), K(user_id), K(old_account), K(new_account), K(ret)); ret = OB_SUCCESS; //rename fail, try next, recover if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } } return ret; } int ObDDLService::rename_user_in_trans( const uint64_t tenant_id, const uint64_t user_id, const ObAccountArg &new_account, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant_id or user_id is invalid", K(ret)); } else { int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.rename_user(tenant_id, user_id, new_account, ddl_stmt_str, trans))) { LOG_WARN("fail to rename user", K(ret), K(tenant_id), K(user_id), K(new_account)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::set_passwd(const ObSetPasswdArg &arg) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; uint64_t user_id = OB_INVALID_ID; const ObString &user_name = arg.user_; const ObString &host_name = arg.host_; const ObString &passwd = arg.passwd_; const bool modify_max_connections = arg.modify_max_connections_; const uint64_t max_connections_per_hour = arg.max_connections_per_hour_; const uint64_t max_user_connections = arg.max_user_connections_; const share::schema::ObSSLType ssl_type = arg.ssl_type_; ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat faile", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else { ObSqlString ddl_stmt_str; ObString ddl_sql; if (OB_FAIL(schema_guard.get_user_id(tenant_id, user_name, host_name, user_id))) { LOG_WARN("get_user_id failed", K(tenant_id), K(user_name), K(host_name), K(ret)); } else if (OB_INVALID_ID == user_id) { ret = OB_USER_NOT_EXIST; //no such user LOG_WARN("Try to set password for non-exist user", K(tenant_id), K(user_name), K(host_name), K(ret)); } else if (share::schema::ObSSLType::SSL_TYPE_NOT_SPECIFIED == ssl_type) { if (modify_max_connections) { if (OB_FAIL(ObDDLSqlGenerator::gen_set_max_connections_sql( ObAccountArg(user_name, host_name), max_connections_per_hour, max_user_connections, ddl_stmt_str))) { LOG_WARN("gen_set_passwd_sql failed", K(ret), K(arg)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(set_max_connection_in_trans(tenant_id, user_id, max_connections_per_hour, max_user_connections, &ddl_sql, schema_guard))) { LOG_WARN("Set passwd failed", K(tenant_id), K(user_id), K(passwd), K(ret)); } } else { if (OB_FAIL(ObDDLSqlGenerator::gen_set_passwd_sql(ObAccountArg(user_name, host_name), passwd, ddl_stmt_str))) { LOG_WARN("gen_set_passwd_sql failed", K(ret), K(arg)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(set_passwd_in_trans(tenant_id, user_id, passwd, &ddl_sql, schema_guard))) { LOG_WARN("Set passwd failed", K(tenant_id), K(user_id), K(passwd), K(ret)); } } } else { if (OB_FAIL(ObDDLSqlGenerator::gen_alter_user_require_sql( ObAccountArg(user_name, host_name), arg, ddl_stmt_str))) { LOG_WARN("gen_alter_user_require_sql failed", K(ret), K(arg)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(alter_user_require_in_trans(tenant_id, user_id, arg, &ddl_sql, schema_guard))) { LOG_WARN("alter_user_require failed", K(tenant_id), K(user_id), K(arg), K(ret)); } } } return ret; } int ObDDLService::set_passwd_in_trans( const uint64_t tenant_id, const uint64_t user_id, const common::ObString &new_passwd, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant_id or user_id is invalid", K(tenant_id), K(user_id), K(ret)); } else { int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.set_passwd(tenant_id, user_id, new_passwd, ddl_stmt_str, trans))) { LOG_WARN("fail to set password", K(ret), K(tenant_id), K(user_id), K(new_passwd)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("pubish schema failed", K(ret)); } } return ret; } int ObDDLService::set_max_connection_in_trans( const uint64_t tenant_id, const uint64_t user_id, const uint64_t max_connections_per_hour, const uint64_t max_user_connections, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant_id or user_id is invalid", K(tenant_id), K(user_id), K(ret)); } else { int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.set_max_connections(tenant_id, user_id, max_connections_per_hour, max_user_connections, ddl_stmt_str, trans))) { LOG_WARN("fail to set max connections", K(ret), K(tenant_id), K(user_id)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("pubish schema failed", K(ret)); } } return ret; } int ObDDLService::alter_user_require_in_trans( const uint64_t tenant_id, const uint64_t user_id, const obrpc::ObSetPasswdArg &arg, const common::ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant_id or user_id is invalid", K(tenant_id), K(user_id), K(ret)); } else { int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.alter_user_require(tenant_id, user_id, arg, ddl_stmt_str, trans))) { LOG_WARN("fail to alter user require", K(ret), K(tenant_id), K(user_id)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("pubish schema failed", K(ret)); } } return ret; } /* grant sys privs to grantee grantee info is in roles[0], roles[1] and remain_roles*/ int ObDDLService::grant_sys_priv_to_ur( const ObGrantArg &arg, ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; const ObIArray &roles = arg.roles_; const ObIArray &remain_roles = arg.remain_roles_; ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; const ObUserInfo *user_info = NULL; ObSysPriv *sys_priv = NULL; uint64_t grantee_id; ObSArray users_name; ObSArray hosts_name; CK (OB_LIKELY(roles.count() == GRANT_SYS_ROLE_NUM)); OZ (users_name.push_back(roles.at(0))); OZ (hosts_name.push_back(roles.at(1))); CK (OB_LIKELY(remain_roles.count() % 2 == 0)); for (int i = 0; OB_SUCC(ret) && i < remain_roles.count(); i += 2) { OZ (users_name.push_back(remain_roles.at(i))); OZ (hosts_name.push_back(remain_roles.at(i + 1))); } if (FAILEDx(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } OZ (trans.start(sql_proxy_, tenant_id, refreshed_schema_version)); for (int i = 0; OB_SUCC(ret) && i < users_name.count(); ++i) { const ObString &user_name = users_name.at(i); const ObString &host_name = hosts_name.at(i); /* check grantee info */ OZ (schema_guard.get_user_info(tenant_id, user_name/*user_name*/, host_name/*host_name*/, user_info)); if (OB_SUCC(ret)) { if (NULL == user_info) { ret = OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST; LOG_USER_ERROR(OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST, user_name.length(), user_name.ptr()); LOG_WARN("user doesn't exist", K(ret), K(user_name)); } else { grantee_id = user_info->get_user_id(); } } OZ (schema_guard.get_sys_priv_with_grantee_id(tenant_id, grantee_id, sys_priv)); if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); OZ (ddl_operator.grant_sys_priv_to_ur(tenant_id, grantee_id, sys_priv, arg.option_, arg.sys_priv_array_, trans, true /*is_grant*/, &arg.ddl_stmt_str_, schema_guard)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (FAILEDx(publish_schema(tenant_id))) { LOG_WARN("pubish schema failed", K(ret)); } LOG_INFO("finish grant_sys_priv_to_ur", K(ret)); return ret; } int ObDDLService::grant_table_and_col_privs_to_user( const ObGrantArg &arg, uint64_t grantee_id, ObString &user_name, ObString &host_name, ObNeedPriv &need_priv, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; /* 1. deal with table privs first if has */ ObObjPrivSortKey obj_priv_key(arg.tenant_id_, arg.object_id_, static_cast(arg.object_type_), OB_COMPACT_COLUMN_INVALID_ID, arg.grantor_id_, grantee_id); OZ (grant_priv_to_user(arg.tenant_id_, grantee_id, user_name, host_name, need_priv, arg.obj_priv_array_, arg.option_, obj_priv_key, schema_guard)); /* 2. deal with cols privs */ /* 2.1 reorg privs according to colid */ uint64_t colid; ObRawObjPrivArray obj_priv_array; for (int i = 0; i < arg.ins_col_ids_.count() && OB_SUCC(ret); i++) { obj_priv_array.reset(); obj_priv_array.push_back(OBJ_PRIV_ID_INSERT); colid = arg.ins_col_ids_.at(i); ObObjPrivSortKey obj_priv_key(arg.tenant_id_, arg.object_id_, static_cast(arg.object_type_), colid, arg.grantor_id_, grantee_id); OZ (grant_priv_to_user(arg.tenant_id_, grantee_id, user_name, host_name, need_priv, obj_priv_array, arg.option_, obj_priv_key, schema_guard)); } for (int i = 0; i < arg.upd_col_ids_.count() && OB_SUCC(ret); i++) { obj_priv_array.reset(); obj_priv_array.push_back(OBJ_PRIV_ID_UPDATE); colid = arg.upd_col_ids_.at(i); ObObjPrivSortKey obj_priv_key(arg.tenant_id_, arg.object_id_, static_cast(arg.object_type_), colid, arg.grantor_id_, grantee_id); OZ (grant_priv_to_user(arg.tenant_id_, grantee_id, user_name, host_name, need_priv, obj_priv_array, arg.option_, obj_priv_key, schema_guard)); } for (int i = 0; i < arg.ref_col_ids_.count() && OB_SUCC(ret); i++) { obj_priv_array.reset(); obj_priv_array.push_back(OBJ_PRIV_ID_REFERENCES); colid = arg.ref_col_ids_.at(i); ObObjPrivSortKey obj_priv_key(arg.tenant_id_, arg.object_id_, static_cast(arg.object_type_), colid, arg.grantor_id_, grantee_id); OZ (grant_priv_to_user(arg.tenant_id_, grantee_id, user_name, host_name, need_priv, obj_priv_array, arg.option_, obj_priv_key, schema_guard)); } return ret; } /* Check whether there is role_id information in user_info, recursively */ int ObDDLService::exists_role_grant_cycle( ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, uint64_t role_id, const ObUserInfo *user_info) { int ret = OB_SUCCESS; bool found = false; CK (user_info != NULL); ObSEArray role_id_array = user_info->get_role_id_array(); for (int j = 0; OB_SUCC(ret) && !found && j < role_id_array.count(); ++j) { if (role_id == role_id_array.at(j)) { found = true; } else { const ObUserInfo *tmp_role_info = NULL; OZ (schema_guard.get_user_info(tenant_id, role_id_array.at(j), tmp_role_info)); if (OB_SUCC(ret) && tmp_role_info != NULL) { OZ (exists_role_grant_cycle(schema_guard, tenant_id, role_id, tmp_role_info)); } } } if (OB_SUCC(ret) && found) { ret = OB_ERR_CIRCULAR_ROLE_GRANT_DETECTED; } return ret; } int ObDDLService::grant(const ObGrantArg &arg) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; ObSchemaGetterGuard schema_guard; lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_UNLIKELY(!arg.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("arg is invalid", K(arg), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) { LOG_WARN("failed to get compat mode", K(ret), K(tenant_id)); } else { const ObIArray &roles = arg.roles_; // The user_name and host_name of the first user are stored in role[0] and role[1] respectively // The user_name and host_name of the remaining users are stored in remain_role const ObIArray &remain_roles = arg.remain_roles_; const bool is_grant_sys_or_role = roles.count() >= GRANT_SYS_ROLE_NUM; const bool is_grant_role = roles.count() > GRANT_SYS_ROLE_NUM; if (is_grant_sys_or_role) { // grant role and sys_privs to user_list ObSArray users_name; ObSArray hosts_name; ObArray users_info; // Start transaction at beginning ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { // Save all user names in array const ObUserInfo *user_info = NULL; OZ (users_name.push_back(roles.at(0))); OZ (hosts_name.push_back(roles.at(1))); CK (OB_LIKELY(remain_roles.count() % 2 == 0)); for (int i = 0; OB_SUCC(ret) && i < remain_roles.count(); i += 2) { OZ (users_name.push_back(remain_roles.at(i))); OZ (hosts_name.push_back(remain_roles.at(i + 1))); } CK (OB_LIKELY(users_name.count() == hosts_name.count())); // Save all user infos for (int i = 0; OB_SUCC(ret) && i < users_name.count(); ++i) { const ObString &user_name = users_name.at(i); const ObString &host_name = hosts_name.at(i); if (OB_FAIL(schema_guard.get_user_info(tenant_id, user_name, host_name, user_info))) { LOG_WARN("Failed to get role info", K(tenant_id), K(user_name), K(host_name), K(ret)); } else if (OB_UNLIKELY(NULL == user_info)) { ret = OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST; LOG_USER_ERROR(OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST, user_name.length(), user_name.ptr()); LOG_WARN("user doesn't exist", K(ret), K(user_name)); } else if (OB_FAIL(users_info.push_back(*user_info))) { LOG_WARN("Failed to push back user info", K(ret), K(*user_info)); } } } // Grant role if (OB_SUCC(ret)) { if (is_grant_role) { // grant roles to user // 1. Get the specified user granted // 2. Get all permissions owned by each role (three levels: user/db/table) // 3. All permissions of compatible role are appended to the specified user /* resolve grantee, priv info stored in roles[0], roles[1], grantee info stored in roles[2], roles[3], ... */ ObArray role_ids; ObArray roles_info; // Resolve each role id and role info for (int64_t i = GRANT_ROLE_MIN_ROLE_NUM - 1; OB_SUCC(ret) && i < roles.count(); ++i) { // Oracle currently does not support specifying hostname to create a role const ObString host_name(OB_DEFAULT_HOST_NAME); const ObString role = roles.at(i); const ObUserInfo *role_info = NULL; if (OB_FAIL(schema_guard.get_user_info(tenant_id, role, host_name, role_info))) { LOG_WARN("Failed to get role info", K(tenant_id), K(role), K(host_name), K(ret)); } else if (NULL == role_info) { ret = OB_ROLE_NOT_EXIST; LOG_WARN("role doesn't exist", K(ret), K(role)); LOG_USER_ERROR(OB_ROLE_NOT_EXIST, role.length(), role.ptr()); } else if (OB_FAIL(role_ids.push_back(role_info->get_user_id()))) { LOG_WARN("Failed to push back role_id", K(ret), K(tenant_id), K(*role_info)); } else if (OB_FAIL(roles_info.push_back(*role_info))) { LOG_WARN("Failed to push back role_info", K(ret), K(tenant_id), K(*role_info)); } } // Operate on each user_name for (int i = 0; OB_SUCC(ret) && i < users_info.count(); ++i) { const ObUserInfo &user_info = users_info.at(i); if (user_info.is_role()) { // Check if there is a cyclic grant for (int j = 0; OB_SUCC(ret) && j < roles_info.count(); ++j) { if (OB_FAIL(exists_role_grant_cycle(schema_guard, tenant_id, user_info.get_user_id(), &roles_info.at(j)))) { LOG_WARN("role cycle exists", K(ret), K(roles_info.at(j))); } } } // Do the operation if (OB_SUCC(ret) && role_ids.count() > 0) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); bool log_operation = true; if (true == arg.sys_priv_array_.empty()) { // No sys_priv granted toghther, log grant role operation log_operation = true; } else { // "grant role" will be logged with "grant sys_priv" later log_operation = false; } if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id, user_info, role_ids, NULL, /*Single specified role info*/ trans, log_operation, true /*is_grant*/, arg.option_))) { LOG_WARN("falied to grant_revoke_role", K(ret), K(tenant_id), K(user_info)); } } } } } // Grant sys_privs if (OB_SUCC(ret)) { if (false == arg.sys_priv_array_.empty()) { ObSysPriv *sys_priv = NULL; for (int i = 0; OB_SUCC(ret) && i < users_info.count(); ++i) { const ObUserInfo &user_info = users_info.at(i); uint64_t grantee_id = user_info.get_user_id(); OZ (schema_guard.get_sys_priv_with_grantee_id(tenant_id, grantee_id, sys_priv)); if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); OZ (ddl_operator.grant_sys_priv_to_ur(tenant_id, grantee_id, sys_priv, arg.option_, arg.sys_priv_array_, trans, true /*is_grant*/, &arg.ddl_stmt_str_, schema_guard)); } } } } // End transaction if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (FAILEDx(publish_schema(tenant_id))) { LOG_WARN("pubish schema failed", K(ret)); } } else { const ObSArray &users_passwd = arg.users_passwd_; const ObSArray &hosts = arg.hosts_; if (OB_UNLIKELY(users_passwd.count() % 2 != 0)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("users should have even string", K(users_passwd.count()), K(ret)); } else { ObString user_name; ObString host_name; ObString pwd; int tmp_ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < users_passwd.count(); i += 2) { bool is_user_exist = false; uint64_t user_id = OB_INVALID_ID; if (OB_FAIL(users_passwd.at(i, user_name))) { SQL_ENG_LOG(WARN, "Get string from ObSArray error", "count", users_passwd.count(), K(i), K(ret)); } else if (OB_FAIL(hosts.at(i / 2, host_name))) { SQL_ENG_LOG(WARN, "Get string from ObSArray error", "count", hosts.count(), K(i), K(ret)); } else if (OB_FAIL(users_passwd.at(i + 1, pwd))) { SQL_ENG_LOG(WARN, "Get string from ObSArray error", "count", users_passwd.count(), K(i), K(ret)); } else if (OB_FAIL(schema_service_->check_user_exist(arg.tenant_id_, user_name, host_name, user_id, is_user_exist))) { LOG_WARN("Failed to check whether user exist", K(arg.tenant_id_), K(user_name), K(host_name), K(ret)); } else if (is_user_exist) { if (!pwd.empty()) { //change password ObSqlString ddl_stmt_str; ObString ddl_sql; if (OB_FAIL(ObDDLSqlGenerator::gen_set_passwd_sql(ObAccountArg(user_name, host_name), pwd, ddl_stmt_str))) { LOG_WARN("gen set passwd sql failed", K(user_name), K(host_name), K(pwd), K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(set_passwd_in_trans(arg.tenant_id_, user_id, pwd, &ddl_sql, schema_guard))) { LOG_WARN("Set password error", KR(ret), K(arg), K(user_id), K(pwd), K(ddl_sql)); } } } else if (!is_user_exist) { if (arg.need_create_user_ || !pwd.empty()) { if (!arg.has_create_user_priv_) { tmp_ret = OB_ERR_CREATE_USER_WITH_GRANT; } else { ObUserInfo user_info; user_info.set_tenant_id(arg.tenant_id_); user_info.set_passwd(pwd); if (OB_FAIL(user_info.set_user_name(user_name))) { LOG_WARN("set_user_name error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret)); } else if (OB_FAIL(user_info.set_host(host_name))) { LOG_WARN("set_host error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret)); } else if (OB_FAIL(user_info.set_passwd(pwd))) { LOG_WARN("set_passwd error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret)); } else if (OB_FAIL(create_user(user_info, OB_INVALID_ID, user_id))) { LOG_WARN("Create user error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret)); } else { is_user_exist = true; } } } else { if (lib::Worker::CompatMode::ORACLE == compat_mode) { tmp_ret = OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST; LOG_USER_ERROR(OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST, user_name.length(), user_name.ptr()); } else { tmp_ret = OB_PASSWORD_WRONG; LOG_USER_ERROR(OB_PASSWORD_WRONG, user_name.length(), user_name.ptr(), host_name.length(), host_name.ptr(), "NO"); } } } if (OB_SUCC(ret) && is_user_exist) { ObNeedPriv need_priv(arg.db_, arg.table_, arg.priv_level_, arg.priv_set_, false); bool is_owner = false; // In oracle mode, if it is oracle syntax, it need to determine grantee is obj owner, // if yes, return success directly if (lib::Worker::CompatMode::ORACLE == compat_mode && (arg.priv_level_ == OB_PRIV_SYS_ORACLE_LEVEL || (arg.ins_col_ids_.count() + arg.upd_col_ids_.count() + arg.ref_col_ids_.count() + arg.obj_priv_array_.count() > 0))) { is_owner = share::ObOraPrivCheck::user_is_owner(user_name, arg.db_); } if (!is_owner) { /* No column level permissions */ if (arg.ins_col_ids_.count() + arg.upd_col_ids_.count() + arg.ref_col_ids_.count() == 0) { ObObjPrivSortKey obj_priv_key(arg.tenant_id_, arg.object_id_, static_cast(arg.object_type_), OB_COMPACT_COLUMN_INVALID_ID, arg.grantor_id_, user_id); if (OB_FAIL(grant_priv_to_user(arg.tenant_id_, user_id, user_name, host_name, need_priv, arg.obj_priv_array_, arg.option_, obj_priv_key, schema_guard))) { LOG_WARN("Grant priv to user failed", K(ret)); } } else { // Contains column-level permissions, only supported by oracle grant statement in oracle mode OZ (grant_table_and_col_privs_to_user(arg, user_id, user_name, host_name, need_priv, schema_guard)); } } } } if (OB_SUCC(ret) && tmp_ret != OB_SUCCESS) { ret = tmp_ret; } } } } return ret; } int ObDDLService::revoke(const ObRevokeUserArg &arg) { int ret = OB_SUCCESS; const ObUserInfo *user_info = NULL; const bool grant = false; const uint64_t tenant_id = arg.tenant_id_; ObSchemaGetterGuard schema_guard; const bool is_revoke_role = arg.role_ids_.count() > 0; uint64_t user_id = OB_INVALID_ID; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_INVALID_ID == arg.tenant_id_ || OB_INVALID_ID == arg.user_id_) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(arg)); } else if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.user_id_, user_info))) { LOG_WARN("get_user_info failed", K(arg), K(ret)); } else if (OB_ISNULL(user_info)) { ret = OB_USER_NOT_EXIST; LOG_WARN("user not exist", K(ret), K(arg)); } else if (is_revoke_role) { // this process include revoke role user_id = user_info->get_user_id(); ObArray role_ids; for (int64_t i = 0; OB_SUCC(ret) && i < arg.role_ids_.count(); ++i) { const uint64_t role_id = arg.role_ids_.at(i); const ObUserInfo *role_info = NULL; if (OB_FAIL(schema_guard.get_user_info(tenant_id, role_id, role_info))) { LOG_WARN("Failed to get role info", K(ret), K(tenant_id), K(role_id)); } else if (NULL == role_info) { ret = OB_ERR_UNEXPECTED; LOG_WARN("role doesn't exist", K(ret), K(role_id)); } else { // Determine whether the current role is granted to the user; otherwise, it will not be processed ObSEArray role_id_array = user_info->get_role_id_array(); for (int j = 0; OB_SUCC(ret) && j < role_id_array.count(); ++j) { if (role_info->get_user_id() == role_id_array.at(j)) { if (OB_FAIL(role_ids.push_back(role_id_array.at(j)))) { LOG_WARN("Failed to push back role_id", K(ret), K(tenant_id), K(*role_info)); } break; } } } } // need revoke role if (OB_SUCC(ret) && role_ids.count() > 0) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); ObUserInfo user = *user_info; int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id, user, role_ids, NULL /*Single specified role info*/, trans, true /*log_operation*/, false /*is_grant*/, NO_OPTION))) { LOG_WARN("falied to grant_revoke_role", K(ret), K(tenant_id), K(user), K(*user_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (FAILEDx(publish_schema(tenant_id))) { LOG_WARN("pubish schema failed", K(ret)); } } } else if (!arg.revoke_all_) { ObSqlString ddl_stmt_str; ObString ddl_sql; ObNeedPriv need_priv; need_priv.priv_set_ = arg.priv_set_; need_priv.priv_level_ = OB_PRIV_USER_LEVEL; if (OB_FAIL(ObDDLSqlGenerator::gen_user_priv_sql(ObAccountArg(user_info->get_user_name_str(), user_info->get_host_name_str()), need_priv, grant, ddl_stmt_str))) { LOG_WARN("gen user_priv sql failed", K(arg), K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(grant_revoke_user(arg.tenant_id_, arg.user_id_, arg.priv_set_, grant, &ddl_sql, schema_guard))) { LOG_WARN("Revoke user failed", K(arg), K(grant), K(ret)); } } else { if (OB_FAIL(revoke_all(arg.tenant_id_, user_info->get_user_name_str(), user_info->get_host_name_str(), arg.user_id_, schema_guard))) { LOG_WARN("Revoke all failed", K(arg), K(ret)); } } return ret; } int ObDDLService::grant_priv_to_user(const uint64_t tenant_id, const uint64_t user_id, const ObString &user_name, const ObString &host_name, const ObNeedPriv &need_priv, const share::ObRawObjPrivArray &obj_priv_array, const uint64_t option, ObObjPrivSortKey &obj_priv_key, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Input arguments error", K(tenant_id), K(user_id), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id)); } else { ObSqlString ddl_stmt_str; ObString ddl_sql; switch (need_priv.priv_level_) { case OB_PRIV_USER_LEVEL: { if (OB_FAIL(ObDDLSqlGenerator::gen_user_priv_sql(ObAccountArg(user_name, host_name), need_priv, true, ddl_stmt_str))) { LOG_WARN("gen user_priv sql failed", K(ret), K(user_name), K(host_name)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(grant_revoke_user(tenant_id, user_id, need_priv.priv_set_, true, &ddl_sql, schema_guard))) { LOG_WARN("Grant user error", KR(ret), K(tenant_id), K(user_id), K(ddl_sql), K(need_priv)); } break; } case OB_PRIV_DB_LEVEL: { ObOriginalDBKey db_key(tenant_id, user_id, need_priv.db_); if (OB_FAIL(ObDDLSqlGenerator::gen_db_priv_sql(ObAccountArg(user_name, host_name), need_priv, true, ddl_stmt_str))) { LOG_WARN("gen_db_priv sql failed", K(need_priv), K(user_name), K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(grant_database(db_key, need_priv.priv_set_, &ddl_sql, schema_guard))) { LOG_WARN("Grant database error", KR(ret), K(db_key), K(ddl_sql), K(need_priv)); } break; } case OB_PRIV_TABLE_LEVEL: { ObTablePrivSortKey table_key(tenant_id, user_id, need_priv.db_, need_priv.table_); if (OB_FAIL(ObDDLSqlGenerator::gen_table_priv_sql(ObAccountArg(user_name, host_name), need_priv, true, ddl_stmt_str))) { LOG_WARN("gen_table_priv sql failed", K(need_priv), K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(grant_table(table_key, need_priv.priv_set_, &ddl_sql, obj_priv_array, option, obj_priv_key, schema_guard))) { LOG_WARN("Grant table error", K(ret)); } break; } default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unexpected grant level", "GrantLevel", need_priv.priv_level_); } } } return ret; } int ObDDLService::revoke_all( const uint64_t tenant_id, const ObString &user_name, const ObString &host_name, const uint64_t user_id, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id || user_name.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant id or user_id is invalid", K(tenant_id), K(user_id), K(user_name), K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObSqlString ddl_stmt_str; ObString ddl_sql; if (OB_FAIL(ObDDLSqlGenerator::gen_revoke_all_sql(ObAccountArg(user_name, host_name), ddl_stmt_str))) { LOG_WARN("gen revoke all sql failed", K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(ddl_operator.grant_revoke_user(tenant_id, user_id, OB_PRIV_ALL|OB_PRIV_GRANT, false, &ddl_sql, trans))) { LOG_WARN("Revoke user error", K(ret)); } else if (OB_FAIL(ddl_operator.drop_db_table_privs(tenant_id, user_id, trans))) { LOG_WARN("Drop db table priv error", K(ret)); } else { } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::grant_revoke_user( const uint64_t tenant_id, const uint64_t user_id, const ObPrivSet priv_set, const bool grant, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant id is invalid", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.grant_revoke_user(tenant_id, user_id, priv_set, grant, ddl_stmt_str, trans))) { LOG_WARN("fail to grant revoke user", K(ret), K(tenant_id), K(user_id), K(priv_set), K(grant)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::get_all_users_in_tenant_with_profile(const uint64_t tenant_id, const uint64_t profile_id, ObSchemaGetterGuard &schema_guard, ObIArray &user_ids) { int ret = OB_SUCCESS; ObSchemaService *schema_sql_service = NULL; ObSEArray user_infos; if (OB_UNLIKELY(!is_valid_id(profile_id) || !is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(profile_id), K(tenant_id)); } else if (OB_ISNULL(schema_sql_service = schema_service_->get_schema_service())) { ret = OB_ERR_SYS; LOG_ERROR("schema_sql_service must not null", K(ret)); } else if (OB_FAIL(schema_guard.get_user_infos_with_tenant_id(tenant_id, user_infos))) { LOG_WARN("fail to get all user in tenant", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < user_infos.count(); ++i) { const ObUserInfo *user_info = NULL; if (OB_ISNULL(user_info = user_infos.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema is NULL", K(ret)); } else if (user_info->get_profile_id() == profile_id) { if (OB_FAIL(user_ids.push_back(user_info->get_user_id()))) { LOG_WARN("fail to push back", K(ret)); } } } return ret; } #define PUSH_BACK_ROLE_ID_INFO(target_disable_flag) \ OZ (role_id_array.push_back(role_id)); \ OZ (disable_flag_array.push_back(target_disable_flag)); \ OX (need_flush = true); int ObDDLService::build_need_flush_role_array( ObSchemaGetterGuard &schema_guard, uint64_t tenant_id, const ObUserInfo *user_info, const ObAlterUserProfileArg &arg, bool &need_flush, ObIArray &role_id_array, ObIArray &disable_flag_array) { int ret = OB_SUCCESS; uint64_t role_id; uint64_t target_disable_flag = 0; uint64_t org_disable_flag = 0; ObSEArray org_role_id_array; ObSEArray org_disable_flag_array; UNUSED(schema_guard); UNUSED(tenant_id); need_flush = false; CK (OB_NOT_NULL(user_info)); switch (arg.default_role_flag_) { case OB_DEFAULT_ROLE_ALL: case OB_DEFAULT_ROLE_NONE: if (OB_DEFAULT_ROLE_ALL == arg.default_role_flag_) { target_disable_flag = 0; } else { target_disable_flag = 1; } CK (user_info->get_role_count() == user_info->get_role_id_option_array().count()); for (int i = 0; OB_SUCC(ret) && i < user_info->get_role_count(); i++) { role_id = user_info->get_role_id_array().at(i); OX (org_disable_flag = user_info->get_disable_option( user_info->get_role_id_option_array().at(i))); if (OB_SUCC(ret) && org_disable_flag != target_disable_flag) { PUSH_BACK_ROLE_ID_INFO(target_disable_flag); } } break; case OB_DEFAULT_ROLE_LIST: case OB_DEFAULT_ROLE_ALL_EXCEPT: uint64_t invited_target_disable_flag; uint64_t black_target_disable_flag; if (arg.default_role_flag_ == OB_DEFAULT_ROLE_LIST) { invited_target_disable_flag = 0; black_target_disable_flag = 1; } else { invited_target_disable_flag = 1; black_target_disable_flag = 0; } CK (user_info->get_role_count() == user_info->get_role_id_option_array().count()); for (int i = 0; OB_SUCC(ret) && i < user_info->get_role_count(); i++) { role_id = user_info->get_role_id_array().at(i); OX (org_disable_flag = user_info->get_disable_option( user_info->get_role_id_option_array().at(i))); if (OB_SUCC(ret)) { if (has_exist_in_array(arg.role_id_array_, role_id)) { // If it is in the whitelist and the original has been disabled, update is required if (org_disable_flag != invited_target_disable_flag) { PUSH_BACK_ROLE_ID_INFO(invited_target_disable_flag); } } else { // If it is not in the whitelist and the original has been enabled, update is required if (org_disable_flag != black_target_disable_flag) { PUSH_BACK_ROLE_ID_INFO(black_target_disable_flag); } } } } break; default: ret = OB_INVALID_ARGUMENT; break; } return ret; } int ObDDLService::alter_user_default_role(const ObAlterUserProfileArg &arg) { int ret = OB_SUCCESS; uint64_t tenant_id = arg.tenant_id_; ObSchemaGetterGuard schema_guard; ObSEArray role_id_array; ObSEArray disable_flag_array; bool need_flush = true; const ObUserInfo *user_info = NULL; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid input schema", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else { /* 1. check user exists */ if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.user_id_, user_info))) { LOG_WARN("get user info fail", K(tenant_id), K(arg.user_id_)); } else if (NULL == user_info) { ret = OB_ERR_USER_NOT_EXIST; LOG_WARN("user is null", K(ret)); } else { /* 2. build role disable flag array */ OZ (build_need_flush_role_array(schema_guard, tenant_id, user_info, arg, need_flush, role_id_array, disable_flag_array)); } } if (OB_SUCC(ret) && need_flush) { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.alter_user_default_role(arg.ddl_stmt_str_, *user_info, role_id_array, disable_flag_array, trans))) { LOG_WARN("fail to alter user profile", K(ret), K(user_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::alter_user_profile(const ObAlterUserProfileArg &arg) { int ret = OB_SUCCESS; const ObUserInfo *user = NULL; const ObProfileSchema *profile = NULL; uint64_t tenant_id = arg.tenant_id_; ObSchemaGetterGuard schema_guard; uint64_t profile_id = OB_INVALID_ID; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid input schema", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else if (OB_INVALID_ID != arg.default_role_flag_) { OZ (alter_user_default_role(arg)); } else if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.user_name_, arg.host_name_, user))) { LOG_WARN("user not exist", K(ret), K(arg.user_name_), K(arg.host_name_), K(tenant_id)); } else if (OB_ISNULL(user)) { ret = OB_ERR_USER_NOT_EXIST; LOG_WARN("user is null", K(ret)); } else if (arg.profile_name_.empty()) { // If it is invalid id, the user is not associated with any profile profile_id = OB_INVALID_ID; } else { if (OB_FAIL(schema_guard.get_profile_schema_by_name(tenant_id, arg.profile_name_, profile))) { LOG_WARN("fail to get profile", K(ret)); } else if (OB_ISNULL(profile)) { ret = OB_ERR_PROFILE_STRING_DOES_NOT_EXIST; LOG_USER_ERROR(OB_ERR_PROFILE_STRING_DOES_NOT_EXIST, arg.profile_name_.length(), arg.profile_name_.ptr()); } else { profile_id = profile->get_profile_id(); } } if (OB_SUCC(ret) && OB_INVALID_ID == arg.default_role_flag_) { ObUserInfo user_info = *user; user_info.set_profile_id(profile_id); ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.alter_user_profile(arg.ddl_stmt_str_, user_info, trans))) { LOG_WARN("fail to alter user profile", K(ret), K(user_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::lock_user(const obrpc::ObLockUserArg &arg, ObIArray &failed_index) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; failed_index.reset(); ObSqlString ddl_stmt_str; ObString ddl_sql; uint64_t user_id = OB_INVALID_ID; ObAccountArg account; for (int64_t i = 0; OB_SUCC(ret) && i < arg.users_.count(); ++i) { ObSchemaGetterGuard schema_guard; //FIXME@xiyu: inside for ? ddl_stmt_str.reuse(); ddl_sql.reset(); user_id = OB_INVALID_ID; account.user_name_ = arg.users_.at(i); account.host_name_ = arg.hosts_.at(i);; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { if (OB_FAIL(schema_guard.get_user_id(tenant_id, account.user_name_, account.host_name_, user_id))) { LOG_WARN("get_user_id failed", K(ret), K(ret), K(account)); } else if (OB_INVALID_ID == user_id) { ret = OB_SUCCESS; //no such user, recover LOG_WARN("Try to lock/unlock non-exist user", K(tenant_id), K(account)); if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } else if (OB_FAIL(ObDDLSqlGenerator::gen_lock_user_sql(account, arg.locked_, ddl_stmt_str))) { LOG_WARN("gen lock_user sql failed", K(ret), K(account), K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(lock_user_in_trans(tenant_id, user_id, arg.locked_, &ddl_sql, schema_guard))) { LOG_WARN("Lock one user failed", K(tenant_id), K(user_id), K(account), "locked", arg.locked_, K(ret)); ret = OB_SUCCESS; //lock fail, try next, recover if (OB_FAIL(failed_index.push_back(i))) { LOG_WARN("push_back failed", K(ret)); } } } } return ret; } int ObDDLService::lock_user_in_trans(const uint64_t tenant_id, const uint64_t user_id, const bool locked, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant id is invalid", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.lock_user(tenant_id, user_id, locked, ddl_stmt_str, trans))) { LOG_WARN("fail to lock user", K(ret), K(user_id), K(locked)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::standby_grant(const ObStandbyGrantArg &arg) { int ret = OB_SUCCESS; ObNeedPriv need_priv(arg.db_, arg.table_, arg.priv_level_, arg.priv_set_, false); ObString ddl_sql; ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(arg.tenant_id_, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(arg)); } else if (OB_PRIV_DB_LEVEL == arg.priv_level_) { ObOriginalDBKey db_key(arg.tenant_id_, arg.user_id_, need_priv.db_); if (OB_FAIL(grant_database(db_key, need_priv.priv_set_, &ddl_sql, schema_guard))) { LOG_WARN("fail to grant database", KR(ret)); } } else if (OB_PRIV_TABLE_LEVEL == arg.priv_level_) { ObTablePrivSortKey table_key(arg.tenant_id_, arg.user_id_, need_priv.db_, need_priv.table_); if (OB_FAIL(grant_table(table_key, need_priv.priv_set_, &ddl_sql, schema_guard))) { LOG_WARN("Grant table error", KR(ret), K(ddl_sql), K(table_key)); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid priv level", KR(ret), K(arg)); } return ret; } int ObDDLService::grant_database( const share::schema::ObOriginalDBKey &db_key, const ObPrivSet priv_set, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = db_key.tenant_id_; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if(!db_key.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("db_key is invalid", K(db_key), K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (!is_user_exist(db_key.tenant_id_, db_key.user_id_)) { ret = OB_USER_NOT_EXIST; LOG_WARN("User is not exist", "tenant_id", db_key.tenant_id_, "user_id", db_key.user_id_, K(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.grant_database(db_key, priv_set, ddl_stmt_str, trans))) { LOG_WARN("fail to grant database", K(ret), K(db_key), K(priv_set)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::revoke_database( const share::schema::ObOriginalDBKey &db_key, const ObPrivSet priv_set) { int ret = OB_SUCCESS; const uint64_t tenant_id = db_key.tenant_id_; ObSchemaGetterGuard schema_guard; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (!db_key.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("db_key is invalid", K(db_key), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); if (!is_user_exist(db_key.tenant_id_, db_key.user_id_)) { ret = OB_USER_NOT_EXIST; LOG_WARN("User is not exist", "tenant_id", db_key.tenant_id_, "user_id", db_key.user_id_, K(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.revoke_database(db_key, priv_set, trans))) { LOG_WARN("fail to revoke database", K(ret), K(db_key), K(priv_set)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::revoke_syspriv( const uint64_t tenant_id, const uint64_t grantee_id, const share::ObRawPrivArray &sys_priv_array, const common::ObSArray &role_ids, const common::ObString *ddl_stmt_str) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; ObSysPriv *sys_priv = NULL; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); if (!is_user_exist(tenant_id, grantee_id)) { /* error because: grantee schema error */ ret = OB_ERR_USER_NOT_EXIST; LOG_WARN("User or Role is not exist", "tenant_id", tenant_id, "user_id", grantee_id, K(ret)); } OZ (schema_guard.get_sys_priv_with_grantee_id(tenant_id, grantee_id, sys_priv), tenant_id, grantee_id); OZ (trans.start(sql_proxy_, tenant_id, refreshed_schema_version)); // revoke sys_priv ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_SUCC(ret) && false == sys_priv_array.empty()) { OZ (ddl_operator.grant_sys_priv_to_ur(tenant_id, grantee_id, sys_priv, NO_OPTION, sys_priv_array, trans, false /*is_grant*/, ddl_stmt_str, schema_guard), tenant_id, grantee_id, sys_priv_array); } // revoke role if (OB_SUCC(ret) && false == role_ids.empty()) { bool log_operation = true; if (true == sys_priv_array.empty()) { // ddl_stmt_str was logged in grant_sys_priv_to_ur before log_operation = true; } else { // no sys_priv, sql_string shall be logged in revoke_role_inner_trans log_operation = false; } if (OB_FAIL(revoke_role_inner_trans(ddl_operator, trans, schema_guard, tenant_id, grantee_id, role_ids, log_operation))) { LOG_WARN("revoke_role_inner_trans failed", K(ret), K(tenant_id), K(grantee_id), K(role_ids)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::revoke_role_inner_trans( ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, const uint64_t user_id, const common::ObSArray &role_ids, const bool log_operation) { int ret = OB_SUCCESS; const ObUserInfo *user_info = NULL; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(tenant_id), K(user_id)); } else if (OB_FAIL(schema_guard.get_user_info(tenant_id, user_id, user_info))) { LOG_WARN("get_user_info failed", K(tenant_id), K(user_id), K(ret)); } else if (OB_ISNULL(user_info)) { ret = OB_USER_NOT_EXIST; LOG_WARN("user not exist", K(ret), K(user_id)); } else if (role_ids.count() > 0) { ObArray role_id_array; for (int64_t i = 0; OB_SUCC(ret) && i < role_ids.count(); ++i) { const uint64_t role_id = role_ids.at(i); const ObUserInfo *role_info = NULL; if (OB_FAIL(schema_guard.get_user_info(tenant_id, role_id, role_info))) { LOG_WARN("Failed to get role info", K(ret), K(tenant_id), K(role_id)); } else if (NULL == role_info) { ret = OB_ERR_UNEXPECTED; LOG_WARN("role doesn't exist", K(ret), K(role_id)); } else { // Determine whether the current role is granted to the user; otherwise, it will not be processed ObSEArray role_id_array_from_user = user_info->get_role_id_array(); for (int j = 0, found_role = 0; OB_SUCC(ret) && 0 == found_role && j < role_id_array_from_user.count(); ++j) { if (role_info->get_user_id() == role_id_array_from_user.at(j)) { if (OB_FAIL(role_id_array.push_back(role_id_array_from_user.at(j)))) { LOG_WARN("Failed to push back role_id", K(ret), K(tenant_id), K(*role_info)); } found_role = 1; } } } } if (OB_SUCC(ret)) { if (role_id_array.count() > 0) { if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id, *user_info, role_id_array, NULL, /*Single specified role info*/ trans, log_operation, /*log_operation*/ false, /*is_grant*/ NO_OPTION))) { LOG_WARN("failed to grant_revoke_role", K(ret), K(tenant_id), K(*user_info)); } } } } return ret; } int ObDDLService::grant_table( const share::schema::ObTablePrivSortKey &table_key, const ObPrivSet priv_set, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_key.tenant_id_; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (!table_key.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("table_key is invalid", K(table_key), K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); if (!is_user_exist(table_key.tenant_id_, table_key.user_id_)) { ret = OB_USER_NOT_EXIST; LOG_WARN("User is not exist", "tenant_id", table_key.tenant_id_, "user_id", table_key.user_id_, K(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.grant_table(table_key, priv_set, ddl_stmt_str, trans, share::ObRawObjPrivArray(), 0, ObObjPrivSortKey()))) { LOG_WARN("fail to grant table", K(ret), K(table_key), K(priv_set)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::grant_table( const share::schema::ObTablePrivSortKey &table_key, const ObPrivSet priv_set, const ObString *ddl_stmt_str, const share::ObRawObjPrivArray &obj_priv_array, const uint64_t option, const share::schema::ObObjPrivSortKey &obj_key, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_key.tenant_id_; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", KR(ret)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (!table_key.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("table_key is invalid", K(table_key), K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); if (!is_user_exist(table_key.tenant_id_, table_key.user_id_)) { ret = OB_USER_NOT_EXIST; LOG_WARN("User is not exist", "tenant_id", table_key.tenant_id_, "user_id", table_key.user_id_, K(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.grant_table(table_key, priv_set, ddl_stmt_str, trans, obj_priv_array, option, obj_key))) { LOG_WARN("fail to grant table", K(ret), K(table_key), K(priv_set)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::revoke_table( const share::schema::ObTablePrivSortKey &table_key, const ObPrivSet priv_set, const share::schema::ObObjPrivSortKey &obj_key, const share::ObRawObjPrivArray &obj_priv_array, const bool revoke_all_ora) { int ret = OB_SUCCESS; const uint64_t tenant_id = table_key.tenant_id_; int64_t refreshed_schema_version = 0; ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (!table_key.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("table_key is invalid", K(table_key), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); if (!is_user_exist(table_key.tenant_id_, table_key.user_id_)) { ret = OB_USER_NOT_EXIST; LOG_WARN("User is not exist", "tenant_id", table_key.tenant_id_, "user_id", table_key.user_id_, K(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.revoke_table(table_key, priv_set, trans, obj_key, obj_priv_array, revoke_all_ora))) { LOG_WARN("fail to revoke table", K(ret), K(table_key), K(priv_set)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } // publish schema if (OB_SUCC(ret)) { ret = publish_schema(tenant_id); if (OB_FAIL(ret)) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::check_outline_exist(share::schema::ObOutlineInfo &outline_info, const bool create_or_replace, bool &is_update) { //can not create same signature with different name, so we must check if signature //exists first int ret = OB_SUCCESS; const uint64_t tenant_id = outline_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_UNLIKELY(OB_INVALID_ID == outline_info.get_tenant_id() || OB_INVALID_ID == outline_info.get_database_id() || outline_info.get_name_str().empty() || (outline_info.get_signature_str().empty() && !ObOutlineInfo::is_sql_id_valid(outline_info.get_sql_id_str())))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(outline_info), K(ret)); } else { is_update = false; bool is_outline_exist_with_name = false; bool is_outline_exist_with_signature_or_sql_id = false; uint64_t outline_id = OB_INVALID_ID; if (OB_FAIL(schema_service_->check_outline_exist_with_name( outline_info.get_tenant_id(), outline_info.get_database_id(), outline_info.get_name_str(), outline_id, is_outline_exist_with_name))) { LOG_WARN("failed to check if outline_name exists", K(outline_info), K(ret)); } else { if (!outline_info.get_signature_str().empty()) { if (OB_FAIL(schema_service_->check_outline_exist_with_sql( outline_info.get_tenant_id(), outline_info.get_database_id(), outline_info.get_signature_str(), is_outline_exist_with_signature_or_sql_id))) { LOG_WARN("failed to check if signature exist", K(outline_info), K(ret)); } } else { if (OB_FAIL(schema_service_->check_outline_exist_with_sql_id( outline_info.get_tenant_id(), outline_info.get_database_id(), outline_info.get_sql_id_str(), is_outline_exist_with_signature_or_sql_id))) { LOG_WARN("failed to check if sql id exist", K(outline_info), K(ret)); } } } if (OB_SUCCESS != ret) { } else if (create_or_replace && is_outline_exist_with_name && is_outline_exist_with_signature_or_sql_id) { const ObOutlineInfo *orig_outline = NULL; ObSchemaGetterGuard schema_guard; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_outline_info_with_name( outline_info.get_tenant_id(), outline_info.get_database_id(), outline_info.get_name_str(), orig_outline))) { LOG_WARN("failed to get origin outline info", K(outline_info), K(ret)); } else if (OB_ISNULL(orig_outline)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("got orig outline is NULL when replace", K(outline_info), K(ret)); } else if (OB_UNLIKELY(orig_outline->get_signature_str() != outline_info.get_signature_str())) { ret = OB_INVALID_OUTLINE; LOG_USER_ERROR(OB_INVALID_OUTLINE, "create or replace outline with same name, but sql signatures are different"); } else { outline_info.set_outline_id(orig_outline->get_outline_id()); is_update = true;//create or replace and the outline with same name and same paramlized sql exists } } else if (is_outline_exist_with_name) { ret = OB_ERR_OUTLINE_EXIST; LOG_USER_ERROR(OB_ERR_OUTLINE_EXIST, outline_info.get_name_str().length(), outline_info.get_name_str().ptr()); } else if (is_outline_exist_with_signature_or_sql_id) { ret = OB_ERR_OUTLINE_EXIST; LOG_USER_ERROR(OB_ERR_OUTLINE_EXIST, outline_info.get_sql_text_str().length(), outline_info.get_sql_text_str().ptr()); } else {/*do nothing*/} } return ret; } int ObDDLService::create_outline(share::schema::ObOutlineInfo &outline_info, const bool is_update, const common::ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = outline_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } // else if (!outline_info.is_valid()) { // ret = OB_INVALID_ARGUMENT; // LOG_WARN("invalid argument", K(outline_info), K(is_update), K(ret)); // } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { if (!is_update) { ret = ddl_operator.create_outline(outline_info, trans, ddl_stmt_str); } else { ret = ddl_operator.replace_outline(outline_info, trans, ddl_stmt_str); } if (OB_FAIL(ret)) { LOG_WARN("failed to create outline", K(outline_info), K(is_update), K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } LOG_INFO("finish create outline", K(outline_info), K(is_update), K(ret)); return ret; } int ObDDLService::alter_outline_in_trans(const obrpc::ObAlterOutlineArg &arg) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { const ObAlterOutlineInfo &alter_outline_info = arg.alter_outline_info_; ObOutlineInfo new_outline_info; uint64_t tenant_id = alter_outline_info.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id)); } else { const ObString &database_name = arg.db_name_; const ObString &outline_name = alter_outline_info.get_name_str(); const ObOutlineInfo *orig_outline_info = NULL; if (database_name.empty() || outline_name.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("database name or outline name is empty", K(alter_outline_info), K(database_name), K(outline_name), K(ret)); } else if (OB_FAIL(schema_guard.get_outline_info_with_name(tenant_id, database_name, outline_name, orig_outline_info))) { LOG_WARN("failed to get_outline_info_with_name", K(tenant_id), K(database_name), K(outline_name), K(ret)); } else if (NULL == orig_outline_info) { ret = OB_OUTLINE_NOT_EXIST; LOG_USER_ERROR(OB_OUTLINE_NOT_EXIST, database_name.length(), database_name.ptr(), outline_name.length(), outline_name.ptr()); } else if (OB_UNLIKELY( orig_outline_info->get_signature_str() != alter_outline_info.get_signature_str())) { ret = OB_INVALID_OUTLINE; LOG_USER_ERROR(OB_INVALID_OUTLINE, "cannot change existing outline using a different SQL text"); } else { //copy from the old outline info new_outline_info = *orig_outline_info; const ObString &outline_name = new_outline_info.get_name_str(); if (alter_outline_info.get_alter_option_bitset().has_member( obrpc::ObAlterOutlineArg::ADD_OUTLINE_CONTENT)) { //add outline_content const ObString &orig_outline_content = orig_outline_info->get_outline_content_str(); if (!orig_outline_content.empty()) { ret = OB_ERR_OUTLINE_CONTENT_EXIST; LOG_USER_ERROR(OB_ERR_OUTLINE_CONTENT_EXIST, orig_outline_content.length(), orig_outline_content.ptr(), outline_name.length(), outline_name.ptr()); LOG_WARN("the outline has already has outline content", K(new_outline_info), K(ret)); } else if (alter_outline_info.get_outline_content_str().empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("the outline content should not be empty", K(alter_outline_info), K(ret)); } else { new_outline_info.set_outline_content(alter_outline_info.get_outline_content_str()); // Only this place is set to sql_text, to ensure that if there is an outline on the sql statement, // the content in sql_text is the sql that created the outline new_outline_info.set_sql_text(alter_outline_info.get_sql_text_str()); } } else if (alter_outline_info.get_alter_option_bitset().has_member( obrpc::ObAlterOutlineArg::ADD_CONCURRENT_LIMIT)) { //add outline_max_concurrent const ObOutlineParamsWrapper &outline_params_wrapper = alter_outline_info.get_outline_params_wrapper(); if (OB_UNLIKELY(1 != outline_params_wrapper.get_outline_params().count())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid alter_outline_info, count of outline_params should be 1", K(alter_outline_info), K(ret)); } else { ObMaxConcurrentParam *concurrent_param = outline_params_wrapper.get_outline_params().at(0); bool has_param = false; if (OB_ISNULL(concurrent_param)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("concurrent_param is NULL", K(ret)); } else if (OB_FAIL(orig_outline_info->get_outline_params_wrapper().has_param(*concurrent_param, has_param))) { LOG_WARN("failed to check if has param", K(ret)); } else if (has_param) { ret = OB_ERR_OUTLINE_MAX_CONCURRENT_EXIST; LOG_USER_ERROR(OB_ERR_OUTLINE_MAX_CONCURRENT_EXIST, outline_name.length(), outline_name.ptr()); } else if (OB_FAIL(new_outline_info.get_outline_params_wrapper().add_param(*concurrent_param))) { LOG_WARN("failed to add param", K(ret)); } else {/*do nothing*/} } } else {/*do nothing*/} if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.alter_outline(new_outline_info, trans, &arg.ddl_stmt_str_))) { LOG_WARN("failed to alter outline", K(new_outline_info), K(ret)); } else {/*do nothing*/} if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } } } } return ret; } int ObDDLService::alter_outline(const obrpc::ObAlterOutlineArg &alter_outline_arg) { int ret = OB_SUCCESS; int64_t start_usec; int64_t end_usec; int64_t cost_usec; const uint64_t tenant_id = alter_outline_arg.alter_outline_info_.get_tenant_id(); start_usec = ObTimeUtility::current_time(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } //do alter table in transaction if (OB_SUCC(ret)) { if (OB_FAIL(alter_outline_in_trans(alter_outline_arg))) { LOG_WARN("alter_outline_in_trans failed", K(ret)); } end_usec = ObTimeUtility::current_time(); cost_usec = end_usec - start_usec; start_usec = end_usec; LOG_INFO("alter_outline_in_trans cost: ", K(cost_usec)); } //refresh table schema if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("refresh_schema after alter outline failed", K(ret)); } end_usec = ObTimeUtility::current_time(); cost_usec = end_usec - start_usec; LOG_INFO("publish_schema cost: ", K(cost_usec)); } return ret; } int ObDDLService::drop_outline(const obrpc::ObDropOutlineArg &arg) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; const ObString &database_name = arg.db_name_; const ObString &outline_name = arg.outline_name_; uint64_t outline_id = OB_INVALID_ID; ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || database_name.empty() || outline_name.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(tenant_id), K(database_name), K(outline_name), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { //check database uint64_t database_id = OB_INVALID_ID; if (OB_SUCC(ret)) { bool database_exist = false; if (database_name == OB_OUTLINE_DEFAULT_DATABASE_NAME) { database_id = OB_OUTLINE_DEFAULT_DATABASE_ID; database_exist = true; } else if (OB_FAIL(schema_service_->check_database_exist(tenant_id, database_name, database_id, database_exist))) { LOG_WARN("failed to check database exist!", K(tenant_id), K(database_name), K(database_id), K(database_exist), K(ret)); } else if (!database_exist) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr()); LOG_WARN("database not exist!", K(arg), K(ret)); } else if (OB_INVALID_ID == database_id) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr()); LOG_WARN("database is invalid!", K(arg), K(ret)); } } bool outline_exist = false; int64_t refreshed_schema_version = 0; if (OB_FAIL(ret)) { //do nothing } else if (OB_FAIL(schema_service_->check_outline_exist_with_name(tenant_id, database_id, outline_name, outline_id, outline_exist))) { LOG_WARN("check_outline_exist failed", K(tenant_id), K(database_name), K(outline_name), K(ret)); } else if (!outline_exist) { ret = OB_OUTLINE_NOT_EXIST; LOG_USER_ERROR(OB_OUTLINE_NOT_EXIST, database_name.length(), database_name.ptr(), outline_name.length(), outline_name.ptr()); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.drop_outline(tenant_id, database_id, outline_id, trans, &arg.ddl_stmt_str_))) { LOG_WARN("ddl_operator drop_outline failed", K(tenant_id), KT(outline_id), K(ret)); } else {/*do nothing*/} } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish drop outline", K(tenant_id), K(database_name), K(outline_name), K(ret)); return ret; } int ObDDLService::create_routine(ObRoutineInfo &routine_info, const ObRoutineInfo* old_routine_info, bool replace, ObErrorInfo &error_info, ObIArray &dep_infos, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; CK((replace && OB_NOT_NULL(old_routine_info)) || (!replace && OB_ISNULL(old_routine_info))); const uint64_t tenant_id = routine_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } if (OB_SUCC(ret)) { if (replace) { if (OB_FAIL(ddl_operator.replace_routine(routine_info, old_routine_info, trans, error_info, dep_infos, ddl_stmt_str))) { LOG_WARN("replace routine failded", K(routine_info), K(ret)); } } else { if (OB_FAIL(ddl_operator.create_routine(routine_info, trans, error_info, dep_infos, ddl_stmt_str))) { LOG_WARN("create procedure failed", K(ret), K(routine_info)); } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::alter_routine(const ObRoutineInfo &routine_info, ObErrorInfo &error_info, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = routine_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed!", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.alter_routine( routine_info, trans, error_info, ddl_stmt_str))) { LOG_WARN("alter routine failed!", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed!", K(ret), K(temp_ret)); ret = OB_SUCCESS == ret ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed!", K(ret), K(tenant_id)); } } } return ret; } int ObDDLService::drop_routine(const ObRoutineInfo &routine_info, ObErrorInfo &error_info, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = routine_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.drop_routine(routine_info, trans, error_info, ddl_stmt_str))) { LOG_WARN("drop procedure failed", K(ret), K(routine_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::create_udt(ObUDTTypeInfo &udt_info, const ObUDTTypeInfo* old_udt_info, bool replace, ObIArray &public_routine_infos, ObErrorInfo &error_info, ObSchemaGetterGuard &schema_guard, ObIArray &dep_infos, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; uint64_t tenant_id = udt_info.get_tenant_id(); CK((replace && OB_NOT_NULL(old_udt_info)) || (!replace && OB_ISNULL(old_udt_info))); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } if (OB_SUCC(ret)) { if (replace) { if (OB_FAIL(ddl_operator.replace_udt(udt_info, old_udt_info, trans, error_info, public_routine_infos, schema_guard, dep_infos, ddl_stmt_str))) { LOG_WARN("replace udt failded", K(udt_info), K(ret)); } } else { if (OB_FAIL(ddl_operator.create_udt(udt_info, trans, error_info, public_routine_infos, schema_guard, dep_infos, ddl_stmt_str))) { LOG_WARN("create udt failed", K(ret), K(udt_info)); } } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::drop_udt(const ObUDTTypeInfo &udt_info, ObSchemaGetterGuard &schema_guard, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; uint64_t tenant_id = udt_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.drop_udt(udt_info, trans, schema_guard, ddl_stmt_str))) { LOG_WARN("drop procedure failed", K(ret), K(udt_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } //----Functions for managing dblinks---- int ObDDLService::create_dblink(const obrpc::ObCreateDbLinkArg &arg, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; ObDbLinkInfo dblink_info = arg.dblink_info_; uint64_t tenant_id = dblink_info.get_tenant_id(); const ObString &dblink_name = dblink_info.get_dblink_name(); ObSchemaGetterGuard schema_guard; bool is_exist = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.check_dblink_exist(tenant_id, dblink_name, is_exist))) { LOG_WARN("failed to check dblink exist", K(ret), K(dblink_info)); } else if (is_exist) { ret = OB_OBJ_ALREADY_EXIST; LOG_WARN("dblink already exist", K(ret), K(dblink_info.get_dblink_name())); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.create_dblink(dblink_info, trans, ddl_stmt_str))) { LOG_WARN("failed to create dblink", K(ret), K(dblink_info)); } if (trans.is_started()) { int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("failed to end trans", K(tmp_ret), "is_commit", OB_SUCC(ret)); ret = (OB_SUCC(ret)) ? tmp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("failed to publish schema", K(ret), K(tenant_id)); } } } LOG_INFO("finish create dblink", K(ret), K(dblink_info)); return ret; } int ObDDLService::drop_dblink(const obrpc::ObDropDbLinkArg &arg, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; uint64_t tenant_id = arg.tenant_id_; const ObString &dblink_name = arg.dblink_name_; ObSchemaGetterGuard schema_guard; const ObDbLinkSchema *dblink_schema = NULL; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_dblink_schema(tenant_id, dblink_name, dblink_schema))) { LOG_WARN("failed to get dblink schema", K(ret), K(tenant_id), K(dblink_name)); } else if (OB_ISNULL(dblink_schema)) { ret = OB_DBLINK_NOT_EXIST_TO_DROP; LOG_WARN("dblink not exist", K(ret), K(tenant_id), K(dblink_name)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDbLinkBaseInfo dblink_info = *dblink_schema; int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.drop_dblink(dblink_info, trans, ddl_stmt_str))) { LOG_WARN("failed to drop dblink", K(ret), K(dblink_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("failed to end trans", K(temp_ret), "is_commit", OB_SUCC(ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("failed to publish schema", K(ret), K(tenant_id)); } } } LOG_INFO("finish drop dblink", K(ret), K(tenant_id), K(dblink_name)); return ret; } //----End of functions for managing dblinks---- int ObDDLService::check_synonym_exist(share::schema::ObSynonymInfo &synonym_info, const bool create_or_replace, bool &is_update) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_UNLIKELY(OB_INVALID_ID == synonym_info.get_tenant_id() || OB_INVALID_ID == synonym_info.get_database_id() || synonym_info.get_synonym_name_str().empty())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(synonym_info.get_synonym_name_str()), K(ret)); } else { bool is_exist = false; const bool is_index = false; uint64_t synonym_id = OB_INVALID_ID; // Indicates to take the latest local schema_guard judgment const int64_t table_schema_version = OB_INVALID_VERSION; if (OB_FAIL(schema_service_->check_table_exist(synonym_info.get_tenant_id(), synonym_info.get_database_id(), synonym_info.get_synonym_name_str(), is_index, table_schema_version, is_exist))) { LOG_WARN("fail to check table exist", K(synonym_info), K(ret)); } else if (is_exist) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("Name is already used by an existing object", K(synonym_info), K(ret)); } else if (OB_FAIL(schema_service_->check_synonym_exist(synonym_info.get_tenant_id(), synonym_info.get_database_id(), synonym_info.get_synonym_name_str(), is_exist, synonym_id))) { LOG_WARN("failed to check if synonym_name exists", K(synonym_info), K(ret)); } else if (is_exist) { if (OB_LIKELY(create_or_replace)) { synonym_info.set_synonym_id(synonym_id); is_update = true; } else { ret = OB_ERR_SYNONYM_EXIST; LOG_USER_ERROR(OB_ERR_SYNONYM_EXIST, synonym_info.get_synonym_name_str().length(), synonym_info.get_synonym_name_str().ptr()); } } } return ret; } int ObDDLService::create_synonym(share::schema::ObSynonymInfo &synonym_info, const common::ObString *ddl_stmt_str, bool is_update, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = synonym_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { if (!is_update) { ret = ddl_operator.create_synonym(synonym_info, trans, ddl_stmt_str); } else { ret = ddl_operator.replace_synonym(synonym_info, trans, ddl_stmt_str); } if (OB_FAIL(ret)) { LOG_WARN("failed to create synonym", K(synonym_info), K(is_update), K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } LOG_INFO("finish create synonym", K(synonym_info), K(is_update), K(ret)); return ret; } int ObDDLService::drop_synonym(const obrpc::ObDropSynonymArg &arg) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; const ObString &database_name = arg.db_name_; const ObString &synonym_name = arg.synonym_name_; uint64_t database_id = OB_INVALID_ID; ObDDLSQLTransaction trans(schema_service_); bool database_exist = false; ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_UNLIKELY(false == arg.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(arg), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_service_->check_database_exist(tenant_id, database_name, database_id, database_exist))) { LOG_WARN("failed to check database exist!", K(tenant_id), K(database_name), K(database_id), K(database_exist), K(ret)); } else if (OB_UNLIKELY(!database_exist)) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr()); LOG_WARN("database not exist!", K(arg), K(ret)); } else if (OB_UNLIKELY(OB_INVALID_ID == database_id)) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr()); LOG_WARN("database is invalid!", K(arg), K(ret)); } //check synonym exist & drop synonym if (OB_SUCC(ret)) { bool is_exist = false; uint64_t synonym_id = OB_INVALID_ID; int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_service_->check_synonym_exist(tenant_id, database_id, synonym_name, is_exist, synonym_id))) { LOG_WARN("check_synonym_exist failed", K(tenant_id), K(database_name), K(synonym_name), K(ret)); } else if (!is_exist) { ret = OB_SYNONYM_NOT_EXIST; LOG_USER_ERROR(OB_SYNONYM_NOT_EXIST, database_name.length(), database_name.ptr(), synonym_name.length(), synonym_name.ptr()); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.drop_synonym(tenant_id, database_id, synonym_id, trans, &arg.ddl_stmt_str_))) { LOG_WARN("ddl_operator drop_synonym failed", K(tenant_id), KT(synonym_id), K(ret)); } else {/*do nothing*/} } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish drop synonym", K(tenant_id), K(database_name), K(synonym_name), K(ret)); return ret; } int ObDDLService::create_package(ObSchemaGetterGuard &schema_guard, const ObPackageInfo *old_package_info, ObPackageInfo &new_package_info, ObIArray &public_routine_infos, ObErrorInfo &error_info, ObIArray &dep_infos, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { const uint64_t tenant_id = new_package_info.get_tenant_id(); ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.create_package(old_package_info, new_package_info, trans, schema_guard, public_routine_infos, error_info, dep_infos, ddl_stmt_str))) { LOG_WARN("create package failed", K(ret), K(new_package_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::drop_package(const ObPackageInfo &package_info, ObErrorInfo &error_info, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; const uint64_t tenant_id = package_info.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.drop_package(package_info, trans, schema_guard, error_info, ddl_stmt_str))) { LOG_WARN("drop procedure failed", K(ret), K(package_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::create_trigger(const ObCreateTriggerArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; ObTriggerInfo new_trigger_info = arg.trigger_info_; //for_insert_errors_ is false, Indicates that the trigger is created normally //true Indicates that the error message is inserted into the system table after the trigger is created //So the following steps can be skipped uint64_t tenant_id = new_trigger_info.get_tenant_id(); uint64_t trigger_database_id = OB_INVALID_ID; uint64_t base_object_id = OB_INVALID_ID; ObSchemaType base_object_type = OB_MAX_SCHEMA; const ObString &trigger_database = arg.trigger_database_; const ObString &base_object_database = arg.base_object_database_; const ObString &base_object_name = arg.base_object_name_; if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("o get schema guard in inner table failed", KR(ret), K(tenant_id)); } else if (!arg.for_insert_errors_) { const ObTriggerInfo *old_trigger_info = NULL; if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_database_id(schema_guard, tenant_id, trigger_database, trigger_database_id))) { LOG_WARN("get database id failed", K(ret)); } else if (OB_FAIL(get_object_info(schema_guard, tenant_id, base_object_database, base_object_name, base_object_type, base_object_id))) { LOG_WARN("get base object info failed", K(ret)); } else if (FALSE_IT(new_trigger_info.set_database_id(trigger_database_id))) { } else if (FALSE_IT(new_trigger_info.set_base_object_type(base_object_type))) { } else if (FALSE_IT(new_trigger_info.set_base_object_id(base_object_id))) { } else if (OB_FAIL(try_get_exist_trigger(schema_guard, new_trigger_info, old_trigger_info, arg.with_replace_))) { LOG_WARN("check trigger exist failed", K(ret)); } else { if (NULL != old_trigger_info) { new_trigger_info.set_trigger_id(old_trigger_info->get_trigger_id()); } } } if (OB_SUCC(ret) && OB_FAIL(create_trigger_in_trans(new_trigger_info, const_cast(arg.error_info_), &arg.ddl_stmt_str_, arg.for_insert_errors_, schema_guard))) { LOG_WARN("create trigger in trans failed", K(ret)); } return ret; } int ObDDLService::create_trigger_in_trans(ObTriggerInfo &trigger_info, ObErrorInfo &error_info, const ObString *ddl_stmt_str, bool for_insert_errors, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = trigger_info.get_tenant_id(); ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.create_trigger(trigger_info, trans, error_info, ddl_stmt_str, for_insert_errors))) { LOG_WARN("create trigger failed", K(ret), K(trigger_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::drop_trigger(const ObDropTriggerArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; uint64_t tenant_id = arg.tenant_id_; uint64_t trigger_database_id = OB_INVALID_ID; const ObString &trigger_database = arg.trigger_database_; const ObString &trigger_name = arg.trigger_name_; const ObTriggerInfo *trigger_info = NULL; bool is_ora_mode = false; if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); } else if (ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_ora_mode)) { LOG_WARN("fail to check is oracle mode", K(ret)); } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("get schema guard in inner table failed", K(ret)); } else if (OB_FAIL(get_database_id(schema_guard, tenant_id, trigger_database, trigger_database_id))) { LOG_WARN("get database id failed", K(ret)); } else if (OB_FAIL(schema_guard.get_trigger_info(tenant_id, trigger_database_id, trigger_name, trigger_info))) { LOG_WARN("get trigger info failed", K(ret), K(trigger_database), K(trigger_name)); } else if (OB_ISNULL(trigger_info)) { ret = OB_ERR_TRIGGER_NOT_EXIST; if (is_ora_mode) { LOG_ORACLE_USER_ERROR(OB_ERR_TRIGGER_NOT_EXIST, trigger_name.length(), trigger_name.ptr()); } } else if (trigger_info->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("trigger is in recyclebin", K(ret), K(trigger_info->get_trigger_id()), K(trigger_info->get_trigger_name())); } else if (OB_FAIL(drop_trigger_in_trans(*trigger_info, &arg.ddl_stmt_str_, schema_guard))) { LOG_WARN("drop trigger in trans failed", K(ret), K(trigger_database), K(trigger_name)); } if (!is_ora_mode && (OB_ERR_TRIGGER_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret)) { ret = OB_ERR_TRIGGER_NOT_EXIST; if (arg.if_exist_) { ret = OB_SUCCESS; LOG_MYSQL_USER_NOTE(OB_ERR_TRIGGER_NOT_EXIST); } else { LOG_MYSQL_USER_ERROR(OB_ERR_TRIGGER_NOT_EXIST); } LOG_WARN("trigger not exist", K(arg.trigger_database_), K(arg.trigger_name_), K(ret)); } return ret; } int ObDDLService::drop_trigger_in_trans(const ObTriggerInfo &trigger_info, const ObString *ddl_stmt_str, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; const uint64_t tenant_id = trigger_info.get_tenant_id(); ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.drop_trigger(trigger_info, trans, ddl_stmt_str))) { LOG_WARN("drop trigger failed", K(ret), K(trigger_info)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::alter_trigger(const ObAlterTriggerArg &arg) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; uint64_t tenant_id = OB_INVALID_ID; bool is_enable = false; int64_t refreshed_schema_version = 0; CK (OB_NOT_NULL(schema_service_) && OB_NOT_NULL(sql_proxy_)); CK (arg.is_valid(), OB_INVALID_ARGUMENT); OX (is_enable = arg.trigger_infos_.at(0).is_enable()); OZ (check_inner_stat()); OX (tenant_id = arg.exec_tenant_id_); OZ (get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard)); OZ (schema_guard.get_schema_version(tenant_id, refreshed_schema_version)); if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); OZ (trans.start(sql_proxy_, tenant_id, refreshed_schema_version), refreshed_schema_version); for (int64_t i = 0; OB_SUCC(ret) && i < arg.trigger_infos_.count(); ++i) { const ObTriggerInfo *old_tg_info = NULL; ObTriggerInfo new_tg_info = arg.trigger_infos_.at(i); OZ (schema_guard.get_trigger_info(tenant_id, new_tg_info.get_trigger_id(), old_tg_info)); CK (OB_NOT_NULL(old_tg_info), OB_ERR_TRIGGER_NOT_EXIST); OZ (new_tg_info.deep_copy(*old_tg_info)); OX (new_tg_info.set_is_enable(is_enable)); OZ (ddl_operator.alter_trigger(new_tg_info, trans, &arg.ddl_stmt_str_)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::try_get_exist_trigger(ObSchemaGetterGuard &schema_guard, const ObTriggerInfo &new_trigger_info, const ObTriggerInfo *&old_trigger_info, bool with_replace) { int ret = OB_SUCCESS; const ObString &trigger_name = new_trigger_info.get_trigger_name(); if (OB_FAIL(schema_guard.get_trigger_info(new_trigger_info.get_tenant_id(), new_trigger_info.get_database_id(), trigger_name, old_trigger_info))) { LOG_WARN("failed to get old trigger info", K(ret)); } else if (NULL != old_trigger_info) { if (new_trigger_info.get_base_object_id() != old_trigger_info->get_base_object_id()) { ret = OB_ERR_TRIGGER_EXIST_ON_OTHER_TABLE; LOG_USER_ERROR(OB_ERR_TRIGGER_EXIST_ON_OTHER_TABLE, trigger_name.length(), trigger_name.ptr()); } else if (!with_replace) { ret = OB_ERR_TRIGGER_ALREADY_EXIST; LOG_USER_ERROR(OB_ERR_TRIGGER_ALREADY_EXIST, trigger_name.length(), trigger_name.ptr()); } } return ret; } int ObDDLService::rebuild_trigger_package(ObSchemaGetterGuard &schema_guard, const ObTableSchema &table_schema, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; const ObDatabaseSchema *database_schema = NULL; const ObString *database_name = NULL; const ObString &table_name = table_schema.get_table_name_str(); const uint64_t tenant_id = table_schema.get_tenant_id(); OZ (schema_guard.get_database_schema(tenant_id, table_schema.get_database_id(), database_schema), table_schema.get_database_id()); OV (OB_NOT_NULL(database_schema), OB_ERR_UNEXPECTED, table_schema.get_database_id()); OX (database_name = &database_schema->get_database_name_str()); OZ (rebuild_trigger_package(schema_guard, tenant_id, table_schema.get_trigger_list(), *database_name, table_name, ddl_operator, trans)); return ret; } int ObDDLService::rebuild_trigger_package(ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, const ObIArray &trigger_list, const ObString &database_name, const ObString &table_name, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; const ObTriggerInfo *trigger_info = NULL; for (int64_t i = 0; OB_SUCC(ret) && i < trigger_list.count(); i++) { OZ (schema_guard.get_trigger_info(tenant_id, trigger_list.at(i), trigger_info), trigger_list.at(i)); OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_list.at(i)); OZ (ddl_operator.rebuild_trigger_package(*trigger_info, database_name, table_name, trans)); } return ret; } int ObDDLService::create_trigger_for_truncate_table(ObSchemaGetterGuard &schema_guard, const ObIArray &origin_trigger_list, ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans) { int ret = OB_SUCCESS; const ObTriggerInfo *origin_trigger_info = NULL; ObTriggerInfo new_trigger_info; ObString spec_source; ObString body_source; ObErrorInfo error_info; ObArenaAllocator inner_alloc; new_table_schema.get_trigger_list().reset(); bool is_update_table_schema_version = false; const ObDatabaseSchema *db_schema = NULL; const uint64_t tenant_id = new_table_schema.get_tenant_id(); OZ (schema_guard.get_database_schema(tenant_id, new_table_schema.get_database_id(), db_schema)); CK (db_schema != NULL); for (int64_t i = 0; OB_SUCC(ret) && i < origin_trigger_list.count(); i++) { is_update_table_schema_version = i == origin_trigger_list.count() - 1 ? true : false; uint64_t new_trigger_id = OB_INVALID_ID; OZ (schema_guard.get_trigger_info(tenant_id, origin_trigger_list.at(i), origin_trigger_info), origin_trigger_list.at(i)); if (OB_SUCC(ret)) { if (OB_FAIL(new_trigger_info.deep_copy(*origin_trigger_info))) { LOG_WARN("failed to create trigger for truncate table", K(ret)); } else if (OB_FAIL(schema_service_->get_schema_service()->fetch_new_trigger_id( origin_trigger_info->get_tenant_id(), new_trigger_id))) { LOG_WARN("failed to fetch_new_trigger_id", K(ret)); } else { new_trigger_info.set_trigger_id(new_trigger_id); new_trigger_info.set_base_object_id(new_table_schema.get_table_id()); new_table_schema.get_trigger_list().push_back(new_trigger_id); OZ (ObTriggerInfo::gen_package_source(new_trigger_info, db_schema->get_database_name(), new_table_schema.get_table_name(), spec_source, body_source, inner_alloc)); if (OB_SUCC(ret) && OB_FAIL(ddl_operator .create_trigger(new_trigger_info, trans, error_info, &origin_trigger_info->get_trigger_body(), false, /* for_insert_error */ is_update_table_schema_version, true))) { LOG_WARN("failed to create trigger for truncate table", K(ret)); } } } } return ret; } int ObDDLService::do_sequence_ddl(const obrpc::ObSequenceDDLArg &arg) { int ret = OB_SUCCESS; const ObString *ddl_stmt_str = &arg.ddl_stmt_str_; ObSequenceSchema seq_schema; const ObBitSet<> &opt_bitset = arg.get_option_bitset(); ObSchemaGetterGuard schema_guard; uint64_t database_id = OB_INVALID_ID; const uint64_t tenant_id = arg.seq_schema_.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); } else if (OB_FAIL(seq_schema.assign(arg.seq_schema_))) { LOG_WARN("fail assign sequence schema", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id)); } if (OB_SUCC(ret)) { if (OB_FAIL(get_database_id(schema_guard, tenant_id, arg.database_name_, database_id))) { LOG_WARN("fail get database id", K(ret)); } else { seq_schema.set_database_id(database_id); } } if (OB_SUCC(ret)) { ObSequenceDDLProxy ddl_operator(*schema_service_); ObDDLSQLTransaction trans(schema_service_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { switch(arg.get_stmt_type()) { case sql::stmt::T_CREATE_SEQUENCE: { if (OB_FAIL(ddl_operator.create_sequence(seq_schema, opt_bitset, trans, schema_guard, ddl_stmt_str))) { LOG_WARN("fail create sequence", K(arg), K(ret)); } break; } case sql::stmt::T_ALTER_SEQUENCE: { if (OB_FAIL(ddl_operator.alter_sequence(seq_schema, opt_bitset, trans, schema_guard, ddl_stmt_str, FROM_SEQUENCE_DDL))) { LOG_WARN("fail alter sequence", K(arg), K(ret)); } break; } case sql::stmt::T_DROP_SEQUENCE: { if (OB_FAIL(ddl_operator.drop_sequence(seq_schema, trans, schema_guard, ddl_stmt_str, FROM_SEQUENCE_DDL))) { LOG_WARN("fail drop sequence", K(arg), K(ret)); } break; } default: ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected stmt type", K(arg), K(ret)); break; } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish do sequence ddl", K(arg), K(ret)); return ret; } int ObDDLService::do_context_ddl(const obrpc::ObContextDDLArg &arg) { int ret = OB_SUCCESS; const ObString *ddl_stmt_str = &arg.ddl_stmt_str_; ObContextSchema context_schema; bool or_replace = arg.or_replace_; ObSchemaGetterGuard schema_guard; const uint64_t tenant_id = arg.ctx_schema_.get_tenant_id(); bool need_clean_global_ctx = false; int64_t refreshed_schema_version = 0; CK (OB_NOT_NULL(schema_service_)); if (OB_FAIL(ret)) { } else if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); } else if (OB_FAIL(context_schema.assign(arg.ctx_schema_))) { LOG_WARN("fail assign context schema", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id)); } if (OB_SUCC(ret)) { bool obj_exist = false; const ObContextSchema *old_schema = nullptr; ObContextDDLProxy ddl_operator(*schema_service_); ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(schema_guard.check_context_exist_with_name(context_schema.get_tenant_id(), context_schema.get_namespace(), old_schema, obj_exist))) { LOG_WARN("fail to check context exist", K(ret), K(context_schema)); } else { switch(arg.get_stmt_type()) { case sql::stmt::T_CREATE_CONTEXT: { if (OB_FAIL(ddl_operator.create_context(context_schema, trans, schema_guard, or_replace, obj_exist, old_schema, need_clean_global_ctx, ddl_stmt_str))) { LOG_WARN("fail create context", K(arg), K(ret)); } break; } case sql::stmt::T_DROP_CONTEXT: { if (!obj_exist) { ret = OB_ERR_OBJECT_STRING_DOES_NOT_EXIST; LOG_USER_ERROR(OB_ERR_OBJECT_STRING_DOES_NOT_EXIST, static_cast(context_schema.get_namespace().length()), context_schema.get_namespace().ptr()); } else if (OB_FAIL(ddl_operator.drop_context(context_schema, trans, schema_guard, old_schema, need_clean_global_ctx, ddl_stmt_str))) { LOG_WARN("fail drop context", K(arg), K(ret)); } break; } default: ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected stmt type", K(arg), K(ret)); break; } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret) && need_clean_global_ctx) { if (OB_FAIL(clean_global_context(context_schema))) { LOG_ERROR("succ to drop but failed to clean global context value", K(context_schema), K(ret)); ret = OB_SUCCESS; } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish do context ddl", K(arg), K(ret)); return ret; } int ObDDLService::clean_global_context(const ObContextSchema &context_schema) { int ret = OB_SUCCESS; ObGlobalContextOperator ctx_operator; CK (OB_NOT_NULL(sql_proxy_)); if (OB_FAIL(ret)) { } else if (OB_FAIL(ctx_operator.delete_global_contexts_by_id(context_schema.get_tenant_id(), context_schema.get_context_id(), *sql_proxy_))) { LOG_WARN("fail to delete global context", K(context_schema.get_tenant_id()), K(ret)); } else { LOG_INFO("success delete global context value", K(context_schema.get_tenant_id()), K(context_schema.get_context_id()), K(context_schema.get_namespace())); } return ret; } int ObDDLService::handle_label_se_policy_ddl(const ObLabelSePolicyDDLArg &arg) { int ret = OB_SUCCESS; ObLabelSePolicySchema schema = arg.schema_; //make a copy uint64_t tenant_id = schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid input schema", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.handle_label_se_policy_function( arg.ddl_type_, arg.ddl_stmt_str_, schema_guard, schema, trans))) { LOG_WARN("handle label security policy failed", K(ret), K(arg)); } if (OB_SUCC(ret) && arg.ddl_type_ == OB_DDL_DROP_LABEL_SE_POLICY) { //When drop policy, cascade delete policy-related components, labels and user levels, as well as security columns if (OB_FAIL(ddl_operator.drop_all_label_se_user_components( tenant_id, OB_INVALID_ID, schema.get_label_se_policy_id(), trans, ObString(), schema_guard))) { LOG_WARN("fail to drop user label components cascaded", K(ret), K(schema)); } else if (OB_FAIL(ddl_operator.drop_all_label_se_labels_in_policy( tenant_id, schema.get_label_se_policy_id(), trans, ObString(), schema_guard))) { LOG_WARN("fail to drop label se labels", K(ret), K(schema)); } else if (OB_FAIL(ddl_operator.drop_all_label_se_components_in_policy( tenant_id, schema.get_label_se_policy_id(), trans, ObString(), schema_guard))) { LOG_WARN("fail to drop label se components", K(ret), K(schema)); } else if (OB_FAIL(ddl_operator.drop_all_label_se_table_column( tenant_id, schema.get_label_se_policy_id(), trans, schema_guard))) { LOG_WARN("fail to drop all label se table column", K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::handle_label_se_component_ddl(const ObLabelSeComponentDDLArg &arg) { int ret = OB_SUCCESS; ObLabelSeComponentSchema schema = arg.schema_; //make a copy uint64_t tenant_id = schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid input schema", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.handle_label_se_component_function(arg.ddl_type_, arg.ddl_stmt_str_, arg.policy_name_, schema_guard, schema, trans))) { LOG_WARN("handle label security policy failed", K(ret), K(arg)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::handle_label_se_label_ddl(const ObLabelSeLabelDDLArg &arg) { int ret = OB_SUCCESS; ObLabelSeLabelSchema schema = arg.schema_; //make a copy uint64_t tenant_id = schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid input schema", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.handle_label_se_label_function(arg.ddl_type_, arg.ddl_stmt_str_, arg.policy_name_, schema_guard, schema, trans))) { LOG_WARN("handle label security policy failed", K(ret), K(arg)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::handle_label_se_user_level_ddl(const ObLabelSeUserLevelDDLArg &arg) { int ret = OB_SUCCESS; ObLabelSeUserLevelSchema schema = arg.level_schema_; //make a copy uint64_t tenant_id = schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid input schema", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.handle_label_se_user_level_function(arg.ddl_type_, arg.ddl_stmt_str_, arg.policy_name_, schema_guard, schema, trans))) { LOG_WARN("handle label security policy failed", K(ret), K(arg)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::check_user_exist(const share::schema::ObUserInfo &user_info) const { int ret = OB_SUCCESS; bool is_user_name_exist = false; bool is_user_id_exist = false; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(schema_service_->check_user_exist(user_info.get_tenant_id(), user_info.get_user_name_str(), user_info.get_host_name_str(), is_user_name_exist))) { LOG_WARN("Failed to check whether user exist", "tenant_id", user_info.get_tenant_id(), "user_name", user_info.get_user_name_str(), K(ret)); } else if (OB_INVALID_ID != user_info.get_user_id() && OB_FAIL(schema_service_->check_user_exist( user_info.get_tenant_id(), user_info.get_user_id(), is_user_id_exist))) { LOG_WARN("Failed to check whether user exist", "tenant_id", user_info.get_tenant_id(), "user_id", user_info.get_user_id(), K(ret)); } else if (is_user_name_exist || is_user_id_exist) { ret = user_info.is_role() ? OB_ROLE_EXIST : OB_ERR_USER_EXIST; LOG_WARN("User/role is exist, cannot create it twice,", "tenant_id", user_info.get_tenant_id(), "user_id", user_info.get_user_id(), "user/role_name", user_info.get_user_name_str(), "host_name", user_info.get_host_name_str(), K(ret)); } return ret; } bool ObDDLService::is_user_exist(const uint64_t tenant_id, const uint64_t user_id) const { bool is_user_id_exist = false; int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Tenant ID and user ID should not be invalid", K(ret)); } else if (OB_FAIL(schema_service_->check_user_exist(tenant_id, user_id, is_user_id_exist))) { LOG_WARN("Failed to check whether user exist", K(tenant_id), K(user_id), K(ret)); } return is_user_id_exist; } int ObDDLService::replay_alter_user(const share::schema::ObUserInfo &user_info, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); const uint64_t tenant_id = user_info.get_tenant_id(); int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { int64_t new_schema_version = OB_INVALID_VERSION; ObSchemaService *schema_service = schema_service_->get_schema_service(); if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id)); } else if (OB_ISNULL(schema_service)) { ret = OB_ERR_SYS; LOG_ERROR("schema_service must exist", K(ret)); } else if (OB_FAIL(schema_service->get_user_sql_service().alter_user( user_info, new_schema_version, NULL, trans))) { LOG_WARN("alter user failed", K(ret), K(user_info)); } } if (trans.is_started()) { int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("Failed to commit trans", K(ret), K(tmp_ret)); ret = OB_SUCC(ret) ? tmp_ret : ret; } } return ret; } int ObDDLService::create_user_in_trans(share::schema::ObUserInfo &user_info, uint64_t creator_id, uint64_t &user_id, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObDDLSQLTransaction trans(schema_service_); const uint64_t tenant_id = user_info.get_tenant_id(); if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObSqlString ddl_stmt_str; ObString ddl_sql; const bool is_role = user_info.is_role(); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("Failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ObDDLSqlGenerator::gen_create_user_sql( ObAccountArg(user_info.get_user_name_str(), user_info.get_host_name_str(), is_role), user_info.get_passwd_str(), ddl_stmt_str))) { LOG_WARN("gen create user sql failed", K(ret)); } else if (OB_FAIL(ObDDLSqlGenerator::append_ssl_info_sql(user_info.get_ssl_type(), user_info.get_ssl_cipher_str(), user_info.get_x509_issuer_str(), user_info.get_x509_subject_str(), ddl_stmt_str))) { LOG_WARN("gen append_ssl_info_sql failed", K(ret)); } else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) { } else if (OB_FAIL(ddl_operator.create_user(user_info, &ddl_sql, trans))) { LOG_WARN("Failed to create user", K(ret)); } else { LOG_INFO("Succeed in inserting user in user table", "tenant_id", user_info.get_tenant_id(), "user_id", user_info.get_user_id(), "user_name", user_info.get_user_name_str(), K(user_info)); user_id = user_info.get_user_id(); } } const ObTenantSchema *tenant_schema = NULL; const ObSysVariableSchema *sys_variable_schema = NULL; bool is_oracle_mode = false; if (OB_FAIL(ret)) { // do-nothing } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { LOG_WARN("tenant not exists", K(ret), K(tenant_id)); } else if (OB_UNLIKELY(NULL == tenant_schema)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("tenant not exist", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) { LOG_WARN("get sys variable schema failed", K(ret)); } else if (OB_ISNULL(sys_variable_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sys variable schema is null", K(ret)); } else if (OB_FAIL(sys_variable_schema->get_oracle_mode(is_oracle_mode))) { LOG_WARN("failed to get oracle mode", K(ret), K(tenant_id)); } else if (is_oracle_mode && !user_info.is_role()) { // For Oracle mode, Creating a user will also create a db with the same name // role does not need to create a db with the same name ObDatabaseSchema db_schema; db_schema.set_tenant_id(user_info.get_tenant_id()); if (OB_FAIL(db_schema.set_database_name(user_info.get_user_name()))) { LOG_WARN("failed to set database name", K(ret), K(user_info.get_user_name_str())); } else if (OB_FAIL(db_schema.set_comment("oracle user database"))) { LOG_WARN("failed to set database comment", K(ret), K(user_info.get_user_name_str())); } else if (OB_FAIL(create_database(false, db_schema, NULL, &trans))) { LOG_WARN("failed to create oracle user database", K(ret), K(tenant_id)); } } else if (is_oracle_mode && user_info.is_role() && OB_INVALID_ID != creator_id) { // For Oracle mode, role shall be granted to creator after creation with admin option ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); const ObUserInfo *creator_info = NULL; ObArray role_ids; if (OB_FAIL(ret)) { LOG_WARN("failed to grant role to creator", K(ret)); } else if (is_root_user(creator_id)) { LOG_WARN("creator_id is OB_SYS_USER_ID, refuse to grant role to it", K(user_id), K(creator_id)); } else if (OB_FAIL(schema_guard.get_user_info(tenant_id, creator_id, creator_info))) { LOG_WARN("get_user_info failed", K(ret), K(tenant_id), K(creator_id)); } else if (NULL == creator_info) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get creator_info", K(ret)); } else if (OB_FAIL(role_ids.push_back(user_id))) { LOG_WARN("failed to add to role_ids", K(ret), K(tenant_id), K(user_id), K(user_info.get_user_name_str())); } else if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id, *creator_info, role_ids, &user_info, trans, true /*log_operation*/, true /*is grant*/, ADMIN_OPTION))) { LOG_WARN("failed to grant_revoke_role", K(ret), K(tenant_id), K(*creator_info)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } return ret; } //----End of functions for managing privileges---- template int ObDDLService::get_schema_primary_regions( const SCHEMA &schema, share::schema::ObSchemaGetterGuard &schema_guard, common::ObIArray &primary_regions) { int ret = OB_SUCCESS; primary_regions.reset(); common::ObArray zone_list; ObArenaAllocator allocator("PrimaryZone"); ObPrimaryZone primary_zone_schema(allocator); if (OB_FAIL(schema.get_primary_zone_inherit(schema_guard, primary_zone_schema))) { LOG_WARN("fail to get primary zone inherit", K(ret)); } else if (ObString(OB_RANDOM_PRIMARY_ZONE) == primary_zone_schema.primary_zone_str_ || primary_zone_schema.primary_zone_str_.empty()) { common::ObArray zone_locality; if (OB_FAIL(schema.get_zone_replica_attr_array_inherit(schema_guard, zone_locality))) { LOG_WARN("fail to get zone replica attr set", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < zone_locality.count(); ++i) { const share::ObZoneReplicaAttrSet &this_locality = zone_locality.at(i); if (this_locality.get_full_replica_num() <= 0) { // bypass } else if (OB_FAIL(append(zone_list, this_locality.get_zone_set()))) { LOG_WARN("fail to append zone set", K(ret)); } } } } else { const ObIArray &primary_zone_array = primary_zone_schema.primary_zone_array_; if (primary_zone_array.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error, tenant primary zone array count less than 0", K(ret)); } else { const ObZoneScore &sample_zone_score = primary_zone_array.at(0); for (int64_t i = 0; OB_SUCC(ret) && i < primary_zone_array.count(); ++i) { const ObZoneScore &this_zone_score = primary_zone_array.at(i); if (this_zone_score.score_ != sample_zone_score.score_) { break; } else if (OB_FAIL(zone_list.push_back(this_zone_score.zone_))) { LOG_WARN("fail to push back", K(ret)); } } } } if (OB_SUCC(ret)) { if (zone_list.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("zone list count is zero", K(ret)); } else if (OB_FAIL(construct_region_list(primary_regions, zone_list))) { LOG_WARN("fail to construct region list", K(ret)); } } return ret; } int ObDDLService::check_tenant_primary_zone_( share::schema::ObSchemaGetterGuard &schema_guard, const share::schema::ObTenantSchema &new_tenant_schema) { int ret = OB_SUCCESS; common::ObArray tenant_primary_regions; if (OB_FAIL(get_schema_primary_regions( new_tenant_schema, schema_guard, tenant_primary_regions))) { LOG_WARN("fail to get tenant primary regions", K(ret)); } else if (tenant_primary_regions.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("primary regions unexpected", K(ret)); } else if (tenant_primary_regions.count() > 1) { ret = OB_NOT_SUPPORTED; LOG_WARN("tenant primary zone span regions not supported", K(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant primary zone span regions"); } return ret; } int ObDDLService::check_alter_tenant_replica_options( const obrpc::ObModifyTenantArg &arg, share::schema::ObTenantSchema &new_tenant_schema, const share::schema::ObTenantSchema &orig_tenant_schema, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObArray zone_list; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(new_tenant_schema.get_zone_list(zone_list))) { LOG_WARN("fail to get zone list", K(ret)); } else if (OB_FAIL(check_alter_schema_replica_options( arg.alter_option_bitset_.has_member(ObModifyTenantArg::PRIMARY_ZONE), new_tenant_schema, orig_tenant_schema, zone_list, schema_guard))) { LOG_WARN("fail to check replica options", K(ret)); } else {} // no more return ret; } int ObDDLService::check_create_tenant_replica_options( share::schema::ObTenantSchema &tenant_schema, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObArray zone_list; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) { LOG_WARN("fail to get zone list", K(ret)); } else if (OB_FAIL(check_create_schema_replica_options( tenant_schema, zone_list, schema_guard))) { LOG_WARN("fail to check replica options", K(ret)); } return ret; } int ObDDLService::check_schema_zone_list( common::ObArray &zone_list) { int ret = OB_SUCCESS; std::sort(zone_list.begin(), zone_list.end()); for (int64_t i = 0; OB_SUCC(ret) && i < zone_list.count() - 1; ++i) { if (zone_list.at(i) == zone_list.at(i+1)) { ret = OB_ZONE_DUPLICATED; LOG_USER_ERROR(OB_ZONE_DUPLICATED, to_cstring(zone_list.at(i)), to_cstring(zone_list)); LOG_WARN("duplicate zone in zone list", K(zone_list), K(ret)); } } if (OB_SUCC(ret)) { for (int64_t i = 0; OB_SUCC(ret) && i < zone_list.count(); ++i) { bool zone_exist = false; if (OB_FAIL(zone_mgr_->check_zone_exist(zone_list.at(i), zone_exist))) { LOG_WARN("check_zone_exist failed", "zone", zone_list.at(i), K(ret)); } else if (!zone_exist) { ret = OB_ZONE_INFO_NOT_EXIST; LOG_USER_ERROR(OB_ZONE_INFO_NOT_EXIST, to_cstring(zone_list.at(i))); LOG_WARN("zone not exist", "zone", zone_list.at(i), K(ret)); break; } } } return ret; } /* 1 First clarify two concepts: * 1.1 mix locality: mix locality is refers to the locality of multiple zones after the'@' mark * such as F,L@[z1,z2] in the locality. We call the locality of F,L@[z1,z2] a mixe locality. * 1.2 independent locality: independent locality refers to the locality where there is only one zone after the'@' * in the locality. We call the locality like F@z1 an "independent" locality. * * 2 After locality adds a mixed scene, the relationship between primary zone and locality * includes the following restrictions: * 2.1 The region where the primary zone is located has at least two fully functional copies * 2.2 Each zone in the mixed locality must belong to the same region * 2.3 The zone in the mixed locality cannot contain the primary zone with the highest priority. * for example locality='F,F{memstore_percent:0},L@[z1,z2,z3]',primary_zone='z1' is not allowed * If there is a need to set the primary zone on z1. The locality and primary zone can be set as follows: * locality = 'F@z1,F{memstore_percent:0},L@[z2,z3]', primary_zone = 'z1' * 2.4 Contrary to the logic of 2.3, if the locality contains both mixed locality and independent locality, * the highest priority primary zone must be set to one of the independent locality. * for example locality='F@z1, F{memstore_percent:0},L@[z2,z3]' It is not allowed not to set the primary zone * If there is no preference location setting for the primary zone, the locality can be set as follows: * locality = 'F,F{memstore_percent:0},L@[z1,z2,z3]' * 2.5 Currently, there are no application scenarios for multiple mixed localities, * and multiple mixed localities are not supported for the time being. */ template int ObDDLService::check_and_set_primary_zone( SCHEMA &schema, const common::ObIArray &zone_list, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; common::ObArray zone_region_list; if (schema.get_primary_zone().empty()) { LOG_INFO("primary zone is null, noting todo"); //nothing todo } else if (OB_FAIL(construct_zone_region_list(zone_region_list, zone_list))) { LOG_WARN("fail to construct zone region list", K(ret)); } else { char primary_zone_str[MAX_ZONE_LENGTH]; int64_t pos = 0; ObPrimaryZoneUtil primary_zone_util(schema.get_primary_zone(), &zone_region_list); if (OB_FAIL(primary_zone_util.init(zone_list))) { LOG_WARN("fail to init primary zone util", K(ret)); } else if (OB_FAIL(primary_zone_util.check_and_parse_primary_zone())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid primary zone", K(ret)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "primary zone"); } else if (OB_FAIL(primary_zone_util.output_normalized_primary_zone( primary_zone_str, MAX_ZONE_LENGTH, pos))) { LOG_WARN("fail to output normalized primary zone", K(ret)); } else if (OB_FAIL(schema.set_primary_zone(primary_zone_str))) { LOG_WARN("fail to set primary zone", K(ret)); } else if (OB_FAIL(schema.set_primary_zone_array(primary_zone_util.get_zone_array()))) { LOG_WARN("fail to set primary zone array", K(ret)); } else if (OB_FAIL(check_primary_zone_locality_condition( schema, zone_list, zone_region_list, schema_guard))) { LOG_WARN("fail to check primary zone region condition", K(ret)); } else {} // no more to do } return ret; } template int ObDDLService::check_empty_primary_zone_locality_condition( SCHEMA &schema, const common::ObIArray &zone_list, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObArenaAllocator allocator("PrimaryZone"); ObPrimaryZone primary_zone_schema(allocator); ObArray zone_locality; ObArray first_primary_zone_array; ObArray zone_region_list; if (OB_FAIL(construct_zone_region_list(zone_region_list, zone_list))) { LOG_WARN("fail to construct zone region list", K(ret)); } else if (OB_FAIL(schema.get_primary_zone_inherit(schema_guard, primary_zone_schema))) { LOG_WARN("fail to get primary zone inherit", K(ret)); } else if (OB_FAIL(extract_first_primary_zone_array( primary_zone_schema, zone_list, first_primary_zone_array))) { LOG_WARN("fail to extract first primary zone array", K(ret)); } else if (OB_FAIL(schema.get_zone_replica_attr_array_inherit( schema_guard, zone_locality))) { LOG_WARN("fail to get zone replica attr array", K(ret)); } else if (OB_FAIL(do_check_mixed_zone_locality_condition( zone_region_list, zone_locality))) { LOG_WARN("fail to do check mixed zone locality condition", K(ret)); } else if (OB_FAIL(do_check_mixed_locality_primary_zone_condition( first_primary_zone_array, zone_locality))) { LOG_WARN("fail to check mixed locality primary zone condition", K(ret)); } return ret; } template int ObDDLService::check_create_schema_replica_options( SCHEMA &schema, common::ObArray &zone_list, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (!ObPrimaryZoneUtil::is_specific_primary_zone(schema.get_primary_zone())) { const common::ObArray empty_pz_array; if (OB_FAIL(schema.set_primary_zone_array(empty_pz_array))) { LOG_WARN("fail to set primary zone array empty", K(ret)); } else if (OB_FAIL(check_empty_primary_zone_locality_condition( schema, zone_list, schema_guard))) { LOG_WARN("fail to check empty primary zone locality condition", K(ret)); } } else { if (OB_FAIL(check_schema_zone_list(zone_list))) { LOG_WARN("fail to check schema zone list", K(ret), K(zone_list)); } else if (OB_FAIL(check_and_set_primary_zone(schema, zone_list, schema_guard))) { LOG_WARN("fail to check and set primary zone", K(ret)); } } if (OB_SUCC(ret)) { int64_t paxos_num = 0; bool is_standby = false; if (OB_FAIL(schema.get_paxos_replica_num(schema_guard, paxos_num))) { LOG_WARN("fail to get paxos replica num", K(ret)); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("failed to get is standby cluster", K(ret)); } else if ((!is_standby && paxos_num <= 0) || paxos_num > common::OB_MAX_MEMBER_NUMBER) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid paxos replica num", K(ret), K(schema)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality paxos replica num"); } else {} // good } return ret; } template int ObDDLService::check_alter_schema_replica_options( const bool alter_primary_zone, SCHEMA &new_schema, const SCHEMA &orig_schema, common::ObArray &zone_list, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (!ObPrimaryZoneUtil::is_specific_primary_zone( new_schema.get_primary_zone())) { const common::ObArray empty_pz_array; if (OB_FAIL(new_schema.set_primary_zone_array(empty_pz_array))) { LOG_WARN("fail to set primary zone array empty", K(ret)); } else if (OB_FAIL(check_empty_primary_zone_locality_condition( new_schema, zone_list, schema_guard))) { LOG_WARN("fail to check empty primary zone locality condition", K(ret)); } } else { if (OB_FAIL(check_schema_zone_list(zone_list))) { LOG_WARN("fail to check schema zone list", K(ret), K(zone_list)); } else if (alter_primary_zone) { if (OB_FAIL(check_and_set_primary_zone(new_schema, zone_list, schema_guard))) { LOG_WARN("fail to check and set primary zone", K(ret)); } } else { // Currently alter tenant/database/table may cause zone_list to change // We need to remove the zones that are not in the zone_list and in the primary zone if (OB_FAIL(trim_and_set_primary_zone(new_schema, orig_schema, zone_list, schema_guard))) { LOG_WARN("fail to trim and set primary zone", K(ret)); } } } if (OB_SUCC(ret)) { int64_t paxos_num = 0; bool is_standby = false; if (OB_FAIL(new_schema.get_paxos_replica_num(schema_guard, paxos_num))) { LOG_WARN("fail to get paxos replica num", K(ret)); } else if (OB_FAIL(get_is_standby_cluster(is_standby))) { LOG_WARN("failed to get is standby cluster", K(ret)); } else if ((!is_standby && paxos_num <= 0) || paxos_num > common::OB_MAX_MEMBER_NUMBER) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid paxos replica num", K(ret)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality paxos replica num"); } else {} // good } return ret; } template int ObDDLService::trim_and_set_primary_zone( SCHEMA &new_schema, const SCHEMA &orig_schema, const common::ObIArray &zone_list, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; common::ObArray new_zone_score_array; const ObIArray &orig_zone_score_array = orig_schema.get_primary_zone_array(); common::ObZone zone; for (int64_t i = 0; i < orig_zone_score_array.count() && OB_SUCC(ret); ++i) { zone.reset(); if (OB_FAIL(zone.assign(orig_zone_score_array.at(i).zone_.ptr()))) { LOG_WARN("fail to assign zone", K(ret)); } else if (!has_exist_in_array(zone_list, zone)) { // No longer in zone_list, remove this zone } else if (OB_FAIL(new_zone_score_array.push_back(orig_zone_score_array.at(i)))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } if (new_zone_score_array.count() <= 0) { const common::ObArray empty_pz_array; if (OB_FAIL(new_schema.set_primary_zone(ObString::make_string("")))) { LOG_WARN("fail to set primary zone", K(ret)); } else if (OB_FAIL(new_schema.set_primary_zone_array(empty_pz_array))) { LOG_WARN("fail to set primary zone array empty", K(ret)); } else if (OB_FAIL(check_empty_primary_zone_locality_condition( new_schema, zone_list, schema_guard))) { LOG_WARN("fail to check empty primary zone locality condition", K(ret)); } else {} // no more to do } else { std::sort(new_zone_score_array.begin(), new_zone_score_array.end()); char primary_zone_str[MAX_ZONE_LENGTH]; if (OB_FAIL(format_primary_zone_from_zone_score_array( new_zone_score_array, primary_zone_str, MAX_ZONE_LENGTH))) { LOG_WARN("fail to construct primary zone from zone score array", K(ret)); } else if (OB_FAIL(new_schema.set_primary_zone(ObString::make_string(primary_zone_str)))) { LOG_WARN("fail to set primary zone", K(ret)); } else if (OB_FAIL(check_and_set_primary_zone(new_schema, zone_list, schema_guard))) { LOG_WARN("fail to check and set primary zone", K(ret)); } else {} // no more to do } return ret; } int ObDDLService::format_primary_zone_from_zone_score_array( common::ObIArray &zone_score_array, char *buf, int64_t buf_len) { int ret = OB_SUCCESS; MEMSET(buf, 0, buf_len); if (zone_score_array.count() <= 0) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), "array num", zone_score_array.count()); } else { int64_t pos = 0; bool start_format = false; int64_t prev_zone_score = zone_score_array.at(0).score_; const char *separator_token = NULL; for (int64_t i = 0; i < zone_score_array.count() && OB_SUCC(ret); ++i) { ObZoneScore &cur_zone_score = zone_score_array.at(i); const bool same_p = (cur_zone_score.score_ == prev_zone_score); separator_token = (same_p ? "," : ";"); if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", (!start_format ? "" : separator_token)))) { LOG_WARN("fail to format separator", K(ret)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%.*s", static_cast(cur_zone_score.zone_.length()), cur_zone_score.zone_.ptr()))) { LOG_WARN("fail to format zone", K(ret)); } else { start_format = true; prev_zone_score = cur_zone_score.score_; } } // for } return ret; } int ObDDLService::construct_region_list( common::ObIArray ®ion_list, const common::ObIArray &zone_list) { int ret = OB_SUCCESS; region_list.reset(); if (OB_UNLIKELY(NULL == zone_mgr_)) { ret = OB_NOT_INIT; LOG_WARN("zone mgr is null", K(ret)); } else { common::ObArray zone_infos; if (OB_FAIL(zone_mgr_->get_zone(zone_infos))) { LOG_WARN("fail to get zone", K(ret)); } else { for (int64_t i = 0; i < zone_infos.count() && OB_SUCC(ret); ++i) { ObRegion region; share::ObZoneInfo &zone_info = zone_infos.at(i); if (OB_FAIL(region.assign(zone_info.region_.info_.ptr()))) { LOG_WARN("fail to assign region", K(ret)); } else if (!has_exist_in_array(zone_list, zone_info.zone_)) { // this zone do not exist in my zone list, ignore it } else if (has_exist_in_array(region_list, region)) { // this region already exist in array } else if (OB_FAIL(region_list.push_back(region))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } } } return ret; } int ObDDLService::get_zone_region( const common::ObZone &zone, const common::ObIArray &zone_region_list, common::ObRegion ®ion) { int ret = OB_SUCCESS; bool find = false; for (int64_t i = 0; !find && i < zone_region_list.count(); ++i) { const share::schema::ObZoneRegion &zone_region = zone_region_list.at(i); if (zone_region.zone_ == zone) { region = zone_region.region_; find = true; } else {} // go on check next; } if (!find) { ret = OB_ENTRY_NOT_EXIST; LOG_WARN("entry not exist", K(ret)); } return ret; } int ObDDLService::construct_zone_region_list( common::ObIArray &zone_region_list, const common::ObIArray &zone_list) { int ret = OB_SUCCESS; zone_region_list.reset(); if (OB_UNLIKELY(NULL == zone_mgr_)) { ret = OB_NOT_INIT; LOG_WARN("zone mgr is null", K(ret)); } else { common::ObArray zone_infos; if (OB_FAIL(zone_mgr_->get_zone(zone_infos))) { LOG_WARN("fail to get zone", K(ret)); } else { ObZoneRegion zone_region; for (int64_t i = 0; i < zone_infos.count() && OB_SUCC(ret); ++i) { zone_region.reset(); share::ObZoneInfo &zone_info = zone_infos.at(i); if (OB_FAIL(zone_region.zone_.assign(zone_info.zone_.ptr()))) { LOG_WARN("fail to assign zone", K(ret)); } else if (OB_FAIL(zone_region.region_.assign(zone_info.region_.info_.ptr()))) { LOG_WARN("fail to assign region", K(ret)); } else if (OB_FAIL(zone_region.set_check_zone_type(zone_info.zone_type_.value_))) { LOG_WARN("fail to set check zone type", KR(ret)); } else if (!has_exist_in_array(zone_list, zone_region.zone_)) { // this zone do not exist in my zone list, ignore it } else if (OB_FAIL(zone_region_list.push_back(zone_region))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } } } return ret; } int ObDDLService::get_is_standby_cluster(bool &is_standby) const { int ret = OB_SUCCESS; is_standby = false; return ret; } template int ObDDLService::set_schema_replica_num_options( SCHEMA &schema, ObLocalityDistribution &locality_dist, ObIArray &unit_infos) { int ret = OB_SUCCESS; common::ObArray zone_replica_attr_array; if (OB_FAIL(locality_dist.get_zone_replica_attr_array( zone_replica_attr_array))) { LOG_WARN("fail to get zone region replica num array", K(ret)); } else if (OB_FAIL(schema.set_zone_replica_attr_array(zone_replica_attr_array))) { LOG_WARN("fail to set zone replica num set", K(ret)); } else { int64_t full_replica_num = 0; for (int64_t i = 0; i < zone_replica_attr_array.count(); ++i) { ObZoneReplicaNumSet &zone_replica_num_set = zone_replica_attr_array.at(i); full_replica_num += zone_replica_num_set.get_full_replica_num(); } if (full_replica_num <= 0) { ret = OB_INVALID_ARGUMENT; LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality"); LOG_WARN("full replica num is zero", K(ret), K(full_replica_num), K(schema)); } } if (OB_SUCC(ret)) { for (int64_t i = 0; i < zone_replica_attr_array.count() && OB_SUCC(ret); ++i) { ObZoneReplicaAttrSet &zone_replica_set = zone_replica_attr_array.at(i); if (zone_replica_set.zone_set_.count() > 1) { if (zone_replica_set.zone_set_.count() != zone_replica_set.get_paxos_replica_num()) { ret = OB_INVALID_ARGUMENT; LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality, too many paxos replicas in multiple zones"); LOG_WARN("too many paxos replicas in multi zone", K(ret)); } } else if (zone_replica_set.get_full_replica_num() > 1 || zone_replica_set.get_logonly_replica_num() > 1 || zone_replica_set.get_encryption_logonly_replica_num() > 1) { ret = OB_INVALID_ARGUMENT; LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality"); LOG_WARN("one zone should only have one paxos replica", K(ret), K(zone_replica_set)); } else if (zone_replica_set.get_full_replica_num() == 1 && (zone_replica_set.get_logonly_replica_num() == 1 || zone_replica_set.get_encryption_logonly_replica_num() == 1)) { bool find = false; for (int64_t j = 0; j < unit_infos.count() && OB_SUCC(ret); j++) { if (unit_infos.at(j).unit_.zone_ == zone_replica_set.zone_ && REPLICA_TYPE_LOGONLY == unit_infos.at(j).unit_.replica_type_) { find = true; break; } } //end for unit_infos if (!find) { ret = OB_INVALID_ARGUMENT; LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality"); LOG_WARN("no logonly unit exist", K(ret), K(zone_replica_set)); } } } } return ret; } int ObDDLService::get_pools(const ObIArray &pool_strs, ObIArray &pools) { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < pool_strs.count(); ++i) { ObResourcePoolName pool; if (OB_FAIL(pool.assign(pool_strs.at(i)))) { LOG_WARN("assign failed", K(ret)); } else if (OB_FAIL(pools.push_back(pool))) { LOG_WARN("push_back failed", K(ret)); } } return ret; } template int ObDDLService::extract_first_primary_zone_array( const SCHEMA &schema, const ObIArray &zone_list, ObIArray &first_primary_zone_array) { int ret = OB_SUCCESS; if (schema.get_primary_zone_array().count() <= 0) { // bypass } else { const ObIArray &primary_zone_score_array = schema.get_primary_zone_array(); const ObZoneScore &sample_zone = primary_zone_score_array.at(0); for (int64_t i = 0; OB_SUCC(ret) && i < primary_zone_score_array.count(); ++i) { common::ObZone this_zone; if (sample_zone.score_ != primary_zone_score_array.at(i).score_) { break; } else if (OB_FAIL(this_zone.assign(primary_zone_score_array.at(i).zone_.ptr()))) { LOG_WARN("fail to assign zone", K(ret), "zone", primary_zone_score_array.at(i).zone_); } else if (!has_exist_in_array(zone_list, this_zone)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("primary zone not in zone list", K(ret), K(zone_list), K(this_zone)); LOG_USER_ERROR(OB_INVALID_ARGUMENT, "primary zone, primary zone not in zone list"); } else if (OB_FAIL(first_primary_zone_array.push_back(this_zone))) { LOG_WARN("fail to push back", K(ret)); } else {} // no more to do } } return ret; } int ObDDLService::get_primary_regions_and_zones( const ObIArray &zone_list, const ObIArray &zone_region_list, const ObIArray &first_primary_zone_array, ObIArray &primary_regions, ObIArray &zones_in_primary_regions) { int ret = OB_SUCCESS; primary_regions.reset(); zones_in_primary_regions.reset(); for (int64_t i = 0; OB_SUCC(ret) && i < first_primary_zone_array.count(); ++i) { const ObZone &this_zone = first_primary_zone_array.at(i); if (this_zone.is_empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("this zone is invalid", K(ret), K(this_zone)); } else if (!has_exist_in_array(zone_list, this_zone)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("this zone is not exist in zone list", K(ret), K(this_zone), K(zone_list)); } else { ObRegion this_region; bool find = false; for (int64_t j = 0; !find && j < zone_region_list.count(); ++j) { if (this_zone == zone_region_list.at(j).zone_) { this_region = zone_region_list.at(j).region_; find = true; } else {} // go on to check next } if (!find) { ret = OB_ERR_UNEXPECTED; LOG_WARN("region not found", K(ret), K(this_zone)); } else if (this_region.is_empty()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("region invalid", K(ret), K(this_region)); } else if (has_exist_in_array(primary_regions, this_region)) { // Already exist in primary regions, ignore } else if (OB_FAIL(primary_regions.push_back(this_region))) { LOG_WARN("fail to push back", K(ret), K(this_region)); } else {} // no more to do // Push all the zones belonging to this region to zones_in_primary_regions for (int64_t i = 0; OB_SUCC(ret) && i < zone_region_list.count(); ++i) { const share::schema::ObZoneRegion &zone_region = zone_region_list.at(i); if (zone_region.region_ != this_region) { // ignore } else if (has_exist_in_array(zones_in_primary_regions, zone_region.zone_)) { // ignore } else if (OB_FAIL(zones_in_primary_regions.push_back(zone_region.zone_))) { LOG_WARN("fail to push back", K(ret), "region", zone_region.zone_); } else {} // no more to do } } } return ret; } int ObDDLService::drop_index_to_recyclebin(const ObTableSchema &table_schema) { int ret = OB_SUCCESS; const ObTableSchema *data_table_schema = NULL; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else { ObTableSchema new_index_table_schema; ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; int64_t refreshed_schema_version = 0; const uint64_t tenant_id = table_schema.get_tenant_id(); if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (schema_guard.get_table_schema(tenant_id, table_schema.get_data_table_id(), data_table_schema)) { LOG_WARN("get table schema failed", K(ret), K(tenant_id)); } else if (OB_ISNULL(data_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("data table schema is null", K(ret)); } else if (OB_FAIL(ddl_operator.alter_index_drop_options(table_schema, data_table_schema->get_table_name_str(), new_index_table_schema, trans))) { LOG_WARN("alter index invisible failed", K(table_schema), K(ret)); } else if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(new_index_table_schema, schema_guard, trans, NULL))) { LOG_WARN("drop index to recyclebin failed", K(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } //refresh table schema if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("refresh_schema failed", K(ret)); } } } return ret; } /* 1 First clarify two concepts: * 1.1 mix locality: mix locality is refers to the locality of multiple zones after the'@' mark * such as F,L@[z1,z2] in the locality. We call the locality of F,L@[z1,z2] a mixe locality. * 1.2 independent locality: independent locality refers to the locality where there is only one zone after the'@' * in the locality. We call the locality like F@z1 an "independent" locality. * * 2 After locality adds a mixed scene, the relationship between primary zone and locality * includes the following restrictions: * 2.1 The region where the primary zone is located has at least two fully functional copies * 2.2 Each zone in the mixed locality must belong to the same region * 2.3 The zone in the mixed locality cannot contain the primary zone with the highest priority. * for example locality='F,F{memstore_percent:0},L@[z1,z2,z3]',primary_zone='z1' is not allowed * If there is a need to set the primary zone on z1. The locality and primary zone can be set as follows: * locality = 'F@z1,F{memstore_percent:0},L@[z2,z3]', primary_zone = 'z1' * 2.4 Contrary to the logic of 2.3, if the locality contains both mixed locality and independent locality, * the highest priority primary zone must be set to one of the independent locality. * for example locality='F@z1, F{memstore_percent:0},L@[z2,z3]' It is not allowed not to set the primary zone * If there is no preference location setting for the primary zone, the locality can be set as follows: * locality = 'F,F{memstore_percent:0},L@[z1,z2,z3]' * 2.5 Currently, there are no application scenarios for multiple mixed localities, * and multiple mixed localities are not supported for the time being. */ template int ObDDLService::check_primary_zone_locality_condition( const SCHEMA &schema, const ObIArray &zone_list, const ObIArray &zone_region_list, share::schema::ObSchemaGetterGuard &schema_guard) { int ret = OB_SUCCESS; ObArray first_primary_zone_array; ObArray primary_regions; ObArray zones_in_primary_regions; ObArray zone_locality; ObString locality_str; if (zone_list.count() != zone_region_list.count()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), "zone count", zone_list.count(), "zone region count", zone_region_list.count()); } else if (zone_list.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid argument", K(ret), "zone count", zone_list.count()); } else if (OB_FAIL(extract_first_primary_zone_array( schema, zone_list, first_primary_zone_array))) { LOG_WARN("fail to extract first primary zone array", K(ret)); } else if (OB_FAIL(get_primary_regions_and_zones( zone_list, zone_region_list, first_primary_zone_array, primary_regions, zones_in_primary_regions))) { LOG_WARN("fail to get primary regions and zones", K(ret)); } else if (OB_FAIL(schema.get_zone_replica_attr_array_inherit( schema_guard, zone_locality))) { LOG_WARN("fail to get zone replica attr array", K(ret)); } else if (OB_FAIL(do_check_primary_zone_locality_condition( zone_region_list, first_primary_zone_array, zones_in_primary_regions, primary_regions, zone_locality))) { LOG_WARN("fail to do check primary zone region condition", K(ret)); } else {} // no more to do return ret; } int ObDDLService::do_check_primary_zone_locality_condition( const ObIArray &zone_region_list, const ObIArray &first_primary_zone_array, const ObIArray &zones_in_primary_regions, const ObIArray &primary_regions, const ObIArray &zone_locality) { int ret = OB_SUCCESS; if (OB_SUCC(ret) && zone_region_list.count() > 1 && first_primary_zone_array.count() > 0) { if (OB_FAIL(do_check_primary_zone_region_condition( zones_in_primary_regions, primary_regions, zone_locality))) { LOG_WARN("fail to check primary zone region condition", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(do_check_mixed_zone_locality_condition( zone_region_list, zone_locality))) { LOG_WARN("fail to check mixed zone locality condition", K(ret)); } } if (OB_SUCC(ret)) { if (OB_FAIL(do_check_mixed_locality_primary_zone_condition( first_primary_zone_array, zone_locality))) { LOG_WARN("fail to check mixed locality primary zone condition", K(ret)); } } return ret; } int ObDDLService::do_check_primary_zone_region_condition( const ObIArray &zones_in_primary_regions, const ObIArray &primary_regions, const ObIArray &zone_locality) { int ret = OB_SUCCESS; int64_t full_replica_num = 0; UNUSED(primary_regions); for (int64_t i = 0; i < zones_in_primary_regions.count(); ++i) { const ObZone &this_zone = zones_in_primary_regions.at(i); for (int64_t j = 0; j < zone_locality.count(); ++j) { const ObZoneReplicaAttrSet &zone_replica_num = zone_locality.at(j); if (this_zone == zone_replica_num.zone_) { full_replica_num += zone_replica_num.get_full_replica_num(); break; } else {} // go on and check next } } if (full_replica_num <= 1) { ret = OB_OP_NOT_ALLOW; LOG_WARN("primary zone F type replica is not enough in its region is not allowed", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "primary zone F type replica not enough in its region"); } return ret; } int ObDDLService::do_check_mixed_zone_locality_condition( const ObIArray &zone_region_list, const ObIArray &zone_locality) { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < zone_locality.count(); ++i) { const ObZoneReplicaAttrSet &zone_attr = zone_locality.at(i); const common::ObIArray &zone_set = zone_attr.get_zone_set(); if (zone_set.count() <= 1) { // bypass } else { common::ObRegion sample_region; for (int64_t j = 0; OB_SUCC(ret) && j < zone_set.count(); ++j) { const common::ObZone &this_zone = zone_set.at(j); common::ObRegion this_region; if (OB_FAIL(get_zone_region(this_zone, zone_region_list, this_region))) { LOG_WARN("fail to get zone region", K(ret), K(this_zone), K(zone_region_list)); } else if (sample_region.is_empty()) { sample_region = this_region; } else if (sample_region != this_region) { ret = OB_OP_NOT_ALLOW; LOG_WARN("mixed zone locality in more than one region is not allowed", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "mixed zone locality in more than one region"); } else {} // next zone } } } return ret; } int ObDDLService::do_check_mixed_locality_primary_zone_condition( const ObIArray &first_primary_zone_array, const ObIArray &zone_locality) { int ret = OB_SUCCESS; // first primary zone cannot be assigned in mixed locality zones ObArray independent_zones; ObArray independent_zone_idxs; int64_t mixed_locality_count = 0; for (int64_t i = 0; OB_SUCC(ret) && i < zone_locality.count(); ++i) { const share::ObZoneReplicaAttrSet &zone_attr = zone_locality.at(i); const ObIArray &zone_set = zone_attr.get_zone_set(); if (zone_set.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("zone set count unexpected", K(ret)); } else if (zone_set.count() == 1) { if (OB_FAIL(independent_zones.push_back(zone_set.at(0)))) { LOG_WARN("fail to push back independent zones", K(ret)); } else if (OB_FAIL(independent_zone_idxs.push_back(i))) { LOG_WARN("fail to push back", K(ret)); } } else { ++mixed_locality_count; for (int64_t j = 0; OB_SUCC(ret) && j < first_primary_zone_array.count(); ++j) { const common::ObZone &first_primary_zone = first_primary_zone_array.at(j); if (!has_exist_in_array(zone_set, first_primary_zone)) { // good, go on check } else { ret = OB_OP_NOT_ALLOW; LOG_WARN("primary zone assigned with mix zone locality is not allowed", K(ret), K(first_primary_zone), K(zone_set)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "primary zone assigned with mix zone locality"); } } } } if (OB_FAIL(ret)) { // bypass } else if (0 == mixed_locality_count) { // bypass } else if (1 == mixed_locality_count) { if (independent_zones.count() <= 0) { // bypass } else if (independent_zones.count() != independent_zone_idxs.count()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("array count not match", K(ret), "zone_count", independent_zones.count(), "idx_count", independent_zone_idxs.count()); } else { bool all_non_full_independent_zone = true; bool find_full_pz = false; for (int64_t i = 0; !find_full_pz && i < independent_zones.count(); ++i) { const common::ObZone &this_zone = independent_zones.at(i); const int64_t locality_idx = independent_zone_idxs.at(i); const ObZoneReplicaAttrSet &zone_replica_set = zone_locality.at(locality_idx); if (!has_exist_in_array(first_primary_zone_array, this_zone)) { if (zone_replica_set.get_full_replica_num() > 0) { all_non_full_independent_zone = false; } } else { if (zone_replica_set.get_full_replica_num() > 0) { find_full_pz = true; } } } if (find_full_pz) { // good, find full primary zone } else if (all_non_full_independent_zone) { // no full replica primary zone, but all others independent zones are not full, still good } else { ret = OB_OP_NOT_ALLOW; LOG_WARN("independent locality with mixed locality without full primary zone is not allowed", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "independent locality with mixed locality without full primary zone"); } } } else { ret = OB_OP_NOT_ALLOW; LOG_WARN("more than one mixed zone locality is not allowed", K(ret)); LOG_USER_ERROR(OB_OP_NOT_ALLOW, "more than one mixed zone locality"); } return ret; } int ObDDLService::try_check_and_set_table_schema_in_tablegroup( share::schema::ObSchemaGetterGuard &schema_guard, share::schema::ObTableSchema &schema) { int ret = OB_SUCCESS; const uint64_t tablegroup_id = schema.get_tablegroup_id(); if (!schema.has_partition()) { // include standalone table and binding table // do nohthing } else if (OB_INVALID_ID != tablegroup_id && !is_sys_tablegroup_id(tablegroup_id)) { ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_); if (OB_FAIL(helper.check_partition_option_for_create_table(schema_guard, schema))) { LOG_WARN("fail to check tablegroup partition", K(ret), K(schema)); } } else { } return ret; } /* * this fuction is used to broadcast new schema version in tenant space. * [intput] : * - arg.schema_operation_ : info that should be recorded in __all_ddl_operation. * arg.schema_operation_.tenant_id_ is used to specific the namespace. */ int ObDDLService::log_nop_operation(const obrpc::ObDDLNopOpreatorArg &arg) { int ret = OB_SUCCESS; if (OB_FAIL(check_inner_stat())) { LOG_WARN("inner stat error", KR(ret)); } else { const uint64_t tenant_id = arg.schema_operation_.tenant_id_; int64_t new_schema_version = OB_INVALID_VERSION; share::schema::ObSchemaService *tmp_schema_service = schema_service_->get_schema_service(); ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; if (OB_ISNULL(tmp_schema_service)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", KR(ret), KP(tmp_schema_service), KP(schema_service_)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get tenant schema guard", KR(ret), K(tenant_id)); } else { share::schema::ObDDLSqlService ddl_sql_service(*tmp_schema_service); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans", KR(ret), K(tenant_id), K(arg), K(refreshed_schema_version)); } else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) { LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id)); } else if (OB_FAIL(ddl_sql_service.log_nop_operation(arg.schema_operation_, new_schema_version, arg.ddl_stmt_str_, trans))) { LOG_WARN("log non ddl operation failed", KR(ret), K(arg)); } else { LOG_INFO("log nop opereation success", K(arg), K(new_schema_version)); } if (trans.is_started()) { int tmp_ret = OB_SUCCESS; if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("failed to end trans", KR(ret), K(tmp_ret)); ret = (OB_SUCCESS == ret ? tmp_ret : ret); } } if (OB_SUCC(ret)) { // publish_schema() will modify THIS_WORKER's timeout. // we should recovery the timeout ctx after publish schema. const int64_t timeout_us = THIS_WORKER.get_timeout_ts(); if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", KR(ret)); } THIS_WORKER.set_timeout_ts(timeout_us); } } } return ret; } int ObDDLService::create_user_defined_function(share::schema::ObUDF &udf_info, const common::ObString &ddl_stmt_str) { int ret = OB_SUCCESS; const uint64_t tenant_id = udf_info.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { ret = ddl_operator.create_user_defined_function(udf_info, trans, &ddl_stmt_str); if (OB_FAIL(ret)) { LOG_WARN("failed to create udf", K(udf_info), K(ddl_stmt_str), K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } LOG_INFO("finish create UDF", K(udf_info), K(ret)); return ret; } int ObDDLService::drop_user_defined_function(const obrpc::ObDropUserDefinedFunctionArg &drop_func_arg) { int ret = OB_SUCCESS; const uint64_t tenant_id = drop_func_arg.tenant_id_; const ObString &name = drop_func_arg.name_; const bool if_exist = drop_func_arg.if_exist_; ObDDLSQLTransaction trans(schema_service_); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_UNLIKELY(false == drop_func_arg.is_valid()) || OB_ISNULL(schema_service_)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(drop_func_arg), K(ret)); } //check udf exist & drop udf if (OB_SUCC(ret)) { bool is_exist = false; int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_service_->check_udf_exist(tenant_id, name, is_exist))) { LOG_WARN("check_udf_exist failed", K(tenant_id), K(name), K(ret)); } else if (!is_exist) { if (if_exist) { LOG_USER_NOTE(OB_ERR_FUNCTION_UNKNOWN, name.length(), name.ptr()); LOG_INFO("function not exist, no need to delete it", K(tenant_id), K(name)); } else { ret = OB_ERR_FUNCTION_UNKNOWN; LOG_WARN("function not exist, can't delete it", K(tenant_id), K(name), K(ret)); } } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id)); } else { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(ddl_operator.drop_user_defined_function(tenant_id, name, trans, &drop_func_arg.ddl_stmt_str_))) { LOG_WARN("ddl_operator drop_user_defined_function failed", K(tenant_id), K(ret)); } else {/*do nothing*/} } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish drop UDF", K(tenant_id), K(name), K(ret)); return ret; } int ObDDLService::check_udf_exist(uint64 tenant_id, const common::ObString &name, bool &is_exist) { int ret = OB_SUCCESS; if (OB_FAIL(schema_service_->check_udf_exist(tenant_id, name, is_exist))) { LOG_WARN("failed to check if udf_name exists", K(name), K(ret)); } return ret; } int ObDDLService::reconstruct_table_schema_from_recyclebin(ObTableSchema &index_table_schema, const ObRecycleObject &recycle_obj, ObSchemaGetterGuard &guard) { int ret = OB_SUCCESS; uint64_t tenant_id = index_table_schema.get_tenant_id(); index_table_schema.set_tablegroup_id(recycle_obj.get_tablegroup_id()); const ObDatabaseSchema *db_schema = NULL; if (OB_INVALID_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invaild tenant id", K(tenant_id), K(ret)); } else if (OB_FAIL(guard.get_database_schema(tenant_id, recycle_obj.get_database_id(), db_schema))) { LOG_WARN("get database schema failed", K(ret)); } else if (NULL == db_schema) { ret = OB_ERR_UNEXPECTED; LOG_WARN("database not exist", K(recycle_obj), K(ret)); } else if (db_schema->is_in_recyclebin()) { ret = OB_OP_NOT_ALLOW; LOG_WARN("database in recyclebin is not allowed", K(recycle_obj), K(*db_schema), K(ret)); } else if (OB_FAIL(index_table_schema.set_table_name(recycle_obj.get_original_name()))) { LOG_WARN("set table name failed", K(ret), K(recycle_obj)); } else { index_table_schema.set_database_id(recycle_obj.get_database_id()); } if (OB_SUCC(ret)) { if (OB_INVALID_ID != recycle_obj.get_tablegroup_id()) { bool is_tablegroup_exist = false; if (OB_FAIL(guard.check_tablegroup_exist(recycle_obj.get_tenant_id(), recycle_obj.get_tablegroup_id(), is_tablegroup_exist))) { LOG_WARN("check tablegroup exist failed", K(recycle_obj), K(ret)); } else if (!is_tablegroup_exist) { //replace tablegroup id with invalid id index_table_schema.set_tablegroup_id(OB_INVALID_ID); LOG_WARN("tablegroup is not exist", K(recycle_obj), K(ret)); } else { index_table_schema.set_tablegroup_id(recycle_obj.get_tablegroup_id()); } } else { index_table_schema.set_tablegroup_id(OB_INVALID_ID); } } return ret; } int ObDDLService::get_database_id(ObSchemaGetterGuard &schema_guard, uint64_t tenant_id, const ObString &database_name, uint64_t &database_id) { int ret = OB_SUCCESS; const ObDatabaseSchema *db_schema = NULL; if (OB_FAIL(schema_guard.get_database_schema(tenant_id, database_name, db_schema))) { LOG_WARN("get database schema failed", K(ret)); } else if (NULL == db_schema) { ret = OB_ERR_BAD_DATABASE; LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr()); } else if (db_schema->is_or_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("Can't not operate db in recyclebin", K(tenant_id), K(database_name), K(database_id), K(*db_schema), K(ret)); } else if (OB_INVALID_ID == (database_id = db_schema->get_database_id())) { ret = OB_ERR_BAD_DATABASE; LOG_WARN("database id is invalid", K(tenant_id), K(database_name), K(database_id), K(*db_schema), K(ret)); } return ret; } int ObDDLService::get_object_info(ObSchemaGetterGuard &schema_guard, const uint64_t tenant_id, const ObString &object_database, const ObString &object_name, ObSchemaType &object_type, uint64_t &object_id) { int ret = OB_SUCCESS; uint64_t database_id = OB_INVALID_ID; const ObTableSchema *table_schema = NULL; if (OB_FAIL(get_database_id(schema_guard, tenant_id, object_database, database_id))) { LOG_WARN("failed to get database id", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, database_id, object_name, false, table_schema))) { LOG_WARN("failed to get table schema", K(ret), K(tenant_id)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_BAD_TABLE; LOG_WARN("table schema is invalid", K(ret), K(object_name), K(object_name)); } else if (table_schema->is_in_recyclebin()) { ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT; LOG_WARN("table is in recyclebin", K(ret), K(object_name), K(object_name)); } else if (!table_schema->is_user_table() && !table_schema->is_user_view()) { ret = OB_NOT_SUPPORTED; LOG_WARN("trigger only support create on user table or user view now", K(ret)); } else { object_type = table_schema->is_user_table() ? TABLE_SCHEMA : VIEW_SCHEMA; object_id = table_schema->get_table_id(); } return ret; } int ObDDLService::update_mysql_tenant_sys_var( const ObTenantSchema &tenant_schema, const ObSysVariableSchema &sys_variable_schema, ObSysParam *sys_params, int64_t params_capacity) { int ret = OB_SUCCESS; const uint64_t tenant_id = sys_variable_schema.get_tenant_id(); if (OB_ISNULL(sys_params) || OB_UNLIKELY(params_capacity < ObSysVarFactory::ALL_SYS_VARS_COUNT)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity)); } else if (tenant_schema.is_mysql_tenant()) { HEAP_VAR(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf) { // If it is a tenant in mysql mode, you need to consider setting the charset and collation // corresponding to the tenant to sys var VAR_INT_TO_STRING(val_buf, tenant_schema.get_collation_type()); // set collation and char set SET_TENANT_VARIABLE(SYS_VAR_COLLATION_DATABASE, val_buf); SET_TENANT_VARIABLE(SYS_VAR_COLLATION_SERVER, val_buf); SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_DATABASE, val_buf); SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_SERVER, val_buf); } // end HEAP_VAR } return ret; } int ObDDLService::update_oracle_tenant_sys_var( const ObTenantSchema &tenant_schema, const ObSysVariableSchema &sys_variable_schema, ObSysParam *sys_params, int64_t params_capacity) { int ret = OB_SUCCESS; const uint64_t tenant_id = sys_variable_schema.get_tenant_id(); if (OB_ISNULL(sys_params) || OB_UNLIKELY(params_capacity < ObSysVarFactory::ALL_SYS_VARS_COUNT)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity)); } else if (tenant_schema.is_oracle_tenant()) { HEAP_VAR(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf) { // For oracle tenants, the collation of sys variable and tenant_option is set to binary by default. // set group_concat_max_len = 4000 // set autocommit = off // When setting oracle variables, try to keep the format consistent VAR_INT_TO_STRING(val_buf, OB_DEFAULT_GROUP_CONCAT_MAX_LEN_FOR_ORACLE); SET_TENANT_VARIABLE(SYS_VAR_GROUP_CONCAT_MAX_LEN, val_buf); SET_TENANT_VARIABLE(SYS_VAR_AUTOCOMMIT, "0"); VAR_INT_TO_STRING(val_buf, tenant_schema.get_collation_type()); SET_TENANT_VARIABLE(SYS_VAR_COLLATION_DATABASE, val_buf); SET_TENANT_VARIABLE(SYS_VAR_COLLATION_SERVER, val_buf); SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_DATABASE, val_buf); SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_SERVER, val_buf); // Here is the collation of the connection, OB currently only supports the client as utf8mb4 VAR_INT_TO_STRING(val_buf, CS_TYPE_UTF8MB4_BIN); SET_TENANT_VARIABLE(SYS_VAR_COLLATION_CONNECTION, val_buf); SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_CONNECTION, val_buf); /* * In Oracle mode, we are only compatible with binary mode, so collate can only end with _bin */ if (CS_TYPE_UTF8MB4_BIN == tenant_schema.get_collation_type() || CS_TYPE_GBK_BIN == tenant_schema.get_collation_type() || CS_TYPE_UTF16_BIN == tenant_schema.get_collation_type() || CS_TYPE_GB18030_BIN == tenant_schema.get_collation_type()) { VAR_INT_TO_STRING(val_buf, tenant_schema.get_collation_type()); SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_SERVER, val_buf); SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_DATABASE, val_buf); if (CHARSET_UTF8MB4 == ObCharset::charset_type_by_coll(tenant_schema.get_collation_type())) { OZ(databuff_printf(val_buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s", "AL32UTF8")); } else if (CHARSET_GBK == ObCharset::charset_type_by_coll(tenant_schema.get_collation_type())) { OZ(databuff_printf(val_buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s", "ZHS16GBK")); } else if (CHARSET_UTF16 == ObCharset::charset_type_by_coll(tenant_schema.get_collation_type())) { OZ(databuff_printf(val_buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s", "AL16UTF16")); } else if (CHARSET_GB18030 == ObCharset::charset_type_by_coll(tenant_schema.get_collation_type())) { OZ(databuff_printf(val_buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s", "ZHS32GB18030")); } SET_TENANT_VARIABLE(SYS_VAR_NLS_CHARACTERSET, val_buf); } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant collation set error", K(ret), K(tenant_schema.get_collation_type())); } // update oracle tenant schema if (OB_SUCC(ret)) { if (OB_FAIL(databuff_printf(sys_params[SYS_VAR_SQL_MODE].value_, sizeof(sys_params[SYS_VAR_SQL_MODE].value_), "%llu", DEFAULT_ORACLE_MODE))) { ret = OB_BUF_NOT_ENOUGH; LOG_WARN("set oracle tenant default sql mode failed", K(ret)); } } } // end HEAP_VAR } return ret; } // The value of certain system variables of the system/meta tenant int ObDDLService::update_special_tenant_sys_var( const ObSysVariableSchema &sys_variable_schema, ObSysParam *sys_params, int64_t params_capacity) { int ret = OB_SUCCESS; const uint64_t tenant_id = sys_variable_schema.get_tenant_id(); if (OB_ISNULL(sys_params) || OB_UNLIKELY(params_capacity < ObSysVarFactory::ALL_SYS_VARS_COUNT)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity)); } else { HEAP_VAR(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf) { if (is_sys_tenant(tenant_id)) { VAR_INT_TO_STRING(val_buf, sys_variable_schema.get_name_case_mode()); SET_TENANT_VARIABLE(SYS_VAR_LOWER_CASE_TABLE_NAMES, val_buf); OZ(databuff_printf(val_buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s", OB_SYS_HOST_NAME)); SET_TENANT_VARIABLE(SYS_VAR_OB_TCP_INVITED_NODES, val_buf); } else if (is_meta_tenant(tenant_id)) { ObString compatibility_mode("0"); SET_TENANT_VARIABLE(SYS_VAR_OB_COMPATIBILITY_MODE, compatibility_mode); } } // end HEAP_VAR } return ret; } int ObDDLService::init_system_variables( const ObCreateTenantArg &arg, ObTenantSchema &tenant_schema, ObSysVariableSchema &sys_variable_schema) { int ret = OB_SUCCESS; //MAX_SYS_PARAM_NUM is 500. When param num needed greater than 500, //you need to change OB_MAX_SYS_PARAM_NUM in "ob_define.h". const int64_t params_capacity = OB_MAX_SYS_PARAM_NUM; int64_t var_amount = ObSysVariables::get_amount(); const uint64_t tenant_id = tenant_schema.get_tenant_id(); ObMalloc alloc(ObModIds::OB_TEMP_VARIABLES); ObPtrGuard sys_params_guard(alloc); sys_variable_schema.reset(); sys_variable_schema.set_tenant_id(tenant_id); ObSysParam *sys_params = NULL; if (OB_INVALID_TENANT_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", KR(ret), K(tenant_id)); } else if (OB_ISNULL(schema_service_) || OB_ISNULL(sql_proxy_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ptr is null", KR(ret), KP_(schema_service), KP_(sql_proxy)); } else if (OB_FAIL(sys_params_guard.init())) { LOG_WARN("alloc sys parameters failed", KR(ret)); } else if (FALSE_IT(sys_params = sys_params_guard.ptr())) { } else if (OB_ISNULL(sys_params) || OB_UNLIKELY(var_amount > params_capacity)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity), K(var_amount)); } else { HEAP_VAR(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf) { // name_case_mode if (is_meta_tenant(tenant_id)) { sys_variable_schema.set_name_case_mode(OB_ORIGIN_AND_INSENSITIVE); } else if (OB_NAME_CASE_INVALID == arg.name_case_mode_) { sys_variable_schema.set_name_case_mode(OB_LOWERCASE_AND_INSENSITIVE); } else { sys_variable_schema.set_name_case_mode(arg.name_case_mode_); } // init default values for (int64_t i = 0; OB_SUCC(ret) && i < var_amount; ++i) { if (OB_FAIL(sys_params[i].init(tenant_id, ObSysVariables::get_name(i), ObSysVariables::get_type(i), ObSysVariables::get_value(i), ObSysVariables::get_min(i), ObSysVariables::get_max(i), ObSysVariables::get_info(i), ObSysVariables::get_flags(i)))) { LOG_WARN("fail to init param", KR(ret), K(tenant_id), K(i)); } } int64_t set_sys_var_count = arg.sys_var_list_.count(); bool use_default_parallel_servers_target = true; for (int64_t j = 0; OB_SUCC(ret) && j < set_sys_var_count; ++j) { ObSysVarIdValue sys_var; if (OB_FAIL(arg.sys_var_list_.at(j, sys_var))) { LOG_WARN("failed to get sys var", K(j), K(ret)); } else { const ObString &new_value = sys_var.value_; SET_TENANT_VARIABLE(sys_var.sys_id_, new_value); // sync tenant schema if (SYS_VAR_READ_ONLY == sys_var.sys_id_) { bool read_only_value = false; if (is_user_tenant(tenant_id)) { read_only_value = (0 == sys_var.value_.compare("1")); tenant_schema.set_read_only(read_only_value); } else { tenant_schema.set_read_only(read_only_value); } } else if (SYS_VAR_OB_COMPATIBILITY_MODE == sys_var.sys_id_) { if (is_user_tenant(tenant_id) && 0 == sys_var.value_.compare("1")) { tenant_schema.set_compatibility_mode(ObCompatibilityMode::ORACLE_MODE); } else { tenant_schema.set_compatibility_mode(ObCompatibilityMode::MYSQL_MODE); } } else if (SYS_VAR_PARALLEL_SERVERS_TARGET == sys_var.sys_id_) { use_default_parallel_servers_target = false; } } } // end for // For read_only, its priority: sys variable > tenant option. if (OB_SUCC(ret)) { ObString read_only_value = tenant_schema.is_read_only() ? "1" : "0"; SET_TENANT_VARIABLE(SYS_VAR_READ_ONLY, read_only_value); } // For compatibility_mode, its priority: sys variable > tenant option. if (OB_SUCC(ret)) { if (is_meta_tenant(tenant_id) || !tenant_schema.is_oracle_tenant()) { tenant_schema.set_compatibility_mode(ObCompatibilityMode::MYSQL_MODE); } ObString compat_mode_value = tenant_schema.is_oracle_tenant() ? "1" : "0"; SET_TENANT_VARIABLE(SYS_VAR_OB_COMPATIBILITY_MODE, compat_mode_value); } // If the user does not specify parallel_servers_target when creating tenant, // then calculate a default value based on cpu_count. // Considering that a tenant may have multiple resource pools, it is currently rudely considered // that the units in the pool are of the same structure, and directly take the unit config of the first resource pool // WARNING: If the unit is not structured, the number of threads allocated by default may be too large/too small int64_t default_px_thread_count = 0; if (OB_SUCC(ret) && (use_default_parallel_servers_target)) { HEAP_VAR(ObUnitConfig, unit_config) { if (OB_SYS_TENANT_ID == sys_variable_schema.get_tenant_id()) { // When creating a system tenant, the default value of px_thread_count is related to // default sys tenant max cpu const int64_t sys_default_max_cpu = static_cast(GCONF.get_sys_tenant_default_max_cpu()); default_px_thread_count = ObTenantCpuShare::calc_px_pool_share( sys_variable_schema.get_tenant_id(), sys_default_max_cpu); } else if (OB_UNLIKELY(NULL == unit_mgr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unit_mgr_ is null", K(ret), KP(unit_mgr_)); } else if (arg.pool_list_.count() <= 0) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tenant should have at least one pool", K(ret)); } else if (OB_FAIL(unit_mgr_->get_unit_config_by_pool_name( arg.pool_list_.at(0), unit_config))) { LOG_WARN("fail to get unit config", K(ret)); } else { int64_t cpu_count = static_cast(unit_config.unit_resource().max_cpu()); default_px_thread_count = ObTenantCpuShare::calc_px_pool_share( sys_variable_schema.get_tenant_id(), cpu_count); } } } if (OB_SUCC(ret) && use_default_parallel_servers_target && default_px_thread_count > 0) { // target cannot be less than 3, otherwise any px query will not come in int64_t default_px_servers_target = std::max(3L, static_cast(default_px_thread_count * 0.8)); VAR_INT_TO_STRING(val_buf, default_px_servers_target); SET_TENANT_VARIABLE(SYS_VAR_PARALLEL_SERVERS_TARGET, val_buf); } if (FAILEDx(update_mysql_tenant_sys_var( tenant_schema, sys_variable_schema, sys_params, params_capacity))) { LOG_WARN("failed to update_mysql_tenant_sys_var", KR(ret), K(tenant_schema), K(sys_variable_schema)); } else if (OB_FAIL(update_oracle_tenant_sys_var( tenant_schema, sys_variable_schema, sys_params, params_capacity))) { LOG_WARN("failed to update_oracle_tenant_sys_var", KR(ret), K(tenant_schema), K(sys_variable_schema)); } else if (OB_FAIL(update_special_tenant_sys_var( sys_variable_schema, sys_params, params_capacity))) { LOG_WARN("failed to update_special_tenant_sys_var", K(ret), K(sys_variable_schema)); } // set sys_variable if (OB_SUCC(ret)) { ObSysVarSchema sysvar_schema; for (int64_t i = 0; OB_SUCC(ret) && i < var_amount; i++) { sysvar_schema.reset(); if (OB_FAIL(ObSchemaUtils::convert_sys_param_to_sysvar_schema(sys_params[i], sysvar_schema))) { LOG_WARN("convert to sysvar schema failed", K(ret)); } else if (OB_FAIL(sys_variable_schema.add_sysvar_schema(sysvar_schema))) { LOG_WARN("add system variable failed", K(ret)); } } //end for } } // end HEAP_VAR } return ret; } /* * @description: * start transaction for DDL, lock and check schema has refreshed * @param[in] proxy * @param[in] tenant_id : the tenant of DDL * @param[in] tenant_refreshed_schema_version : the schema of tenant refreshed, if is 0, no need to lock and check. * @param[in] with_snapshot * */ int ObDDLSQLTransaction::start(ObISQLClient *proxy, const uint64_t &tenant_id, const int64_t &tenant_refreshed_schema_version, bool with_snapshot /*= false*/) { int ret = OB_SUCCESS; if (OB_ISNULL(proxy) || OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id || OB_INVALID_VERSION == tenant_refreshed_schema_version)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", KR(ret), K(tenant_id), KP(proxy), K(tenant_refreshed_schema_version)); } else if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", KR(ret), KP(schema_service_), KP(schema_service_->get_schema_service())); } else { tenant_id_ = tenant_id; share::schema::ObSchemaService *schema_service_impl = schema_service_->get_schema_service(); start_operation_schema_version_ = schema_service_impl->get_last_operation_schema_version(); start_operation_tenant_id_ = schema_service_impl->get_last_operation_tenant_id(); if (OB_FAIL(common::ObMySQLTransaction::start(proxy, tenant_id, with_snapshot))) { LOG_WARN("fail to start trans", KR(ret), K(with_snapshot), K(tenant_id_)); } else if (0 == tenant_refreshed_schema_version) { //maybe in bootstrap or second transaction of create tenant LOG_INFO("no need to lock and check schema is newest", KR(ret), K(tenant_id), K(tenant_refreshed_schema_version)); } else if (OB_FAIL(lock_all_ddl_operation(*this, tenant_id_))) { LOG_WARN("fail to lock all ddl operation", K(ret), K(tenant_id_)); } else { //double check, after lock success, check schema_version is newest before lock ObRefreshSchemaStatus schema_status; schema_status.tenant_id_ = tenant_id; int64_t version_in_inner_table = 0; if (OB_FAIL(schema_service_->get_schema_version_in_inner_table(*proxy, schema_status, version_in_inner_table))) { LOG_WARN("failed to get version in inner table", KR(ret), K(schema_status)); } else if (tenant_refreshed_schema_version != version_in_inner_table) { ret = OB_EAGAIN; LOG_WARN("RS not refresh the newest schema version, try again", KR(ret), K(tenant_id), K(tenant_refreshed_schema_version), K(version_in_inner_table)); } } } return ret; } int ObDDLSQLTransaction::start( ObISQLClient *proxy, const uint64_t tenant_id, bool with_snapshot /*= false*/) { int ret = OB_NOT_SUPPORTED; UNUSEDx(proxy, with_snapshot, tenant_id); return ret; } int ObDDLSQLTransaction::end(const bool commit) { int ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS; if (OB_ISNULL(schema_service_)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("schema service is null", K(ret)); } else if (commit && need_end_signal_) { share::schema::ObSchemaService *schema_service_impl = schema_service_->get_schema_service(); if (OB_ISNULL(schema_service_impl)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema service is null", K(ret), KP(schema_service_impl), KP(schema_service_)); } else if (start_operation_schema_version_ == schema_service_impl->get_last_operation_schema_version() && start_operation_tenant_id_ == schema_service_impl->get_last_operation_tenant_id()) { LOG_INFO("ddl operation is same, just skip", K(ret), K_(start_operation_schema_version), K_(start_operation_tenant_id)); } else { int64_t new_schema_version = OB_INVALID_VERSION; obrpc::ObDDLNopOpreatorArg arg; arg.schema_operation_.op_type_ = OB_DDL_END_SIGN; share::schema::ObDDLSqlService ddl_sql_service(*schema_service_impl); arg.schema_operation_.tenant_id_ = tenant_id_; if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id_, new_schema_version))) { LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id_)); } if (OB_FAIL(ret)) { } else if (OB_FAIL(ddl_sql_service.log_nop_operation(arg.schema_operation_, new_schema_version, arg.ddl_stmt_str_, *this))) { LOG_WARN("log end ddl operation failed", K(ret), K(arg)); } } } if (OB_SUCC(ret) && commit) { if (OB_FAIL(regist_multi_source_data_())) { LOG_WARN("regist_multi_source_data fail", KR(ret)); } } if (OB_SUCCESS != (tmp_ret = common::ObMySQLTransaction::end(commit && OB_SUCC(ret)))) { LOG_WARN("failed to end transaction", K(ret), K(tmp_ret), K(commit)); } ret = OB_SUCC(ret) ? tmp_ret : ret; // Clear tenant_id_ for success or failure tenant_id_ = OB_INVALID_ID; return ret; } int ObDDLSQLTransaction::regist_multi_source_data_() { int ret = OB_SUCCESS; char buf[] = "ddl_trans commit"; int64_t buf_len = strlen(buf); observer::ObInnerSQLConnection *conn = static_cast(get_connection()); if (OB_ISNULL(conn)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("connection cast to ObInnerSQLConnection failed", KR(ret)); } else if (OB_FAIL(conn->register_multi_data_source(tenant_id_, SYS_LS, transaction::ObTxDataSourceType::DDL_TRANS, buf, buf_len))) { LOG_WARN("register_msd_into_tx for DDL_TRANS failed", KR(ret), K_(tenant_id)); } else { // success } return ret; } int ObDDLService::do_schema_revise(const obrpc::ObSchemaReviseArg &arg) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; int64_t refreshed_schema_version = 0; ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(ret), K(arg)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { switch(arg.type_) { case obrpc::ObSchemaReviseArg::SchemaReviseType::REVISE_CONSTRAINT_COLUMN_INFO: { if (OB_FAIL(ddl_operator.revise_constraint_column_info(arg, trans))) { LOG_WARN("fail to create keystore", K(ret), K(arg), K(ret)); } break; } case obrpc::ObSchemaReviseArg::SchemaReviseType::REVISE_NOT_NULL_CONSTRAINT: { if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); LOG_WARN("get schema guard failed", K(ret)); } else if (OB_FAIL(ddl_operator.revise_not_null_constraint_info(arg, schema_guard, trans))) { LOG_WARN("fail to revise not null constraint info", K(ret), K(arg)); } break; } default: ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected schema revise type", K(ret), K(arg.type_)); break; } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } else { LOG_INFO("finish do schema revise", K(ret), K(arg)); } } return ret; } int ObDDLService::do_keystore_ddl(const obrpc::ObKeystoreDDLArg &arg) { int ret = OB_SUCCESS; ObKeystoreSchema &keystore_schema = const_cast(arg.schema_); const ObString *ddl_stmt_str = &arg.ddl_stmt_str_; const uint64_t tenant_id = keystore_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); bool is_exist = false; bool is_set_key = false; if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { switch(arg.type_) { case obrpc::ObKeystoreDDLArg::DDLType::CREATE_KEYSTORE: { if (OB_FAIL(schema_guard.check_keystore_exist(tenant_id, is_exist))) { LOG_WARN("fail get keystore from schema guard", K(ret), K(arg)); } else if (is_exist) { ret = OB_KEYSTORE_EXIST; LOG_WARN("keystore already exist", K(ret)); } else if (OB_FAIL(ddl_operator.create_keystore(keystore_schema, trans, schema_guard, ddl_stmt_str))) { LOG_WARN("fail create keystore", K(arg), K(ret)); } break; } case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_SET_KEY: is_set_key = true; case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_PASSWORD: case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_CLOSE: case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_OPEN: { if (OB_FAIL(schema_guard.check_keystore_exist(tenant_id, is_exist))) { LOG_WARN("fail get keystore from schema guard", K(ret), K(arg)); } else if (!is_exist) { ret = OB_KEYSTORE_NOT_EXIST; LOG_WARN("keystore is not exist", K(ret)); } else if (OB_FAIL(ddl_operator.alter_keystore(keystore_schema, trans, schema_guard, ddl_stmt_str, is_set_key, arg.is_kms_))) { LOG_WARN("fail alter keystore", K(arg), K(ret)); } break; } default: ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected stmt type", K(arg), K(ret)); break; } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish do keystore ddl", K(arg), K(ret)); return ret; } // bugfix: https://work.aone.alibaba-inc.com/issue/29886344 // In order to avoid the problem that log_id and schema_version cannot maintain the partial order relationship // caused by concurrent submission of DDL transactions of different sessions of inner_sql. // Before the DDL transaction is submitted, lock the same row of __all_ddl_operation (non-partitioned table) // to provide a table lock-like function to keep RS serially submitting DDL transactions. // // insert requires special support) and to avoid modifying the internal table during upgrade, lock the min(schema_version) // corresponding line of __all_ddl_operation. // 1. __all_ddl_operation is currently unclear about history, // and it can ensure that min (schema_version) corresponding lines are not deleted. // 2. Only bootstrap, and when building tenant transaction 2, __all_ddl_operation is empty, // and the DDL of other tenants can guarantee that the corresponding row of min (schema_version) can be locked. // However, the two scenarios either cause bootstrap to fail, or tenant creation fails. // You can restart bootstrap or rebuild the tenant to avoid this problem. int ObDDLSQLTransaction::lock_all_ddl_operation( ObMySQLTransaction &trans, const uint64_t tenant_id) { int ret = OB_SUCCESS; if (OB_INVALID_TENANT_ID == tenant_id || OB_INVALID_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id)); } else { ObSqlString sql; SMART_VAR(ObMySQLProxy::MySQLResult, res) { common::sqlclient::ObMySQLResult *result = NULL; if (OB_FAIL(sql.append_fmt( "select schema_version from %s where schema_version in (" "select min(schema_version) as schema_version from %s) for update", OB_ALL_DDL_OPERATION_TNAME, OB_ALL_DDL_OPERATION_TNAME))) { LOG_WARN("failed to append sql", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.read(res, tenant_id, sql.ptr()))) { LOG_WARN("failed to execute sql", KR(ret), K(tenant_id), K(sql)); } else if (OB_ISNULL(result = res.get_result())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get result", K(ret)); } else if (OB_FAIL(result->next())) { LOG_WARN("fail to get result", KR(ret), K(tenant_id)); } else { int64_t schema_version = OB_INVALID_VERSION; EXTRACT_INT_FIELD_MYSQL(*result, "schema_version", schema_version, int64_t); LOG_DEBUG("lock __all_ddl_operation by schema_version", KR(ret), K(tenant_id), K(schema_version)); if (OB_SUCC(ret)) { int64_t tmp_ret = result->next(); if (OB_SUCCESS == tmp_ret) { ret = OB_ERR_UNEXPECTED; LOG_WARN("more than one row", KR(ret), K(tenant_id)); } else if (OB_ITER_END == tmp_ret) { ret = OB_SUCCESS; } else { ret = tmp_ret; LOG_WARN("fail to get next row", KR(ret), K(tenant_id)); } } } } } return ret; } int ObDDLService::do_tablespace_ddl(const obrpc::ObTablespaceDDLArg &arg) { int ret = OB_SUCCESS; ObTablespaceSchema &tablespace_schema = const_cast(arg.schema_); const ObString &tablespace_name = tablespace_schema.get_tablespace_name(); const ObString *ddl_stmt_str = &arg.ddl_stmt_str_; const uint64_t tenant_id = tablespace_schema.get_tenant_id(); ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init"); } else if (!arg.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(arg), K(ret)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id)); } if (OB_SUCC(ret)) { ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDDLSQLTransaction trans(schema_service_); const ObTablespaceSchema *ts_schema = NULL; int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else { switch(arg.type_) { case obrpc::ObTablespaceDDLArg::DDLType::CREATE_TABLESPACE: { if (OB_FAIL(schema_guard.get_tablespace_schema_with_name(tenant_id, tablespace_name, ts_schema))) { LOG_WARN("fail get tablespace from schema guard", K(ret), K(tablespace_name), K(arg)); } else if (ts_schema) { ret = OB_TABLESPACE_EXIST; LOG_USER_ERROR(OB_TABLESPACE_EXIST, tablespace_name.length(), tablespace_name.ptr()); LOG_WARN("tablespace already exist", K(ret), K(tablespace_name)); } else if (OB_FAIL(ddl_operator.create_tablespace(tablespace_schema, trans, schema_guard, ddl_stmt_str))) { LOG_WARN("fail create tablespace", K(arg), K(ret)); } break; } case obrpc::ObTablespaceDDLArg::DDLType::DROP_TABLESPACE: { if (OB_FAIL(schema_guard.get_tablespace_schema_with_name(tenant_id, tablespace_name, ts_schema))) { LOG_WARN("fail get tablespace from schema guard", K(ret), K(tablespace_name), K(arg)); } else if (OB_ISNULL(ts_schema)) { ret = OB_TABLESPACE_NOT_EXIST; LOG_USER_ERROR(OB_TABLESPACE_NOT_EXIST, tablespace_name.length(), tablespace_name.ptr()); LOG_WARN("tablespace does not exist", K(ret), K(tablespace_name), K(arg)); } else { // deep copy tablespace_schema = *ts_schema; if (OB_FAIL(ddl_operator.drop_tablespace(tablespace_schema, trans, schema_guard, ddl_stmt_str))) { LOG_WARN("fail drop tablespace", K(arg), K(ret)); } } break; } case obrpc::ObTablespaceDDLArg::DDLType::ALTER_TABLESPACE: { if (OB_FAIL(schema_guard.get_tablespace_schema_with_name(tenant_id, tablespace_name, ts_schema))) { LOG_WARN("fail get tablespace from schema guard", K(ret), K(tablespace_name), K(arg)); } else if (OB_ISNULL(ts_schema)) { ret = OB_TABLESPACE_NOT_EXIST; LOG_USER_ERROR(OB_TABLESPACE_NOT_EXIST, tablespace_name.length(), tablespace_name.ptr()); LOG_WARN("tablespace does not exist", K(ret), K(tablespace_name), K(arg)); } else if (OB_FAIL(ddl_operator.alter_tablespace(tablespace_schema, trans, schema_guard, ddl_stmt_str))) { LOG_WARN("fail to alter tablespace", K(arg), K(ret)); } break; } default: ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected stmt type", K(arg), K(ret)); break; } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } LOG_INFO("finish do tablespace ddl", K(arg), K(ret)); return ret; } int ObDDLService::handle_profile_ddl(const ObProfileDDLArg &arg) { int ret = OB_SUCCESS; ObProfileSchema schema = arg.schema_; //make a copy uint64_t tenant_id = schema.get_tenant_id(); int64_t refreshed_schema_version = 0; ObSchemaGetterGuard schema_guard; if (OB_FAIL(check_inner_stat())) { LOG_WARN("check inner stat failed", K(ret)); } else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid input schema", K(ret), K(tenant_id)); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("fail to get schema guard", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version), K(tenant_id)); } if (OB_SUCC(ret)) { if (OB_FAIL(ddl_operator.handle_profile_function(schema, trans, arg.ddl_type_, arg.ddl_stmt_str_, schema_guard))) { LOG_WARN("handle profile function failed", K(ret), K(arg)); } } // Handle user cascading delete if (OB_SUCC(ret) && OB_DDL_DROP_PROFILE == arg.ddl_type_) { ObSEArray user_ids_with_profile; if (OB_FAIL(get_all_users_in_tenant_with_profile(tenant_id, schema.get_profile_id(), schema_guard, user_ids_with_profile))) { LOG_WARN("fail to get all users with profile id", K(schema.get_profile_id()), K(ret)); } else if (user_ids_with_profile.count() > 0 && !arg.is_cascade_) { ret = OB_ERR_PROFILE_STRING_HAS_USERS_ASSIGNED; LOG_USER_ERROR(OB_ERR_PROFILE_STRING_HAS_USERS_ASSIGNED, schema.get_profile_name_str().length(), schema.get_profile_name_str().ptr()); } else if (arg.is_cascade_) { ret = OB_NOT_SUPPORTED; LOG_WARN("not support feature", K(ret)); } } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } } return ret; } int ObDDLService::create_directory(const obrpc::ObCreateDirectoryArg &arg, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; const bool is_or_replace = arg.or_replace_; const uint64_t tenant_id = arg.schema_.get_tenant_id(); const uint64_t user_id = arg.user_id_; const ObString &directory_name = arg.schema_.get_directory_name(); const ObString &directory_path = arg.schema_.get_directory_path(); const ObDirectorySchema *schema_ptr = NULL; bool is_exist = false; bool is_oracle_mode = false; lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID; ObDirectorySchema new_schema; ObSchemaGetterGuard schema_guard; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) { LOG_WARN("failed to get compat mode", K(ret), K(tenant_id)); } else if (lib::Worker::CompatMode::ORACLE == compat_mode && FALSE_IT(is_oracle_mode = true)) { // do nothing } else if (!is_oracle_mode) { ret = OB_NOT_SUPPORTED; LOG_WARN("create directory under non oracle mode is not supported", K(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "create directory under non oracle mode"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_directory_schema_by_name(tenant_id, directory_name, schema_ptr))) { LOG_WARN("failed to get directory schema by name", K(ret), K(tenant_id), K(directory_name)); } else if (NULL != schema_ptr) { is_exist = true; if (OB_FAIL(new_schema.assign(*schema_ptr))) { LOG_WARN("failed to assign new directory schema", K(ret), K(*schema_ptr)); } else if (OB_FAIL(new_schema.set_directory_path(directory_path))) { LOG_WARN("failed to set directory path", K(ret), K(directory_path)); } } else if (NULL == schema_ptr) { if (OB_FAIL(new_schema.assign(arg.schema_))) { LOG_WARN("failed to assign new directory schema", K(ret), K(arg)); } } if (OB_FAIL(ret)) { // do nothing } else if (is_exist && !is_or_replace) { ret = OB_ERR_EXIST_OBJECT; LOG_WARN("directory already exists and is not replace operation", K(ret), K(is_or_replace), K(directory_name)); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (is_exist && is_or_replace && OB_FAIL(ddl_operator.alter_directory(*ddl_stmt_str, new_schema, trans))) { LOG_WARN("failed to alter directory", K(ret), K(new_schema)); } else if (!is_exist && OB_FAIL(ddl_operator.create_directory(*ddl_stmt_str, user_id, new_schema, trans))) { LOG_WARN("failed to create directory", K(ret), K(new_schema)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::drop_directory(const obrpc::ObDropDirectoryArg &arg, const ObString *ddl_stmt_str) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; const ObString &directory_name = arg.directory_name_; const ObDirectorySchema *schema_ptr = NULL; bool is_exist = false; bool is_oracle_mode = false; lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID; ObSchemaGetterGuard schema_guard; int64_t refreshed_schema_version = 0; if (OB_FAIL(check_inner_stat())) { LOG_WARN("variable is not init", K(ret)); } else if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) { LOG_WARN("failed to get compat mode", K(ret), K(tenant_id)); } else if (lib::Worker::CompatMode::ORACLE == compat_mode && FALSE_IT(is_oracle_mode = true)) { // do nothing } else if (!is_oracle_mode) { ret = OB_NOT_SUPPORTED; LOG_WARN("drop directory under non oracle mode is not supported", K(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop directory under non oracle mode"); } else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_directory_schema_by_name(tenant_id, directory_name, schema_ptr))) { LOG_WARN("failed to get schema by directory name", K(ret), K(tenant_id), K(directory_name)); } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); } else if (NULL != schema_ptr) { is_exist = true; } if (OB_FAIL(ret)) { // do nothing } else if (!is_exist) { ret = OB_ERR_OBJECT_STRING_DOES_NOT_EXIST; LOG_WARN("directory does not exist", K(ret), K(directory_name)); LOG_USER_ERROR(OB_ERR_OBJECT_STRING_DOES_NOT_EXIST, static_cast(directory_name.length()), directory_name.ptr()); } else { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); ObDirectorySchema schema; if (OB_FAIL(schema.assign(*schema_ptr))) { LOG_WARN("fail to assign directory schema", K(ret), K(*schema_ptr)); } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.drop_directory(*ddl_stmt_str, schema, trans))) { LOG_WARN("failed to drop directory", K(ret), K(schema)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } } if (OB_SUCC(ret)) { if (OB_FAIL(publish_schema(tenant_id))) { LOG_WARN("publish schema failed", K(ret)); } } return ret; } int ObDDLService::force_set_locality( ObSchemaGetterGuard &schema_guard, ObTenantSchema &new_tenant) { int ret = OB_SUCCESS; common::ObArray zones_in_pool; common::ObArray zone_region_list; common::ObArray resource_pool_names; uint64_t tenant_id = new_tenant.get_tenant_id(); const ObTenantSchema *orig_meta_tenant = NULL; ObTenantSchema new_meta_tenant; obrpc::ObModifyTenantArg dummy_arg; if (OB_FAIL(get_new_tenant_pool_zone_list(dummy_arg, new_tenant, resource_pool_names, zones_in_pool, zone_region_list))) { LOG_WARN("fail to get new tenant pool zone list", KR(ret), K(new_tenant)); } else if (OB_FAIL(parse_and_set_create_tenant_new_locality_options( schema_guard, new_tenant, resource_pool_names, zones_in_pool, zone_region_list))) { LOG_WARN("fail to parse and set new locality option", KR(ret), K(new_tenant)); } else { // deal with meta tenant related to a certain user tenant if (is_user_tenant(tenant_id)) { if (OB_FAIL(schema_guard.get_tenant_info(gen_meta_tenant_id(tenant_id), orig_meta_tenant))) { LOG_WARN("fail to get meta tenant schema", KR(ret), "meta_tenant_id", gen_meta_tenant_id(tenant_id)); } else if (OB_ISNULL(orig_meta_tenant)) { ret = OB_TENANT_NOT_EXIST; LOG_WARN("meta tenant not exist", KR(ret), "meta_tenant_id", gen_meta_tenant_id(tenant_id)); } else if (OB_FAIL(new_meta_tenant.assign(*orig_meta_tenant))) { LOG_WARN("fail to assgin meta tenant schema", KR(ret), KPC(orig_meta_tenant)); } else if (OB_FAIL(new_meta_tenant.set_locality(new_tenant.get_locality_str()))) { LOG_WARN("fail to set locality", KR(ret), "locality str", new_tenant.get_locality_str()); } else if (OB_FAIL(new_meta_tenant.set_previous_locality(ObString("")))) { LOG_WARN("fail to reset meta tenant previous locality", KR(ret)); } else if (OB_FAIL(parse_and_set_create_tenant_new_locality_options( schema_guard, new_meta_tenant, resource_pool_names, zones_in_pool, zone_region_list))) { LOG_WARN("fail to parse and set meta tenant new locality option", KR(ret), K(new_meta_tenant)); } } if (OB_SUCC(ret)) { ObDDLSQLTransaction trans(schema_service_); ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); int64_t refreshed_schema_version = 0; if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("failed to get tenant schema version", KR(ret)); } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) { LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version)); } else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant, trans))) { LOG_WARN("failed to alter tenant", KR(ret)); } else if (is_user_tenant(tenant_id) && OB_FAIL(ddl_operator.alter_tenant(new_meta_tenant, trans))) { LOG_WARN("failed to alter meta tenant", KR(ret)); } if (trans.is_started()) { int temp_ret = OB_SUCCESS; if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { LOG_WARN("trans end failed", "is_commit", OB_SUCC(ret), K(temp_ret)); ret = (OB_SUCC(ret)) ? temp_ret : ret; } } // publish schema if (OB_SUCC(ret) && OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) { LOG_WARN("publish schema failed, ", K(ret)); } } } return ret; } int ObDDLService::add_sys_table_lob_aux( const int64_t tenant_id, const uint64_t table_id, ObTableSchema &meta_schema, ObTableSchema &data_schema) { int ret = OB_SUCCESS; meta_schema.reset(); data_schema.reset(); if (OB_ALL_CORE_TABLE_TID == table_id) { // do nothing } else if (OB_FAIL(get_sys_table_lob_aux_schema(table_id, meta_schema, data_schema))) { LOG_WARN("fail to get sys table's index schema", KR(ret), K(table_id)); } else if (OB_FAIL(ObSchemaUtils::construct_tenant_space_full_table( tenant_id, meta_schema))) { LOG_WARN("fail to construct tenant meta table", KR(ret), K(tenant_id)); } else if (OB_FAIL(ObSchemaUtils::construct_tenant_space_full_table( tenant_id, data_schema))) { LOG_WARN("fail to construct tenant data schema", KR(ret), K(tenant_id)); } return ret; } int ObDDLService::get_tenant_external_consistent_ts(const int64_t tenant_id, int64_t &ts) { int ret = OB_SUCCESS; const int64_t timeout_us = THIS_WORKER.is_timeout_ts_valid() ? THIS_WORKER.get_timeout_remain() : GCONF.rpc_timeout; bool is_external_consistent = false; if (OB_FAIL(transaction::ObTsMgr::get_instance().get_ts_sync(tenant_id, timeout_us, ts, is_external_consistent))) { LOG_WARN("fail to get_ts_sync", K(ret), K(tenant_id)); } else if (!is_external_consistent) { ret = OB_STATE_NOT_MATCH; LOG_WARN("got ts of tenant is not external consistent", K(ret), K(tenant_id), K(ts), K(is_external_consistent)); } else { LOG_INFO("success to get_tenant_external_consistent_ts", K(tenant_id), K(ts), K(is_external_consistent)); } return ret; } int ObDDLService::check_has_multi_autoinc(ObTableSchema &table_schema) { int ret = OB_SUCCESS; ObTableSchema::const_column_iterator it_begin = table_schema.column_begin(); ObTableSchema::const_column_iterator it_end = table_schema.column_end(); ObColumnSchemaV2 *new_column_schema = nullptr; bool has_autoinc_col = false; for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) { if (OB_ISNULL(new_column_schema = static_cast(*it_begin))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("*it_begin is NULL", K(ret)); } else if (new_column_schema->is_autoincrement()) { if (has_autoinc_col) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "More than one auto increment column"); LOG_WARN("Only one auto increment row is allowed", K(ret)); } else { table_schema.set_autoinc_column_id(new_column_schema->get_column_id()); has_autoinc_col = true; } } } return ret; } // check whether it's modify column not null or modify constraint state, which need send two rpc. int ObDDLService::need_modify_not_null_constraint_validate( const obrpc::ObAlterTableArg &alter_table_arg, bool &is_add_not_null_col, bool &need_modify) const { int ret = OB_SUCCESS; need_modify = false; is_add_not_null_col = false; ObSchemaGetterGuard schema_guard; schema_guard.set_session_id(alter_table_arg.session_id_); const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_; const uint64_t tenant_id = alter_table_schema.get_tenant_id(); const ObString &origin_database_name = alter_table_schema.get_origin_database_name(); const ObString &origin_table_name = alter_table_schema.get_origin_table_name(); const ObTableSchema *orig_table_schema = NULL; if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not init", KR(ret)); } else if (obrpc::ObAlterTableArg::ADD_CONSTRAINT != alter_table_arg.alter_constraint_type_ && obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE != alter_table_arg.alter_constraint_type_) { // skip } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { LOG_WARN("fail to get tenant schema guard", KR(ret), K(tenant_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, origin_database_name, origin_table_name, false, orig_table_schema))) { LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), K(origin_database_name), K(origin_table_name)); } else if (OB_ISNULL(orig_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("NULL ptr", K(ret), KR(tenant_id), K(alter_table_arg), K(schema_guard.get_session_id())); } else if (alter_table_arg.alter_table_schema_.get_constraint_count() == 1) { ObTableSchema::const_constraint_iterator iter = alter_table_arg.alter_table_schema_.constraint_begin(); if (OB_ISNULL(iter) || OB_ISNULL(*iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("constraint is null", K(ret)); } else if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) { if (OB_UNLIKELY(1 != (*iter)->get_column_cnt())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected column count of not null constraint", K(ret), KPC(*iter)); } else if (!(*iter)->get_need_validate_data()) { // don't need validate data, do nothing. } else if (OB_INVALID_ID == *(*iter)->cst_col_begin()) { is_add_not_null_col = true; } else { need_modify = true; } } } else if (alter_table_arg.alter_table_schema_.get_constraint_count() > 1) { // more than one constraint, check column_id of all not null constraint must be invalid. // since we only support add more than one not null column in one ddl, // not support modify more than one column not null in one ddl. ObTableSchema::const_constraint_iterator iter = alter_table_arg.alter_table_schema_.constraint_begin(); for(; iter != alter_table_arg.alter_table_schema_.constraint_end() && OB_SUCC(ret); iter++) { if (OB_ISNULL(iter) || OB_ISNULL(*iter)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("constraint is null", K(ret)); } else if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) { if (OB_UNLIKELY(1 != (*iter)->get_column_cnt())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected column count of not null constraint", K(ret), KPC(*iter)); } else if (OB_UNLIKELY(OB_INVALID_ID != *(*iter)->cst_col_begin())) { ret = OB_NOT_SUPPORTED; LOG_WARN("modify not null column is not allowed with other DDL", K(ret)); LOG_USER_ERROR(OB_NOT_SUPPORTED, "Add/modify not null constraint together with other DDLs"); } } } is_add_not_null_col = true; } return ret; } bool ObDDLService::need_check_constraint_validity(const obrpc::ObAlterTableArg &alter_table_arg) const { bool need_check_validity = false; ObTableSchema::const_constraint_iterator iter = alter_table_arg.alter_table_schema_.constraint_begin(); // a ddl query add at most one constraint before. // after support not null constraint, in order to support alter table add multiple // not null columns, we have to support a ddl query add multiple constraint. for (; !need_check_validity && iter != alter_table_arg.alter_table_schema_.constraint_end(); iter++) { need_check_validity = (*iter)->get_need_validate_data(); } return need_check_validity; } } // end namespace rootserver } // end namespace oceanbase