22783 lines
		
	
	
		
			1.0 MiB
		
	
	
	
	
	
	
	
			
		
		
	
	
			22783 lines
		
	
	
		
			1.0 MiB
		
	
	
	
	
	
	
	
/**
 | 
						|
 * 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 "share/ob_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_user_sql_service.h"
 | 
						|
#include "share/partition_table/ob_partition_table_operator.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/ob_partition_modify.h"
 | 
						|
#include "share/sequence/ob_sequence_ddl_proxy.h"
 | 
						|
#include "share/ob_schema_status_proxy.h"
 | 
						|
#include "share/ob_tenant_mgr.h"
 | 
						|
#include "share/ob_worker.h"
 | 
						|
#include "sql/resolver/ob_stmt_type.h"
 | 
						|
#include "sql/resolver/ddl/ob_ddl_resolver.h"
 | 
						|
#include "storage/ob_i_partition_storage.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_leader_coordinator.h"
 | 
						|
#include "rootserver/ob_zone_unit_provider.h"
 | 
						|
#include "rootserver/ob_alloc_replica_strategy.h"
 | 
						|
#include "rootserver/ob_alter_locality_checker.h"
 | 
						|
#include "rootserver/ob_rs_job_table_operator.h"
 | 
						|
#include "rootserver/ob_ddl_sql_generator.h"
 | 
						|
#include "rootserver/ob_ddl_help.h"
 | 
						|
#include "rootserver/ob_locality_util.h"
 | 
						|
#include "rootserver/ob_partition_creator.h"
 | 
						|
#include "rootserver/ob_partition_spliter.h"
 | 
						|
#include "rootserver/ob_root_service.h"
 | 
						|
#include "ob_freeze_info_manager.h"
 | 
						|
#include "share/ob_multi_cluster_util.h"
 | 
						|
#include "lib/utility/ob_tracepoint.h"
 | 
						|
#include "observer/ob_server_struct.h"
 | 
						|
#include "storage/transaction/ob_ts_mgr.h"
 | 
						|
#include "storage/transaction/ob_i_ts_source.h"
 | 
						|
#include "sql/engine/px/ob_px_util.h"
 | 
						|
#include "rootserver/ob_single_partition_balance.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"
 | 
						|
 | 
						|
namespace oceanbase {
 | 
						|
using namespace common;
 | 
						|
using namespace share;
 | 
						|
using namespace schema;
 | 
						|
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(id, value)                                    \
 | 
						|
  if (OB_SUCC(ret)) {                                                     \
 | 
						|
    if (OB_FAIL(sys_params[id].init(sys_variable_schema.get_tenant_id(),  \
 | 
						|
            "",                                                           \
 | 
						|
            ObSysVariables::get_name(id),                                 \
 | 
						|
            ObSysVariables::get_type(id),                                 \
 | 
						|
            value,                                                        \
 | 
						|
            ObSysVariables::get_min(id),                                  \
 | 
						|
            ObSysVariables::get_max(id),                                  \
 | 
						|
            ObSysVariables::get_info(id),                                 \
 | 
						|
            ObSysVariables::get_flags(id)))) {                            \
 | 
						|
      LOG_WARN("failed to set tenant variable", K(ret), K(value), K(id)); \
 | 
						|
    }                                                                     \
 | 
						|
  }
 | 
						|
// 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<int>(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),
 | 
						|
      pt_operator_(NULL),
 | 
						|
      server_mgr_(NULL),
 | 
						|
      zone_mgr_(NULL),
 | 
						|
      unit_mgr_(NULL),
 | 
						|
      root_balancer_(NULL),
 | 
						|
      freeze_info_manager_(NULL),
 | 
						|
      snapshot_mgr_(NULL),
 | 
						|
      task_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::ObPartitionTableOperator& pt_operator, ObServerManager& server_mgr, ObZoneManager& zone_mgr,
 | 
						|
    ObUnitManager& unit_mgr, ObRootBalancer& root_balancer, ObFreezeInfoManager& freeze_info_manager,
 | 
						|
    ObSnapshotInfoManager& snapshot_mgr, ObRebalanceTaskMgr& task_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;
 | 
						|
    pt_operator_ = &pt_operator;
 | 
						|
    server_mgr_ = &server_mgr;
 | 
						|
    zone_mgr_ = &zone_mgr;
 | 
						|
    unit_mgr_ = &unit_mgr;
 | 
						|
    root_balancer_ = &root_balancer;
 | 
						|
    freeze_info_manager_ = &freeze_info_manager;
 | 
						|
    snapshot_mgr_ = &snapshot_mgr;
 | 
						|
    task_mgr_ = &task_mgr;
 | 
						|
    stopped_ = false;
 | 
						|
    inited_ = true;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::get_zones_in_region(const common::ObRegion& region,
 | 
						|
    const common::ObIArray<common::ObString>& zone_list, common::ObIArray<common::ObZone>& zones)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<share::ObZoneInfo> 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;
 | 
						|
}
 | 
						|
 | 
						|
// for bootstrap
 | 
						|
int ObDDLService::prepare_create_partition(ObPartitionCreator& creator, ObTableSchema& table_schema,
 | 
						|
    const int64_t paxos_replica_num, const ObArray<ObUnit>& units, const ObAddr& suggest_leader,
 | 
						|
    const int64_t frozen_version, const bool is_standby)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (!table_schema.is_valid() || units.empty() || !suggest_leader.is_valid() || frozen_version <= 0 ||
 | 
						|
             paxos_replica_num <= 0) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument",
 | 
						|
        K(ret),
 | 
						|
        K(table_schema),
 | 
						|
        "unit count",
 | 
						|
        units.count(),
 | 
						|
        K(suggest_leader),
 | 
						|
        K(frozen_version),
 | 
						|
        K(paxos_replica_num));
 | 
						|
  } else {
 | 
						|
    ObTablePartitionAddr table_addr;
 | 
						|
    int64_t part_num = table_schema.get_all_part_num();
 | 
						|
    table_addr.reserve(part_num);
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < part_num; ++i) {
 | 
						|
      ObPartitionAddr part_addr;
 | 
						|
      FOREACH_X(u, units, OB_SUCC(ret))
 | 
						|
      {
 | 
						|
        if (OB_ISNULL(u)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("u is null", K(ret));
 | 
						|
        } else {
 | 
						|
          ObReplicaAddr replica;
 | 
						|
          replica.unit_id_ = u->unit_id_;
 | 
						|
          replica.zone_ = u->zone_;
 | 
						|
          replica.addr_ = u->server_;
 | 
						|
          replica.replica_type_ = common::REPLICA_TYPE_FULL;
 | 
						|
          if (suggest_leader == u->server_) {
 | 
						|
            replica.initial_leader_ = true;
 | 
						|
          }
 | 
						|
          if (OB_FAIL(part_addr.push_back(replica))) {
 | 
						|
            LOG_WARN("array push back failed", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        if (OB_FAIL(table_addr.push_back(part_addr))) {
 | 
						|
          LOG_WARN("array push back failed", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      bool is_bootstrap = true;
 | 
						|
      ObArray<int64_t> part_ids;
 | 
						|
      ObArray<ObTableSchema> schemas;
 | 
						|
      ObPartIdsGenerator gen(table_schema);
 | 
						|
      int64_t non_paxos_replica_num = 0;
 | 
						|
      ObCreateTableMode create_mode = OB_CREATE_TABLE_MODE_LOOSE;
 | 
						|
      share::ObSimpleFrozenStatus frozen_status;
 | 
						|
      int64_t restore = REPLICA_NOT_RESTORE;
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(
 | 
						|
                     freeze_info_manager_->get_frozen_status_for_create_partition(OB_SYS_TENANT_ID, frozen_status))) {
 | 
						|
        LOG_WARN("fail to get freeze info", K(ret), K(frozen_status));
 | 
						|
      } else if (OB_FAIL(gen.gen(part_ids))) {
 | 
						|
        LOG_WARN("generate part ids failed", K(ret));
 | 
						|
      } else if (OB_FAIL(schemas.push_back(table_schema))) {
 | 
						|
        LOG_WARN("fail to push back table schema", K(ret));
 | 
						|
      } else if (OB_FAIL(prepare_create_partitions(creator,
 | 
						|
                     table_schema.get_table_id(),
 | 
						|
                     table_schema.get_schema_version(),
 | 
						|
                     table_schema.get_all_part_num(),
 | 
						|
                     table_schema.get_partition_cnt(),
 | 
						|
                     paxos_replica_num,
 | 
						|
                     non_paxos_replica_num,
 | 
						|
                     part_ids,
 | 
						|
                     table_addr,
 | 
						|
                     schemas,
 | 
						|
                     is_bootstrap,
 | 
						|
                     is_standby,
 | 
						|
                     create_mode,
 | 
						|
                     restore,
 | 
						|
                     frozen_status))) {
 | 
						|
        LOG_WARN("create partitions failed", K(ret), K(table_addr), K(table_schema), K(frozen_version));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
  int64_t version_in_inner_table = OB_INVALID_VERSION;
 | 
						|
  ObRefreshSchemaStatus schema_status;
 | 
						|
  bool use_local = false;
 | 
						|
  if (OB_INVALID_TENANT_ID == tenant_id) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid tenant_id", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(get_is_standby_cluster(is_standby))) {
 | 
						|
    LOG_WARN("failed to get is standby cluster", K(ret));
 | 
						|
  } else if (OB_ISNULL(schema_service_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema_service is null", 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 && OB_SYS_TENANT_ID != tenant_id) || is_restore) {
 | 
						|
    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. For standby cluster: schema_status is reset, we can refresh schema now.
 | 
						|
      // 2. For restore tenant: sys replicas are restored.
 | 
						|
    } else if (is_restore) {
 | 
						|
      ret = OB_NOT_SUPPORTED;
 | 
						|
      LOG_WARN("Can't refresh schema when sys replicas are not restored yet", KR(ret), K(tenant_id));
 | 
						|
    } else 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 {
 | 
						|
      use_local = true;
 | 
						|
      LOG_INFO("use local tenant schema guard", K(ret), K(tenant_id));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (use_local) {
 | 
						|
  } else {
 | 
						|
    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<uint64_t> 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;
 | 
						|
  // First get the guard of the system tenant, and then get the guard of the user tenant.
 | 
						|
  // Under the premise that DDL is executed serially on the RS,
 | 
						|
  // it can ensure that the schemas of the two tenants are updated to the latest.
 | 
						|
  // Then use the schema_guard of the user tenant to get the schema.
 | 
						|
  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(get_tenant_schema_guard_with_version_in_inner_table(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 {
 | 
						|
    // previous locality is null, it is necessary to check locality of tablegroup of tenant.
 | 
						|
    common::ObArray<const share::schema::ObSimpleTablegroupSchema*> tablegroup_schemas;
 | 
						|
    if (OB_FAIL(schema_guard.get_tablegroup_schemas_in_tenant(tenant_id, tablegroup_schemas))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schemas in tenant", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; !in_alter_locality && OB_SUCC(ret) && i < tablegroup_schemas.count(); ++i) {
 | 
						|
        bool tablegroup_tenant_locality_match = false;
 | 
						|
        const ObSimpleTablegroupSchema* tablegroup_schema = tablegroup_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == tablegroup_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("unexpected error, tablegroup schema is null", K(ret), KP(tablegroup_schema));
 | 
						|
        } else if (!is_new_tablegroup_id(tablegroup_schema->get_tablegroup_id())) {
 | 
						|
          // only check tablegroup created after 2.0
 | 
						|
        } else if (tablegroup_schema->get_locality_str().empty()) {
 | 
						|
          // tablegroup locality is empty, derived from tenant, ignore it
 | 
						|
        } else {
 | 
						|
          // tablegroup locality is not empty, has its own locality
 | 
						|
          if (!tablegroup_schema->get_previous_locality_str().empty()) {
 | 
						|
            // a tablegroup locality modification is being executed,
 | 
						|
            in_alter_locality = true;
 | 
						|
          } else if (OB_FAIL(check_tablegroup_tenant_locality_match(
 | 
						|
                         schema_guard, *tablegroup_schema, *tenant_schema, tablegroup_tenant_locality_match))) {
 | 
						|
            LOG_WARN("fail to check tablegroup tenant locality match", K(ret));
 | 
						|
          } else if (!tablegroup_tenant_locality_match) {
 | 
						|
            in_alter_locality = true;
 | 
						|
          } else {
 | 
						|
          }  // tablegroup tenant locality match, good
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // previous locality is null, it is necessary to check locality of table.
 | 
						|
    common::ObArray<const share::schema::ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
    if (OB_FAIL(ret) || in_alter_locality) {
 | 
						|
      // do nothing
 | 
						|
    } else if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(tenant_id, table_schemas))) {
 | 
						|
      LOG_WARN("fail to get table schemas in tenant", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; !in_alter_locality && OB_SUCC(ret) && i < table_schemas.count(); ++i) {
 | 
						|
        bool table_tenant_locality_match = false;
 | 
						|
        const ObSimpleTableSchemaV2* table_schema = table_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == table_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("unexpected error, table schema is null", K(ret), KP(table_schema));
 | 
						|
        } else if (!table_schema->has_self_partition()) {
 | 
						|
          // table without partition, ignore it
 | 
						|
        } else if (is_new_tablegroup_id(table_schema->get_tablegroup_id())) {
 | 
						|
          // ignore table in 2.0 tablegorup
 | 
						|
        } else if (table_schema->get_locality_str().empty()) {
 | 
						|
          // table locality is empty, derived from tenant, ignore it
 | 
						|
        } else {
 | 
						|
          // table locality is not empty, has its own locality
 | 
						|
          if (!table_schema->get_previous_locality_str().empty()) {
 | 
						|
            // a table locality modification is being executed,
 | 
						|
            in_alter_locality = true;
 | 
						|
          } else if (OB_FAIL(check_table_tenant_locality_match(
 | 
						|
                         schema_guard, *table_schema, *tenant_schema, table_tenant_locality_match))) {
 | 
						|
            LOG_WARN("fail to check table tenant locality match", K(ret), KPC(table_schema), KPC(tenant_schema));
 | 
						|
          } else if (!table_tenant_locality_match) {
 | 
						|
            in_alter_locality = true;
 | 
						|
          } else {
 | 
						|
          }  // table tenant locality match, good
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_user_tables(const bool if_not_exist, const ObString& ddl_stmt_str,
 | 
						|
    ObIArray<ObTableSchema>& table_schemas, const int64_t frozen_version, obrpc::ObCreateTableMode create_mode,
 | 
						|
    ObSchemaGetterGuard& schema_guard, const uint64_t last_replay_log_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  RS_TRACE(create_user_tables_begin);
 | 
						|
  uint64_t tenant_id = OB_INVALID_TENANT_ID;
 | 
						|
  bool is_standby = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("not init", K(ret));
 | 
						|
  } 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 {
 | 
						|
    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, table_schemas, frozen_version, create_mode, last_replay_log_id))) {
 | 
						|
    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);
 | 
						|
    int tmp_ret = OB_SUCCESS;
 | 
						|
    int64_t orig_schema_version = OB_INVALID_VERSION;
 | 
						|
    uint64_t fetch_tenant_id = is_inner_table(table_schemas.at(0).get_table_id()) ? OB_SYS_TENANT_ID : tenant_id;
 | 
						|
    if (OB_SUCCESS !=
 | 
						|
        (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(fetch_tenant_id, orig_schema_version))) {
 | 
						|
      LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(fetch_tenant_id));
 | 
						|
    } else if (OB_SUCCESS != (tmp_ret = publish_schema(fetch_tenant_id))) {
 | 
						|
      LOG_WARN("publish_schema failed", K(tmp_ret), K(fetch_tenant_id));
 | 
						|
    } else if (OB_CREATE_TABLE_MODE_RESTORE != create_mode) {
 | 
						|
      RS_TRACE(public_schema_end);
 | 
						|
      int64_t new_schema_version = OB_INVALID_VERSION;
 | 
						|
      if (OB_SUCCESS !=
 | 
						|
          (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(tenant_id, new_schema_version))) {
 | 
						|
        LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(tenant_id));
 | 
						|
      } else if (new_schema_version != orig_schema_version) {
 | 
						|
        RS_TRACE(wait_election_begin);
 | 
						|
        for (int64_t i = 0; i < table_schemas.count(); ++i) {
 | 
						|
          // Different tables are mutually exclusive and do not affect each other
 | 
						|
          if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_table_schema(table_schemas.at(i)))) {
 | 
						|
            LOG_WARN("fail to check need wait elect", KR(tmp_ret), K(table_schemas.at(i)), K(tenant_id));
 | 
						|
          }
 | 
						|
          if (OB_SUCC(ret)) {
 | 
						|
            ret = tmp_ret;
 | 
						|
          }
 | 
						|
        }
 | 
						|
        RS_TRACE(wait_election_end);
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  RS_TRACE_EXT(create_user_tables_end, OB_ID(ret), ret);
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_inner_expr_index_in_trans(const ObTableSchema& orig_table_schema,
 | 
						|
    ObTableSchema& new_table_schema, ObIArray<ObColumnSchemaV2*>& new_columns, ObTableSchema& index_schema,
 | 
						|
    const obrpc::ObCreateTableMode create_mode, 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();
 | 
						|
  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(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)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans, ", K(ret));
 | 
						|
    }
 | 
						|
    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, 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_self_partition() && OB_CREATE_TABLE_MODE_RESTORE != create_mode) {
 | 
						|
      ObArray<ObTableSchema> schemas;
 | 
						|
      if (OB_FAIL(schemas.push_back(index_schema))) {
 | 
						|
        LOG_WARN("fail to push back table schema", K(ret));
 | 
						|
      } else if (OB_FAIL(create_table_partitions(
 | 
						|
                     index_schema, last_schema_version, create_mode, frozen_version, schemas))) {
 | 
						|
        LOG_WARN("fail to create table partitions", K(ret), K(index_schema), K(last_schema_version), K(frozen_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::create_inner_expr_index(const ObTableSchema& orig_table_schema, ObTableSchema& new_table_schema,
 | 
						|
    ObIArray<ObColumnSchemaV2*>& new_columns, ObTableSchema& index_schema, const obrpc::ObCreateTableMode create_mode,
 | 
						|
    const int64_t frozen_version, const ObString* ddl_stmt_str)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  uint64_t tenant_id = orig_table_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_FAIL(create_inner_expr_index_in_trans(orig_table_schema,
 | 
						|
                 new_table_schema,
 | 
						|
                 new_columns,
 | 
						|
                 index_schema,
 | 
						|
                 create_mode,
 | 
						|
                 frozen_version,
 | 
						|
                 ddl_stmt_str))) {
 | 
						|
    LOG_WARN("create_ctxcat index_in_trans failed", K(ret), K(new_table_schema), K(index_schema));
 | 
						|
  } else if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
    LOG_WARN("publish_schema failed", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::generate_global_index_locality_and_primary_zone(
 | 
						|
    const share::schema::ObTableSchema& table_schema, share::schema::ObTableSchema& index_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = table_schema.get_tenant_id();
 | 
						|
  // primary zone and locality of global index is the same as its data table.
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObTenantSchema* tenant_schema = NULL;
 | 
						|
  common::ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
  common::ObArray<ObZone> zone_list;
 | 
						|
  bool extend_empty_locality = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(NULL == schema_service_)) {
 | 
						|
    LOG_WARN("schema service ptr is null", 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 if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
 | 
						|
    LOG_WARN("fail to get tenant info", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_UNLIKELY(NULL == tenant_schema)) {
 | 
						|
    ret = OB_TENANT_NOT_EXIST;
 | 
						|
    LOG_WARN("tenant schema is not exist", K(ret));
 | 
						|
  } else if (FALSE_IT(index_schema.set_locality(table_schema.get_locality()))) {
 | 
						|
    // will never be here
 | 
						|
  } else if (FALSE_IT(index_schema.set_primary_zone(table_schema.get_primary_zone()))) {
 | 
						|
    // will never be here
 | 
						|
  } else if (OB_FAIL(tenant_schema->get_zone_list(zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else 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(parse_and_set_new_locality_options(
 | 
						|
                 schema_guard, tenant_id, index_schema, zone_list, zone_region_list, extend_empty_locality))) {
 | 
						|
    LOG_WARN("fail ot parse and set new table locality options", K(ret));
 | 
						|
  } else if (OB_FAIL(check_create_table_replica_options(index_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail to check create table replica options", K(ret));
 | 
						|
  } else {
 | 
						|
  }  // no more to do
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_global_index(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(generate_global_index_locality_and_primary_zone(table_schema, index_schema))) {
 | 
						|
    LOG_WARN("fail to generate global index locality and primary zone", 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_user_table(arg, index_schema, frozen_version))) {
 | 
						|
    LOG_WARN("fail to create global index", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_global_inner_expr_index(const obrpc::ObCreateIndexArg& arg,
 | 
						|
    const share::schema::ObTableSchema& orig_table_schema, share::schema::ObTableSchema& new_table_schema,
 | 
						|
    common::ObIArray<share::schema::ObColumnSchemaV2*>& 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(generate_global_index_locality_and_primary_zone(new_table_schema, index_schema))) {
 | 
						|
    LOG_WARN("fail to generate global index locality and primary zone", K(ret));
 | 
						|
  } else if (OB_FAIL(create_inner_expr_index(orig_table_schema,
 | 
						|
                 new_table_schema,
 | 
						|
                 new_columns,
 | 
						|
                 index_schema,
 | 
						|
                 arg.create_mode_,
 | 
						|
                 frozen_version,
 | 
						|
                 &arg.ddl_stmt_str_))) {
 | 
						|
    LOG_WARN("fail to create inner expr index", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// create_user_table is used by create index
 | 
						|
int ObDDLService::create_user_table(
 | 
						|
    const obrpc::ObCreateIndexArg& arg, ObTableSchema& table_schema, const int64_t frozen_version)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  obrpc::ObCreateTableMode create_mode = arg.create_mode_;
 | 
						|
  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(table_schema.get_database_id(), database_schema))) {
 | 
						|
        LOG_WARN("get_database_schema failed", 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)) {
 | 
						|
      table_schema.set_create_mem_version(frozen_version);
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        if (OB_FAIL(create_table_in_trans(table_schema, frozen_version, create_mode, ddl_stmt_str, NULL))) {
 | 
						|
          LOG_WARN("create_table_in_trans failed", K(table_schema), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    int tmp_ret = OB_SUCCESS;
 | 
						|
    int64_t orig_schema_version = OB_INVALID_VERSION;
 | 
						|
    int64_t cur_schema_version = OB_INVALID_VERSION;
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_SUCCESS !=
 | 
						|
               (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(tenant_id, orig_schema_version))) {
 | 
						|
      LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(tenant_id));
 | 
						|
    } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
 | 
						|
      LOG_WARN("publish_schema failed", K(tmp_ret));
 | 
						|
    } else if (OB_CREATE_TABLE_MODE_RESTORE == create_mode) {
 | 
						|
      // skip
 | 
						|
    } else if (OB_SUCCESS !=
 | 
						|
               (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(tenant_id, cur_schema_version))) {
 | 
						|
      LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(tenant_id));
 | 
						|
    } else if (cur_schema_version != orig_schema_version) {
 | 
						|
      if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_table_schema(table_schema))) {
 | 
						|
        LOG_WARN("fail to check need wait leader by schema", KR(tmp_ret), K(table_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      ret = tmp_ret;
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  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 && !is_sync_primary_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 (extract_pure_id(db_id) >= OB_USER_DATABASE_ID) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      (void)snprintf(err_msg, sizeof(err_msg), "%s %lu", "create db with database_id larger than", OB_USER_DATABASE_ID);
 | 
						|
    } else {
 | 
						|
      schema.set_database_id(combine_id(schema.get_tenant_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 &&
 | 
						|
          !is_sync_primary_ddl()) {  // Only when the configuration item switch is turned on can the internal table is
 | 
						|
                                     // created or the standalone cluster synchronize the DDL of primary cluster.
 | 
						|
        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 table with table_id larger than", OB_MIN_USER_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_id(table_id)) {  // 30001 ~ 50000 is a reserved interval for future expansion
 | 
						|
        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(combine_id(schema.get_tenant_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;
 | 
						|
}
 | 
						|
 | 
						|
/* generate_schema is called when creating a data table,
 | 
						|
 * IndexBuilder::generate_schema is called when create a index table.
 | 
						|
 * In addition, after rebuilding zone_list, when creating a normal data table,
 | 
						|
 * the zone_list passed in by create_table will be ignored
 | 
						|
 * Fill in the zone_list of TableSchema as the tenant's zone_list directly in the ddl_service layer.
 | 
						|
 * At the ddl_operator layer, according to the locality of the table,
 | 
						|
 * decide how to fill in the zone_list field of the __all_table table:
 | 
						|
 * 1. The locality of the table is empty or completely matches the locality of the tenant,
 | 
						|
 *    and the zone_list of the __all_table table is filled null.
 | 
						|
 * 2. The locality of the table does not completely match the tenant,
 | 
						|
 *    so pass the ddl_service to the zone_list of the TableSchema and fill it in
 | 
						|
 */
 | 
						|
int ObDDLService::generate_schema(const ObCreateTableArg& arg, ObTableSchema& schema, const int64_t frozen_version)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const ObIArray<ObConstraint>& 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;
 | 
						|
  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(schema.get_database_id(), database_schema))) {
 | 
						|
      LOG_WARN("get_database_schema failed", "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 {
 | 
						|
      schema.set_table_id(new_table_id);
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (schema.has_partition()) {  // include table of standalone and binding
 | 
						|
      common::ObArray<share::ObResourcePoolName> 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 if (schema.get_locality_str().empty()) {
 | 
						|
        // derived from tenant locality, no need to check
 | 
						|
      } 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_pool_names_of_tenant(schema.get_tenant_id(), pool_names))) {
 | 
						|
        LOG_WARN("fail to get pool names", K(ret));
 | 
						|
      } else if (OB_FAIL(check_pools_unit_num_enough_for_schema_locality(pool_names, guard, schema))) {
 | 
						|
        LOG_WARN("pool unit num is not enough for locality", K(ret));
 | 
						|
      } else {
 | 
						|
      }  // no more to do
 | 
						|
    } else {
 | 
						|
    }  // has no partition
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    schema.set_create_mem_version(frozen_version);
 | 
						|
    if (GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_1440) {
 | 
						|
      schema.get_part_option().set_partition_cnt_within_partition_table(0);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  // constraints
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < constraints.count(); ++i) {
 | 
						|
    ObConstraint& cst = const_cast<ObConstraint&>(constraints.at(i));
 | 
						|
    cst.set_tenant_id(tenant_id);
 | 
						|
    cst.set_table_id(schema.get_table_id());
 | 
						|
    // todo: cons id should be globally unique, currently the only one in the table
 | 
						|
    cst.set_constraint_id(i + 1);
 | 
						|
    // Check whether the name of the constraint is repeated
 | 
						|
    if (!cst.get_constraint_name_str().empty()) {
 | 
						|
      bool is_constraint_name_exist = true;
 | 
						|
      if (OB_FAIL(
 | 
						|
              check_constraint_name_is_exist(guard, schema, cst.get_constraint_name_str(), 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;
 | 
						|
        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(i), K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    // todo: cons id should be globally unique, currently the only one in the table
 | 
						|
    schema.set_max_used_constraint_id(constraints.count());
 | 
						|
  }
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::get_uk_cst_id_for_self_ref(const ObIArray<ObTableSchema>& 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<ObString, 8> uk_columns;
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < index_info.get_size(); ++i) {
 | 
						|
        if (OB_ISNULL(index_col = index_table_schema.get_column_schema(index_info.get_column(i)->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<ObString>& 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;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_constraint_name_is_exist(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTableSchema& table_schema, const common::ObString& constraint_name,
 | 
						|
    bool& is_constraint_name_exist)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  uint64_t constraint_id = OB_INVALID_ID;
 | 
						|
  is_constraint_name_exist = true;
 | 
						|
 | 
						|
  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;
 | 
						|
  } else {
 | 
						|
    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;
 | 
						|
    } else {
 | 
						|
      is_constraint_name_exist = false;
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::try_rollback_alter_table_locality(const obrpc::ObAlterTableArg& arg,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const share::schema::ObTenantSchema& tenant_schema,
 | 
						|
    share::schema::ObTableSchema& new_schema, const share::schema::ObTableSchema& orig_schema,
 | 
						|
    const ObIArray<share::schema::ObZoneRegion>& zone_region_list, const obrpc::ObAlterTableArg& alter_table_arg,
 | 
						|
    AlterLocalityOp& alter_locality_op)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(alter_table_arg);
 | 
						|
  UNUSED(zone_region_list);
 | 
						|
  alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
  if (new_schema.get_locality_str().empty()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid table locality", K(ret));
 | 
						|
  } else if (new_schema.get_locality_str() != orig_schema.get_previous_locality_str()) {
 | 
						|
    // The locality string has been standardized, and you can determine whether it matches by comparing directly
 | 
						|
    MODIFY_LOCALITY_NOT_ALLOWED();
 | 
						|
    LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter table locality while previous locality change in progress");
 | 
						|
  } else {
 | 
						|
    bool is_match = false;
 | 
						|
    if (OB_FAIL(check_table_tenant_locality_match(schema_guard, new_schema, tenant_schema, is_match))) {
 | 
						|
      LOG_WARN("table tenant locality not match", K(ret));
 | 
						|
    } else if (!is_match) {
 | 
						|
      MODIFY_LOCALITY_NOT_ALLOWED();
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "table and tenant locality not match");
 | 
						|
      LOG_WARN("table and tenant locality not match",
 | 
						|
          K(ret),
 | 
						|
          "tenant locality",
 | 
						|
          tenant_schema.get_locality_str(),
 | 
						|
          "new table locality",
 | 
						|
          new_schema.get_locality_str());
 | 
						|
    } else {
 | 
						|
      if (arg.alter_table_schema_.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::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::try_rollback_alter_tablegroup_locality(const ObAlterTablegroupArg& arg,
 | 
						|
    share::schema::ObTablegroupSchema& new_schema, const share::schema::ObTablegroupSchema& orig_schema,
 | 
						|
    AlterLocalityOp& alter_locality_op)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
  if (new_schema.get_locality_str().empty()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid tablegroup locality", K(ret));
 | 
						|
  } else if (new_schema.get_locality_str() != orig_schema.get_previous_locality_str()) {
 | 
						|
    MODIFY_LOCALITY_NOT_ALLOWED();
 | 
						|
    LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter tablegroup locality while previous locality change in progress");
 | 
						|
  } else {
 | 
						|
    if (arg.alter_option_bitset_.has_member(obrpc::ObAlterTablegroupArg::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::try_alter_table_locality(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, const share::schema::ObTableSchema& orig_table_schema,
 | 
						|
    share::schema::ObTableSchema& new_table_schema, const ObIArray<share::schema::ObZoneRegion>& zone_region_list,
 | 
						|
    const obrpc::ObAlterTableArg& alter_table_arg, AlterLocalityOp& alter_locality_op)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(alter_table_arg);
 | 
						|
  bool is_completed_match = true;
 | 
						|
  alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
  if (OB_FAIL(ObLocalityUtil::check_locality_completed_match(
 | 
						|
          schema_guard, new_table_schema, tenant_schema, is_completed_match))) {
 | 
						|
    LOG_WARN("fail to check table tenant locality completed match", K(ret), K(new_table_schema), K(tenant_schema));
 | 
						|
  } else if (is_completed_match) {
 | 
						|
    if (orig_table_schema.get_locality_str().empty()) {
 | 
						|
      new_table_schema.reset_locality_options();
 | 
						|
    } else {
 | 
						|
    }  // do nothing
 | 
						|
  } else {
 | 
						|
    bool is_match = false;
 | 
						|
    const bool just_check_zone_list =
 | 
						|
        (new_table_schema.get_duplicate_scope() == share::ObDuplicateScope::DUPLICATE_SCOPE_CLUSTER);
 | 
						|
    if (OB_FAIL(check_paxos_locality_match(
 | 
						|
            just_check_zone_list, schema_guard, new_table_schema, tenant_schema, zone_region_list, is_match))) {
 | 
						|
      LOG_WARN("table tenant locality not match", K(ret));
 | 
						|
    } else if (!is_match) {
 | 
						|
      MODIFY_LOCALITY_NOT_ALLOWED();
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "table and tenant locality not match");
 | 
						|
      LOG_WARN("table and tenant locality not match",
 | 
						|
          K(ret),
 | 
						|
          "tenant locality",
 | 
						|
          tenant_schema.get_locality_str(),
 | 
						|
          "new table locality",
 | 
						|
          new_table_schema.get_locality_str());
 | 
						|
    } else {
 | 
						|
    }  // ok
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (new_table_schema.get_locality_str() == orig_table_schema.get_locality_str()) {
 | 
						|
    // The locality string has been standardized, and you can determine whether it matches by comparing directly
 | 
						|
    // The new and old locality is the same, no need to deal with
 | 
						|
    alter_locality_op = NOP_LOCALITY_OP;
 | 
						|
  } else {
 | 
						|
    if (orig_table_schema.get_locality_str().empty()) {
 | 
						|
      // The locality changes from empty to non-empty, and previous is set to the tenant's locality
 | 
						|
      new_table_schema.set_previous_locality(tenant_schema.get_locality_str());
 | 
						|
    } else {
 | 
						|
      // The old locality is not empty, use it to fill the previous locality
 | 
						|
      new_table_schema.set_previous_locality(orig_table_schema.get_locality_str());
 | 
						|
    }
 | 
						|
    alter_locality_op = ALTER_LOCALITY;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::try_alter_tablegroup_locality(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, const share::schema::ObTablegroupSchema& orig_tablegroup,
 | 
						|
    share::schema::ObTablegroupSchema& new_tablegroup, const ObIArray<share::schema::ObZoneRegion>& zone_region_list,
 | 
						|
    AlterLocalityOp& alter_locality_op)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  bool is_completed_match = true;
 | 
						|
  alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
  if (OB_FAIL(ObLocalityUtil::check_locality_completed_match(
 | 
						|
          schema_guard, new_tablegroup, tenant_schema, is_completed_match))) {
 | 
						|
    LOG_WARN("fail to check table tenant locality completed match", K(ret), K(new_tablegroup), K(tenant_schema));
 | 
						|
  } else if (is_completed_match) {
 | 
						|
    if (orig_tablegroup.get_locality_str().empty()) {
 | 
						|
      new_tablegroup.reset_locality_options();
 | 
						|
    } else {
 | 
						|
    }  // do nothing
 | 
						|
  } else {
 | 
						|
    bool is_match = false;
 | 
						|
    const bool just_check_zone_list = false;
 | 
						|
    if (OB_FAIL(check_paxos_locality_match(
 | 
						|
            just_check_zone_list, schema_guard, new_tablegroup, tenant_schema, zone_region_list, is_match))) {
 | 
						|
      LOG_WARN("table tenant locality not match", K(ret));
 | 
						|
    } else if (!is_match) {
 | 
						|
      MODIFY_LOCALITY_NOT_ALLOWED();
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "table and tenant locality not match");
 | 
						|
      LOG_WARN("table and tenant locality not match",
 | 
						|
          K(ret),
 | 
						|
          "tenant locality",
 | 
						|
          tenant_schema.get_locality_str(),
 | 
						|
          "new table locality",
 | 
						|
          new_tablegroup.get_locality_str());
 | 
						|
    } else {
 | 
						|
    }  // ok
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (new_tablegroup.get_locality_str() == orig_tablegroup.get_locality_str()) {
 | 
						|
    // The locality string has been standardized,
 | 
						|
    // and you can determine whether it matches directly by comparing it
 | 
						|
    // The new and old locality is the same, no need to deal with
 | 
						|
    alter_locality_op = NOP_LOCALITY_OP;
 | 
						|
  } else {
 | 
						|
    if (orig_tablegroup.get_locality_str().empty()) {
 | 
						|
      // The locality changes from empty to non-empty, and previous is set to the tenant's locality
 | 
						|
      new_tablegroup.set_previous_locality(tenant_schema.get_locality_str());
 | 
						|
    } else {
 | 
						|
      // The old locality is not empty, fill the previous locality with the old one
 | 
						|
      // If the old locality is not empty, use it to fill the previous locality
 | 
						|
      new_tablegroup.set_previous_locality(orig_tablegroup.get_locality_str());
 | 
						|
    }
 | 
						|
    alter_locality_op = ALTER_LOCALITY;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
int ObDDLService::check_tablegroup_locality_with_tenant(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, const share::schema::ObTablegroupSchema& tablegroup)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObTablegroupSchema new_tablegroup;
 | 
						|
  if (OB_FAIL(new_tablegroup.assign(tablegroup))) {
 | 
						|
    LOG_WARN("fail to assign tablegroup", K(ret), K(tablegroup));
 | 
						|
  } else if (OB_FAIL(check_locality_with_tenant(schema_guard, tenant_schema, new_tablegroup))) {
 | 
						|
    LOG_WARN("fail to check locality with tenant", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_table_locality_with_tenant(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, const share::schema::ObSimpleTableSchemaV2& table)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSimpleTableSchemaV2 new_table;
 | 
						|
  if (OB_FAIL(new_table.assign(table))) {
 | 
						|
    LOG_WARN("fail to assign table", K(ret), K(table));
 | 
						|
  } else if (OB_FAIL(check_locality_with_tenant(schema_guard, tenant_schema, new_table))) {
 | 
						|
    LOG_WARN("fail to check locality with tenant", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
template <typename T>
 | 
						|
int ObDDLService::check_locality_with_tenant(
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const share::schema::ObTenantSchema& tenant_schema, T& schema)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  if (schema.get_locality_str().empty()) {
 | 
						|
    // do nothing
 | 
						|
  } else {
 | 
						|
    common::ObArray<common::ObZone> zone_list;
 | 
						|
    common::ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
    bool is_completed_match = false;
 | 
						|
    common::ObArray<share::ObUnitInfo> unit_infos;
 | 
						|
    // Whether it is create_table or alter_table, the current zone list of the tenant should be get
 | 
						|
    // instead of the zone list of the tenant resource pool, which is more strict
 | 
						|
    if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
      RS_LOG(WARN, "fail to get tenant zone list", K(ret));
 | 
						|
    } else if (OB_FAIL(construct_zone_region_list(zone_region_list, zone_list))) {
 | 
						|
      RS_LOG(WARN, "fail to construct zone region list", K(ret));
 | 
						|
    } else if (OB_FAIL(parse_and_set_new_locality_options(
 | 
						|
                   schema_guard, tenant_schema.get_tenant_id(), schema, zone_list, zone_region_list))) {
 | 
						|
      RS_LOG(WARN, "fail to parse and set new locality option", K(ret));
 | 
						|
    } else if (OB_FAIL(ObLocalityUtil::check_locality_completed_match(
 | 
						|
                   schema_guard, schema, tenant_schema, is_completed_match))) {
 | 
						|
      RS_LOG(WARN, "fail to check table tenant locality completed match", K(ret), K(schema), K(tenant_schema));
 | 
						|
    } else if (is_completed_match) {
 | 
						|
      // do nothing
 | 
						|
    } else {
 | 
						|
      bool is_match = false;
 | 
						|
      bool just_check_zone_list = false;
 | 
						|
      if (OB_FAIL(check_paxos_locality_match(
 | 
						|
              just_check_zone_list, schema_guard, schema, tenant_schema, zone_region_list, is_match))) {
 | 
						|
        RS_LOG(WARN, "tenant locality not match with schema", K(ret));
 | 
						|
      } else if (!is_match) {
 | 
						|
        ret = common::OB_INVALID_ARGUMENT;
 | 
						|
        RS_LOG(WARN, "invalid locality", K(ret));
 | 
						|
        LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality");
 | 
						|
      } else {
 | 
						|
        common::ObArray<share::ObResourcePoolName> pool_names;
 | 
						|
        if (OB_UNLIKELY(NULL == unit_mgr_)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          RS_LOG(WARN, "unit_mgr_ is null", K(ret), KP(unit_mgr_));
 | 
						|
        } else if (OB_FAIL(unit_mgr_->get_pool_names_of_tenant(schema.get_tenant_id(), pool_names))) {
 | 
						|
          RS_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, schema))) {
 | 
						|
          RS_LOG(WARN, "pool unit num is not enough for locality", K(ret));
 | 
						|
        } else {
 | 
						|
        }  // no more to do
 | 
						|
      }    // no more to do, ok
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
/*
 | 
						|
 * Check and resolve locality
 | 
						|
 * 1. If the locality of the table/tablegroup schema is empty, it will be returned directly,
 | 
						|
 *    and the locality of the table will be inherited from its tenant.
 | 
						|
 * 2. The locality of the table/tablegroup schema is not empty,
 | 
						|
 *    and it is completely matched with the locality of the tenant,
 | 
						|
 *    that is all types of copies are consistent (FULL, LOGONLY, READONLY).
 | 
						|
 *    At this time, the locality of the table/tablegroup schema needs to be cleared,
 | 
						|
 *    and the locality of the table will be inherited from its tenant.
 | 
						|
 * 3. The locality of the table/tablegroup schema is not empty,
 | 
						|
 *    and the matching conditions with the tenant locality are met,
 | 
						|
 *    the locality of the table/tablegroup schema is retained,
 | 
						|
 *    and the locality of the subsequent table uses this original value.
 | 
						|
 */
 | 
						|
template <typename T>
 | 
						|
int ObDDLService::try_set_and_check_locality_with_tenant(const bool just_check_zone_list,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const share::schema::ObTenantSchema& tenant_schema, T& schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (schema.get_locality_str().empty()) {  // create_table not specify locality
 | 
						|
    schema.reset_locality_options();
 | 
						|
  } else {
 | 
						|
    bool is_completed_match = false;
 | 
						|
    ObArray<ObUnitInfo> unit_infos;
 | 
						|
    ObArray<ObZone> zone_list;
 | 
						|
    ObArray<ObZoneRegion> zone_region_list;
 | 
						|
    // Whether it is create_table or alter_table, the current zone list of the tenant should be obtained
 | 
						|
    // instead of the zone list of the tenant resource pool, which is more strict
 | 
						|
    if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
      LOG_WARN("fail to get tenant zone list", K(ret));
 | 
						|
    } else 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(parse_and_set_new_locality_options(
 | 
						|
                   schema_guard, tenant_schema.get_tenant_id(), schema, zone_list, zone_region_list))) {
 | 
						|
      LOG_WARN("fail to parse and set new locality option", K(ret));
 | 
						|
    } else if (OB_FAIL(ObLocalityUtil::check_locality_completed_match(
 | 
						|
                   schema_guard, schema, tenant_schema, is_completed_match))) {
 | 
						|
      LOG_WARN("fail to check table tenant locality completed match", K(ret), K(schema), K(tenant_schema));
 | 
						|
    } else if (is_completed_match) {
 | 
						|
      // If it matches exactly, the locality of the table schema is cleared,
 | 
						|
      // which means it is exactly the same as the locality of the tenant
 | 
						|
      schema.reset_locality_options();
 | 
						|
    } else {
 | 
						|
      bool is_match = false;
 | 
						|
      if (OB_FAIL(check_paxos_locality_match(
 | 
						|
              just_check_zone_list, schema_guard, schema, tenant_schema, zone_region_list, is_match))) {
 | 
						|
        LOG_WARN("table tenant locality not match", K(ret));
 | 
						|
      } else if (!is_match) {
 | 
						|
        ret = OB_INVALID_ARGUMENT;
 | 
						|
        LOG_WARN("invalid table locality", K(ret));
 | 
						|
        LOG_USER_ERROR(OB_INVALID_ARGUMENT, "table locality");
 | 
						|
      } else {
 | 
						|
      }  // no more to do, ok
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::set_locality_with_tablegroup(
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, share::schema::ObTableSchema& schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tablegroup_id = schema.get_tablegroup_id();
 | 
						|
  const ObTablegroupSchema* tablegroup_schema = NULL;
 | 
						|
  common::ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
  common::ObArray<ObZone> zone_list;
 | 
						|
  bool extend_empty_locality = false;
 | 
						|
  if (OB_FAIL(schema_guard.get_tablegroup_schema(tablegroup_id, tablegroup_schema))) {
 | 
						|
    LOG_WARN("fail to get tablegroup schema", K(ret), K(tablegroup_id));
 | 
						|
  } else if (OB_ISNULL(tablegroup_schema)) {
 | 
						|
    ret = OB_TABLEGROUP_NOT_EXIST;
 | 
						|
    LOG_WARN("tablegroup not exist", K(ret));
 | 
						|
  } else if (OB_FAIL(tablegroup_schema->get_zone_list(schema_guard, zone_list))) {
 | 
						|
    LOG_WARN("fail to get tenant zone list", K(ret));
 | 
						|
  } else 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(parse_and_set_new_locality_options(
 | 
						|
                 schema_guard, schema.get_tenant_id(), schema, zone_list, zone_region_list, extend_empty_locality))) {
 | 
						|
    LOG_WARN("fail to parse and set new table locality options", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
template <typename T>
 | 
						|
int ObDDLService::check_paxos_locality_match(const bool just_check_zone_list,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const T& schema,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& zone_region_list, bool& match)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(just_check_zone_list);
 | 
						|
  UNUSED(zone_region_list);
 | 
						|
  /*
 | 
						|
  ObArray<share::ObZoneReplicaNumSet> zone_locality;
 | 
						|
  bool zone_match = false;
 | 
						|
  if (OB_FAIL(check_zone_list_with_tenant(schema_guard, schema, tenant_schema, zone_match))) {
 | 
						|
    LOG_WARN("fail to check table tenant zone list match", K(ret));
 | 
						|
  } else if (!zone_match) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("table tenant locality not match", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality");
 | 
						|
  } else if (just_check_zone_list) {
 | 
						|
    // bypass, need not check locality, only check zone list
 | 
						|
    match = true;
 | 
						|
  } 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(do_check_locality_in_tenant(
 | 
						|
          schema_guard, zone_locality, tenant_schema, zone_region_list, match))) {
 | 
						|
    LOG_WARN("fail to do check table tenant locality match", K(ret));
 | 
						|
  } else {} // no more to do
 | 
						|
  */
 | 
						|
  /* To support China Life's business, Support featrue the locality of
 | 
						|
   * table/tablegroup-level and tenant-level locality mismatch
 | 
						|
   * As long as the zone list of table/tablegroup is a subset of
 | 
						|
   * the tenant zone list
 | 
						|
   */
 | 
						|
  bool zone_list_legal = false;
 | 
						|
  if (OB_FAIL(check_zone_list_legal_with_tenant(schema_guard, schema, tenant_schema, zone_list_legal))) {
 | 
						|
    LOG_WARN("fail to check zone list legal", K(ret), K(schema));
 | 
						|
  } else if (!zone_list_legal) {
 | 
						|
    match = false;
 | 
						|
  } else {
 | 
						|
    match = true;
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
template <typename SCHEMA>
 | 
						|
int ObDDLService::check_zone_list_legal_with_tenant(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const SCHEMA& schema, const share::schema::ObTenantSchema& tenant_schema, bool& zone_list_legal)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  zone_list_legal = true;
 | 
						|
  ObArray<common::ObZone> table_zone_list;
 | 
						|
  ObArray<common::ObZone> tenant_zone_list;
 | 
						|
  if (OB_FAIL(schema.get_zone_list(schema_guard, table_zone_list))) {
 | 
						|
    LOG_WARN("fail to get table zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(tenant_schema.get_zone_list(tenant_zone_list))) {
 | 
						|
    LOG_WARN("fail to get tenant zone list", K(ret));
 | 
						|
  } else {
 | 
						|
    for (int64_t i = 0; zone_list_legal && i < table_zone_list.count(); ++i) {
 | 
						|
      if (!has_exist_in_array(tenant_zone_list, table_zone_list.at(i))) {
 | 
						|
        zone_list_legal = false;
 | 
						|
      } else {
 | 
						|
      }  // no more to do
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
template <typename SCHEMA>
 | 
						|
int ObDDLService::check_zone_list_with_tenant(share::schema::ObSchemaGetterGuard& schema_guard, const SCHEMA& schema,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, bool& zone_match)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  zone_match = true;
 | 
						|
  ObArray<common::ObZone> table_zone_list;
 | 
						|
  ObArray<common::ObZone> tenant_zone_list;
 | 
						|
  if (OB_FAIL(schema.get_zone_list(schema_guard, table_zone_list))) {
 | 
						|
    LOG_WARN("fail to get table zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(tenant_schema.get_zone_list(tenant_zone_list))) {
 | 
						|
    LOG_WARN("fail to get tenant zone list", K(ret));
 | 
						|
  } else if (tenant_zone_list.count() != table_zone_list.count()) {
 | 
						|
    zone_match = false;
 | 
						|
  } else {
 | 
						|
    std::sort(table_zone_list.begin(), table_zone_list.end());
 | 
						|
    std::sort(tenant_zone_list.begin(), tenant_zone_list.end());
 | 
						|
    for (int64_t i = 0; zone_match && i < table_zone_list.count(); ++i) {
 | 
						|
      if (table_zone_list.at(i) != tenant_zone_list.at(i)) {
 | 
						|
        zone_match = false;
 | 
						|
      } else {
 | 
						|
      }  // no more to do
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// This function is called when the tenant's locality change
 | 
						|
// The outer call ensures that the locality of the simple tablegroup schema here must not be empty
 | 
						|
int ObDDLService::check_tablegroup_tenant_locality_match(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObSimpleTablegroupSchema& tablegroup_schema,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, bool& match)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  bool zone_list_legal = false;
 | 
						|
  if (OB_FAIL(check_zone_list_legal_with_tenant(schema_guard, tablegroup_schema, tenant_schema, zone_list_legal))) {
 | 
						|
    LOG_WARN("fail to check zone list legal", K(ret), K(tablegroup_schema));
 | 
						|
  } else if (!zone_list_legal) {
 | 
						|
    match = false;
 | 
						|
  } else {
 | 
						|
    match = true;
 | 
						|
  }
 | 
						|
  /*
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  common::ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
  common::ObArray<share::ObZoneReplicaAttrSet> zone_locality;
 | 
						|
  rootserver::ObLocalityDistribution locality_dist;
 | 
						|
  bool zone_match = false;
 | 
						|
  if (OB_FAIL(check_zone_list_with_tenant(
 | 
						|
          schema_guard, tablegroup_schema, tenant_schema, zone_match))) {
 | 
						|
    RS_LOG(WARN, "fail to check tenant zone list", K(ret));
 | 
						|
  } else if (!zone_match) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    RS_LOG(WARN, "tablegroup tenant locality not match", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality");
 | 
						|
  } else if (OB_FAIL(tablegroup_schema.get_zone_list(schema_guard, zone_list))) {
 | 
						|
    RS_LOG(WARN, "fail to get zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(construct_zone_region_list(zone_region_list, zone_list))) {
 | 
						|
    RS_LOG(WARN, "fail to construct zone region list", K(ret));
 | 
						|
  } else if (OB_FAIL(locality_dist.init())) {
 | 
						|
    RS_LOG(WARN, "fail to init locality distribution", K(ret));
 | 
						|
  } else if (OB_FAIL(locality_dist.parse_locality(
 | 
						|
          tablegroup_schema.get_locality_str(), zone_list,
 | 
						|
          &zone_region_list))) {
 | 
						|
    RS_LOG(WARN, "fail to parse locality", K(ret));
 | 
						|
  } else if (OB_FAIL(locality_dist.get_zone_replica_attr_array(zone_locality))) {
 | 
						|
    RS_LOG(WARN, "fail to get zone region replica num array", K(ret));
 | 
						|
  } else if (OB_FAIL(do_check_locality_in_tenant(
 | 
						|
          schema_guard, zone_locality, tenant_schema, zone_region_list, match))) {
 | 
						|
    RS_LOG(WARN, "fail to do check tablegroup tenant locality match", K(ret));
 | 
						|
  } else {} // no more to do
 | 
						|
  */
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// This function is called when the tenant's locality change
 | 
						|
// The outer call ensures that the locality of the simple tablegroup schema here must not be empty
 | 
						|
int ObDDLService::check_table_tenant_locality_match(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObSimpleTableSchemaV2& table_schema, const share::schema::ObTenantSchema& tenant_schema,
 | 
						|
    bool& match)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  bool zone_list_legal = false;
 | 
						|
  if (OB_FAIL(check_zone_list_legal_with_tenant(schema_guard, table_schema, tenant_schema, zone_list_legal))) {
 | 
						|
    LOG_WARN("fail to check zone list legal", K(ret), K(table_schema));
 | 
						|
  } else if (!zone_list_legal) {
 | 
						|
    match = false;
 | 
						|
  } else {
 | 
						|
    match = true;
 | 
						|
  }
 | 
						|
  /*
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
  ObArray<share::ObZoneReplicaAttrSet> zone_locality;
 | 
						|
  ObLocalityDistribution locality_dist;
 | 
						|
  bool zone_match = false;
 | 
						|
  if (OB_FAIL(check_zone_list_with_tenant(
 | 
						|
          schema_guard, table_schema, tenant_schema, zone_match))) {
 | 
						|
    LOG_WARN("fail to check tenant zone list", K(ret));
 | 
						|
  } else if (!zone_match) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("table tenant locality not match", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality");
 | 
						|
  } else if (ObDuplicateScope::DUPLICATE_SCOPE_CLUSTER == table_schema.get_duplicate_scope()) {
 | 
						|
    // The copy table does not check whether the locality matches
 | 
						|
    match = true;
 | 
						|
  } else if (OB_FAIL(table_schema.get_zone_list(zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else 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(locality_dist.init())) {
 | 
						|
    LOG_WARN("fail to init locality distribution", K(ret));
 | 
						|
  } else if (OB_FAIL(locality_dist.parse_locality(
 | 
						|
          table_schema.get_locality_str(), zone_list,
 | 
						|
          &zone_region_list))) {
 | 
						|
    LOG_WARN("fail to parse locality", K(ret));
 | 
						|
  } else if (OB_FAIL(locality_dist.get_zone_replica_attr_array(zone_locality))) {
 | 
						|
    LOG_WARN("fail to get zone region replica num array", K(ret));
 | 
						|
  } else if (OB_FAIL(do_check_locality_in_tenant(
 | 
						|
          schema_guard, zone_locality, tenant_schema, zone_region_list, match))) {
 | 
						|
    LOG_WARN("fail to do check table tenant locality match", K(ret));
 | 
						|
  } else {} // no more to do
 | 
						|
  */
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::do_check_locality_in_tenant(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const common::ObIArray<share::ObZoneReplicaAttrSet>& zone_locality,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& zone_region_list, bool& match)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(zone_region_list);
 | 
						|
  const common::ObIArray<share::ObZoneReplicaAttrSet>& left_zone_locality = zone_locality;
 | 
						|
  common::ObArray<share::ObZoneReplicaAttrSet> right_zone_locality;
 | 
						|
  if (OB_FAIL(tenant_schema.get_zone_replica_attr_array_inherit(schema_guard, right_zone_locality))) {
 | 
						|
    LOG_WARN("fail to get zone replica attr array inherit", K(ret));
 | 
						|
  } else if (left_zone_locality.count() != right_zone_locality.count()) {
 | 
						|
    match = false;
 | 
						|
  } else {
 | 
						|
    match = true;
 | 
						|
    for (int64_t i = 0; match && i < left_zone_locality.count(); ++i) {
 | 
						|
      match = (left_zone_locality.at(i).replica_attr_set_.is_paxos_replica_match(
 | 
						|
          right_zone_locality.at(i).replica_attr_set_));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// The number of elements in the current zone_region_list is very small, directly traverse to obtain
 | 
						|
int ObDDLService::get_zones_from_zone_region_list(const common::ObRegion& region,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& zone_region_list,
 | 
						|
    common::ObIArray<common::ObZone>& zones_in_region) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  for (int64_t i = 0; i < zone_region_list.count() && OB_SUCC(ret); ++i) {
 | 
						|
    if (region == zone_region_list.at(i).region_) {
 | 
						|
      if (OB_FAIL(zones_in_region.push_back(zone_region_list.at(i).zone_))) {
 | 
						|
        LOG_WARN("fail to push back", K(ret));
 | 
						|
      } else {
 | 
						|
      }  // no more to do
 | 
						|
    } else {
 | 
						|
    }  // region do not match, go on find
 | 
						|
  }
 | 
						|
  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_table_id(), table_schema_version, is_table_id_exist))) {
 | 
						|
      LOG_WARN("failed to check is table exist", 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(pt_operator_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("server_mgr_ or pt_operator_ is null", K(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::check_table_schema_not_in_new_tablegroup(
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const ObTableSchema& table_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tablegroup_id = table_schema.get_tablegroup_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_INVALID_ID == tablegroup_id || is_new_tablegroup_id(tablegroup_id)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid tablegroup_id", K(ret), K(tablegroup_id));
 | 
						|
  } else {
 | 
						|
    // Tablegroups created before 2.0 cannot directly compare the attributes of tablegroups,
 | 
						|
    // take sample_table for comparison
 | 
						|
    ObArray<const ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
    const ObSimpleTableSchemaV2* sample_table = NULL;
 | 
						|
    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 (NULL == tenant_schema) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("fail to get tenant schema", K(ret), K(table_schema));
 | 
						|
    } 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("get_table_ids_in_tablegroup failed",
 | 
						|
          K(table_schema.get_tenant_id()),
 | 
						|
          K(table_schema.get_tablegroup_id()),
 | 
						|
          K(ret));
 | 
						|
    } 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, return OB_SUCCESS directly
 | 
						|
    } else {
 | 
						|
      // check primary_zone
 | 
						|
      bool primary_zone_match = false;
 | 
						|
      if (OB_FAIL(check_primary_zone_match(schema_guard, table_schema, *sample_table, primary_zone_match))) {
 | 
						|
        LOG_WARN("fail to check primary zone", K(ret));
 | 
						|
      } else if (!primary_zone_match) {
 | 
						|
        ret = OB_DIFFERENT_PRIMARY_ZONE;
 | 
						|
        LOG_WARN("primary_zones in same tablegroup differ",
 | 
						|
            K(ret),
 | 
						|
            K(sample_table->get_primary_zone()),
 | 
						|
            K(sample_table->get_table_id()),
 | 
						|
            K(sample_table->get_table_name()),
 | 
						|
            K(table_schema.get_primary_zone()));
 | 
						|
      } else {
 | 
						|
      }  // good, primary zone match
 | 
						|
 | 
						|
      // check locality
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (table_schema.has_partition()) {
 | 
						|
        if (NULL != table_schema.get_locality_str().ptr()) {
 | 
						|
          bool locality_match = false;
 | 
						|
          if (OB_FAIL(check_schemas_locality_match(table_schema, *sample_table, schema_guard, locality_match))) {
 | 
						|
            LOG_WARN("fail to check locality", K(ret));
 | 
						|
          } else if (!locality_match) {
 | 
						|
            ret = OB_INVALID_ARGUMENT;
 | 
						|
            LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality, locality within same tablegroup should be the same");
 | 
						|
            LOG_WARN("locality within same tablegroup should be the same",
 | 
						|
                K(ret),
 | 
						|
                K(sample_table->get_locality_str()),
 | 
						|
                K(sample_table->get_table_id()),
 | 
						|
                K(sample_table->get_table_name()),
 | 
						|
                K(table_schema.get_locality_str()));
 | 
						|
          } else {
 | 
						|
          }  // good, locality match
 | 
						|
        } else {
 | 
						|
          // Try to get the locality of tenant for comparing with sample_table
 | 
						|
          bool locality_match = false;
 | 
						|
          if (OB_FAIL(check_schemas_locality_match(*tenant_schema, *sample_table, schema_guard, locality_match))) {
 | 
						|
            LOG_WARN("fail to check locality", K(ret));
 | 
						|
          } else if (!locality_match) {
 | 
						|
            ret = OB_INVALID_ARGUMENT;
 | 
						|
            LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality, locality within same tablegroup should be the same");
 | 
						|
            LOG_WARN("locality within same tablegroup should be the same",
 | 
						|
                K(ret),
 | 
						|
                K(sample_table->get_locality_str()),
 | 
						|
                K(sample_table->get_table_id()),
 | 
						|
                K(sample_table->get_table_name()),
 | 
						|
                K(table_schema.get_locality_str()));
 | 
						|
          } else {
 | 
						|
          }  // good, locality match
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
      }  // locality null, no need to check
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::get_sample_table_schema(
 | 
						|
    common::ObIArray<const ObSimpleTableSchemaV2*>& 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 != extract_pure_id(table_schemas.at(i)->get_table_id())) {
 | 
						|
      sample_table_schema = table_schemas.at(i);
 | 
						|
    } else {
 | 
						|
    }  // go on find
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// In order to be compatible with the primary zone from version 1.3 and earlier,
 | 
						|
// we normalize the primary zone of the sample table with ObPrimaryZoneUtil,
 | 
						|
// and then compare it with the primary zone of new_table
 | 
						|
// for example:
 | 
						|
//     Before upgrading to 1.4, there is a table (sample_table) in tablegroup tg1
 | 
						|
//     The zone_list is zone1, zone2, and zone3 are all in sys_region, and the primary_zone is zone1
 | 
						|
//
 | 
						|
//     After upgrading to 1.4, the primary_zone="zone1" of new_table of the input
 | 
						|
//     will be normalized to "zone1;zone2,zone3"
 | 
						|
//     Comparing the primary zone of sample and new directly by byte will be considered as not match.
 | 
						|
//     Therefore, first normalize the primary zone of the sample table,
 | 
						|
//     and then compare it with the new table.
 | 
						|
//     Compare new_table and sample_table directly,
 | 
						|
//     and will not look up according to the inheritance relationship
 | 
						|
template <typename SCHEMA1, typename SCHEMA2>
 | 
						|
int ObDDLService::check_primary_zone_match(
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, SCHEMA1& new_table, SCHEMA2& sample_table, bool& match)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  common::ObArray<ObZoneRegion> zone_region_list;
 | 
						|
  if (new_table.get_primary_zone().empty()) {
 | 
						|
    match = sample_table.get_primary_zone().empty();
 | 
						|
  } else if (0 == new_table.get_primary_zone().compare(OB_RANDOM_PRIMARY_ZONE)) {
 | 
						|
    match = (0 == new_table.get_primary_zone().compare(OB_RANDOM_PRIMARY_ZONE));
 | 
						|
  } else if (OB_FAIL(sample_table.get_zone_list(schema_guard, zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(construct_zone_region_list(zone_region_list, zone_list))) {
 | 
						|
    LOG_WARN("fail to build zone region list", K(ret));
 | 
						|
  } else {
 | 
						|
    char primary_zone_str[MAX_ZONE_LENGTH + 1];
 | 
						|
    int64_t pos = 0;
 | 
						|
    ObPrimaryZoneUtil primary_zone_util(sample_table.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())) {
 | 
						|
      LOG_WARN("fail to check and parse primary zone", K(ret));
 | 
						|
    } 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 {
 | 
						|
      match = (new_table.get_primary_zone() == primary_zone_str);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
template <typename SCHEMA1, typename SCHEMA2>
 | 
						|
int ObDDLService::check_schemas_locality_match(
 | 
						|
    SCHEMA1& left, SCHEMA2& right, share::schema::ObSchemaGetterGuard& schema_guard, bool& match)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZoneReplicaAttrSet> left_zone_locality;
 | 
						|
  ObArray<ObZoneReplicaAttrSet> right_zone_locality;
 | 
						|
  if (OB_FAIL(left.get_zone_replica_attr_array_inherit(schema_guard, left_zone_locality))) {
 | 
						|
    LOG_WARN("fail to get left zone replica num array", K(ret));
 | 
						|
  } else if (OB_FAIL(right.get_zone_replica_attr_array_inherit(schema_guard, right_zone_locality))) {
 | 
						|
    LOG_WARN("fail to get right zone replica num array", K(ret));
 | 
						|
  } else {
 | 
						|
    match = is_tables_locality_match(left_zone_locality, right_zone_locality);
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObDDLService::is_tables_locality_match(const common::ObIArray<share::ObZoneReplicaAttrSet>& left,
 | 
						|
    const common::ObIArray<share::ObZoneReplicaAttrSet>& right)
 | 
						|
{
 | 
						|
  bool match = true;
 | 
						|
  if (left.count() != right.count()) {
 | 
						|
    match = false;
 | 
						|
  } else {
 | 
						|
    for (int64_t i = 0; i < left.count() && match; ++i) {
 | 
						|
      match = is_locality_replica_attr_match(left.at(i).replica_attr_set_, right.at(i).replica_attr_set_) &&
 | 
						|
              left.at(i).is_zone_set_match(right.at(i));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return match;
 | 
						|
}
 | 
						|
 | 
						|
bool ObDDLService::is_locality_replica_attr_match(const ObReplicaAttrSet& left, const ObReplicaAttrSet& right)
 | 
						|
{
 | 
						|
  return (ObLocalityDistribution::ALL_SERVER_CNT != left.get_readonly_replica_num() &&
 | 
						|
          ObLocalityDistribution::ALL_SERVER_CNT != right.get_readonly_replica_num() && left == right);
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_locality_and_primary_zone_complete_match(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTableSchema& orig_table_schema, const share::schema::ObTablegroupSchema& tablegroup_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchema src_schema;
 | 
						|
  ObSchema dst_schema;
 | 
						|
  ObPrimaryZone src_primary_zone(&src_schema);
 | 
						|
  ObPrimaryZone dst_primary_zone(&dst_schema);
 | 
						|
  ObArray<share::ObZoneReplicaAttrSet> src_zone_locality;
 | 
						|
  ObArray<share::ObZoneReplicaAttrSet> dst_zone_locality;
 | 
						|
  // There is still an unnormalized primary_zone_str in the old cluster, which cannot be directly compared.
 | 
						|
  // The primary_zone_array will be normalized when the schema is refreshed,
 | 
						|
  // and it can be taken according to the inheritance relationship.
 | 
						|
  if (OB_FAIL(orig_table_schema.get_primary_zone_inherit(schema_guard, src_primary_zone))) {
 | 
						|
    LOG_WARN("fail to get src primary_zone_array_inherit", K(ret));
 | 
						|
  } else if (OB_FAIL(tablegroup_schema.get_primary_zone_inherit(schema_guard, dst_primary_zone))) {
 | 
						|
    LOG_WARN("fail to get dst primary_zone_array_inherit", K(ret));
 | 
						|
  } else if (!ObPrimaryZoneUtil::check_primary_zone_equal(src_primary_zone, dst_primary_zone)) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("primary_zone not match", K(ret), K(src_primary_zone), K(dst_primary_zone));
 | 
						|
    LOG_USER_ERROR(OB_OP_NOT_ALLOW, "primary_zone differs from tablegroup(or database)");
 | 
						|
  } else if (OB_FAIL(orig_table_schema.get_zone_replica_attr_array_inherit(schema_guard, src_zone_locality))) {
 | 
						|
    LOG_WARN("fail to get zone_replica_num_array_inherit", K(ret));
 | 
						|
  } else if (OB_FAIL(tablegroup_schema.get_zone_replica_attr_array_inherit(schema_guard, dst_zone_locality))) {
 | 
						|
    LOG_WARN("fail to get zone_replica_num_array_inherit", K(ret));
 | 
						|
  } else {
 | 
						|
    std::sort(src_zone_locality.begin(), src_zone_locality.end());
 | 
						|
    std::sort(dst_zone_locality.begin(), dst_zone_locality.end());
 | 
						|
    bool match = is_tables_locality_match(src_zone_locality, dst_zone_locality);
 | 
						|
    if (!match) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("locality not  match", K(ret), K(src_zone_locality), K(dst_zone_locality));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "locality differs from tablegroup/tenant");
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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(table_schema.get_database_id(), db_schema))) {
 | 
						|
      LOG_WARN("fail to get database schema", K(ret), "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), "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());
 | 
						|
    }
 | 
						|
  }
 | 
						|
  // set table binding attribute according to tablegroup
 | 
						|
  if (OB_SUCC(ret) && OB_INVALID_ID != table_schema.get_tablegroup_id() && table_schema.has_partition()) {
 | 
						|
    // If there are tablegroups and there are partitions, check whether the tablegroup is binding
 | 
						|
    const share::schema::ObTablegroupSchema* tg_schema = nullptr;
 | 
						|
    if (OB_FAIL(schema_guard.get_tablegroup_schema(table_schema.get_tablegroup_id(), tg_schema))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schema", K(ret), "tg_id", table_schema.get_tablegroup_id());
 | 
						|
    } else if (OB_UNLIKELY(nullptr == tg_schema)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("tg schema ptr is null", "tg_id", table_schema.get_tablegroup_id());
 | 
						|
    } else if (tg_schema->get_binding()) {
 | 
						|
      table_schema.set_binding(true);
 | 
						|
    } else {
 | 
						|
      table_schema.set_binding(false);
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  // 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 <typename SCHEMA>
 | 
						|
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::try_modify_tenant_primary_zone_entity_count(common::ObMySQLTransaction& trans,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const bool is_inc, const int64_t num, const uint64_t tenant_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const ObSysVariableSchema* sys_variable_schema = nullptr;
 | 
						|
  const ObSysVarSchema* old_pz_schema = nullptr;
 | 
						|
  ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
 | 
						|
  const common::ObString pz_var_name(share::OB_SV__PRIMARY_ZONE_ENTITY_COUNT);
 | 
						|
  common::ObObj pz_obj;
 | 
						|
  int64_t pz_value = -1;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("check inner stat failed");
 | 
						|
  } else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(tenant_id));
 | 
						|
  } else if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2220) {
 | 
						|
    // do nothing
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) {
 | 
						|
    LOG_WARN("fail to get sys variable schema", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == sys_variable_schema)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else {
 | 
						|
    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();
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      int64_t schema_version = OB_INVALID_VERSION;
 | 
						|
      const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR;
 | 
						|
      const int64_t MAX_BUF_SIZE = 64;
 | 
						|
      char buf[MAX_BUF_SIZE] = "";
 | 
						|
      ObSysVarSchema new_schema;
 | 
						|
      ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
      if (OB_FAIL(sys_variable_schema->get_sysvar_schema(pz_var_name, old_pz_schema))) {
 | 
						|
        LOG_WARN("fail to get sysvar schema", K(ret));
 | 
						|
      } else if (OB_UNLIKELY(nullptr == old_pz_schema)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("primary zone count schema ptr is null", K(ret));
 | 
						|
      } else if (OB_FAIL(old_pz_schema->get_value(&allocator, nullptr /*time zone info*/, pz_obj))) {
 | 
						|
        LOG_WARN("fail to get value", K(ret));
 | 
						|
      } else if (OB_FAIL(pz_obj.get_int(pz_value))) {
 | 
						|
        LOG_WARN("fail to get int from pz obj", K(ret));
 | 
						|
      } else if (is_inc && pz_value < 0) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("pz_value unexpected", K(ret), K(pz_value));
 | 
						|
      } else if (!is_inc && pz_value < num) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("pz_value unexpected", K(ret), K(pz_value));
 | 
						|
      } else if (0 > (snprintf(buf, MAX_BUF_SIZE, "%ld", is_inc ? pz_value + num : pz_value - num))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("fail to print data to buf", K(ret));
 | 
						|
      } else if (FALSE_IT(new_schema = *old_pz_schema)) {
 | 
						|
        // shall never be here
 | 
						|
      } else if (OB_UNLIKELY(!new_schema.is_valid())) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("new schema is invalid", K(new_schema));
 | 
						|
      } else if (OB_FAIL(new_schema.set_value(ObString(buf)))) {
 | 
						|
        LOG_WARN("fail to set new schema", K(ret));
 | 
						|
      } else if (OB_FAIL(new_sys_variable_schema.add_sysvar_schema(new_schema))) {
 | 
						|
        LOG_WARN("fail to add sysvar schema to new sys variable schema");
 | 
						|
      } else if (OB_UNLIKELY(nullptr == schema_service_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("schema service ptr is null", K(ret));
 | 
						|
      } 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))) {
 | 
						|
        LOG_WARN("fail to replace sys variable", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
  int64_t database_id = table_schema.get_database_id();
 | 
						|
 | 
						|
  if (OB_ISNULL(buf = static_cast<char*>(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(database_id, database_schema))) {
 | 
						|
    LOG_WARN("failed to get database schema", K(ret), 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(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
 | 
						|
                 table_schema.get_tenant_id(), 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<int32_t>(pos));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_tables_in_trans(const bool if_not_exist, const ObString& ddl_stmt_str,
 | 
						|
    ObIArray<ObTableSchema>& table_schemas, const int64_t frozen_version, obrpc::ObCreateTableMode create_mode,
 | 
						|
    const uint64_t last_replay_log_id)
 | 
						|
{
 | 
						|
  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 {
 | 
						|
    bool need_process_failed = false;
 | 
						|
    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();
 | 
						|
    const share::schema::ObTableSchema& first_table = table_schemas.at(0);
 | 
						|
    if (is_inner_table(table_schemas.at(0).get_table_id())) {
 | 
						|
      trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
    }
 | 
						|
    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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans, ", K(ret));
 | 
						|
    } else if (first_table.get_primary_zone().empty()) {
 | 
						|
      // bypass, since the primary zone is empty, derive from upper level,
 | 
						|
      // no need to statistic primary zone count
 | 
						|
    } else {
 | 
						|
      if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(trans, schema_guard, true /*inc*/, 1, tenant_id))) {
 | 
						|
        LOG_WARN("fail to try inc tenant primary zone entity count", K(ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    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));
 | 
						|
          }
 | 
						|
          if (OB_ISNULL(old_view_schema)) {
 | 
						|
            ret = OB_SUCCESS;
 | 
						|
          } 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++) {
 | 
						|
        if (OB_FAIL(ddl_operator.create_table(
 | 
						|
                table_schemas.at(i), 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_schemas.at(i)))) {
 | 
						|
          LOG_WARN("failed to insert_temp_table_info!", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      // Create a new indexed system table needs to write a schema
 | 
						|
      if (OB_SUCC(ret) && OB_ALL_TABLE_V2_HISTORY_TID == extract_pure_id(first_table.get_table_id())) {
 | 
						|
        ObArray<ObTableSchema> schemas;
 | 
						|
        if (OB_FAIL(add_sys_table_index(first_table.get_table_id(), schemas))) {
 | 
						|
          LOG_WARN("fail to add sys table index", K(ret), "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));
 | 
						|
        }
 | 
						|
      }
 | 
						|
 | 
						|
      // write schema for new create sys table
 | 
						|
      if (OB_SUCC(ret) && OB_ALL_BACKUP_PIECE_FILES_TID == extract_pure_id(first_table.get_table_id())) {
 | 
						|
        ObArray<ObTableSchema> schemas;
 | 
						|
        if (OB_FAIL(add_sys_table_index(first_table.get_table_id(), schemas))) {
 | 
						|
          LOG_WARN("fail to add sys table index", K(ret), "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)) {
 | 
						|
        const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version();
 | 
						|
        bool gts_on = false;
 | 
						|
        int64_t cur_ts_type = 0;
 | 
						|
        if (OB_INVALID_VERSION == last_schema_version) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("invalid last schema version", K(ret));
 | 
						|
        } else if (OB_FAIL(schema_guard.get_timestamp_service_type(tenant_id, cur_ts_type))) {
 | 
						|
          LOG_WARN("fail to get cur ts type", K(ret));
 | 
						|
        } else {
 | 
						|
          gts_on = transaction::is_ts_type_external_consistent(cur_ts_type);
 | 
						|
        }
 | 
						|
        for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
 | 
						|
          const share::schema::ObTableSchema& this_table = table_schemas.at(i);
 | 
						|
          uint64_t table_id = this_table.get_table_id();
 | 
						|
          bool has_partition = false;
 | 
						|
          if (!is_inner_table(table_id)) {
 | 
						|
            has_partition = this_table.has_partition();
 | 
						|
          } else {
 | 
						|
            has_partition = is_inner_table_with_partition(table_id);
 | 
						|
          }
 | 
						|
          if (!has_partition) {
 | 
						|
          } else if (this_table.is_global_index_table() && !gts_on) {
 | 
						|
            ret = OB_NOT_SUPPORTED;
 | 
						|
            LOG_USER_ERROR(OB_NOT_SUPPORTED, "create global index when gts off");
 | 
						|
          } else if (OB_CREATE_TABLE_MODE_RESTORE == create_mode) {
 | 
						|
            // skip
 | 
						|
          } else if (OB_FAIL(create_or_bind_tables_partitions(schema_guard,
 | 
						|
                         this_table,
 | 
						|
                         last_schema_version,
 | 
						|
                         create_mode,
 | 
						|
                         frozen_version,
 | 
						|
                         table_schemas,
 | 
						|
                         last_replay_log_id))) {
 | 
						|
            need_process_failed = true;
 | 
						|
            LOG_WARN("fail to create table partitions", K(ret));
 | 
						|
          } else {
 | 
						|
            need_process_failed = true;
 | 
						|
          }
 | 
						|
          if (OB_SUCC(ret)) {
 | 
						|
            if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans, table_id, last_schema_version))) {
 | 
						|
              LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(table_id), K(last_schema_version));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      RS_TRACE(operator_create_table_end);
 | 
						|
    }
 | 
						|
    DEBUG_SYNC(BEFORE_CREATE_TABLE_TRANS_COMMIT);
 | 
						|
    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;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret) && need_process_failed) {
 | 
						|
      // push schema version
 | 
						|
      int tmp_ret = OB_SUCCESS;
 | 
						|
      if (OB_SUCCESS != (tmp_ret = process_create_partition_failed(tenant_id))) {
 | 
						|
        LOG_ERROR("create table failed, may has garbage partition", K(ret), K(tmp_ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    if (OB_SUCC(ret) && is_standby && is_sys_table(first_table.get_table_id()) && first_table.has_self_partition()) {
 | 
						|
      // clear all info of member list
 | 
						|
      int tmp_ret = OB_SUCCESS;
 | 
						|
      // don't care whether clear success
 | 
						|
      const int64_t max_schema_version = ddl_operator.get_last_operation_schema_version();
 | 
						|
      const bool is_inner_table = true;
 | 
						|
      if (OB_SUCCESS != (tmp_ret = clear_partition_member_list(max_schema_version, tenant_id, is_inner_table))) {
 | 
						|
        LOG_WARN("failed to clear persist member list",
 | 
						|
            KR(ret),
 | 
						|
            KR(tmp_ret),
 | 
						|
            K(max_schema_version),
 | 
						|
            K(tenant_id),
 | 
						|
            K(is_inner_table),
 | 
						|
            K(first_table));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  RS_TRACE(create_tables_in_trans_end);
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_or_bind_tables_partitions(ObSchemaGetterGuard& schema_guard, const ObTableSchema& table_schema,
 | 
						|
    const int64_t last_schema_version, obrpc::ObCreateTableMode create_mode, const int64_t frozen_version,
 | 
						|
    const ObIArray<ObTableSchema>& table_schemas, const uint64_t last_replay_log_id /*= 0*/)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  int64_t start_usec = 0;
 | 
						|
  int64_t end_usec = 0;
 | 
						|
  int64_t cost_usec = 0;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (table_schemas.count() <= 0) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("table schemas cnt invalid", K(ret));
 | 
						|
  } else if (OB_INVALID_VERSION == last_schema_version) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("invalid schema version", K(last_schema_version));
 | 
						|
  } else {
 | 
						|
    PartitionBornMethod part_born_method = PartitionBornMethod::PBM_MAX;
 | 
						|
    const share::schema::ObTablegroupSchema* tg_schema = NULL;
 | 
						|
    if (OB_INVALID_ID == table_schema.get_tablegroup_id()) {
 | 
						|
      part_born_method = PartitionBornMethod::PBM_DIRECTLY_CREATE;  // standalone, need create
 | 
						|
    } else if (OB_FAIL(schema_guard.get_tablegroup_schema(table_schema.get_tablegroup_id(), tg_schema))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schema", K(ret), "tg_id", table_schema.get_tablegroup_id());
 | 
						|
    } else if (OB_UNLIKELY(OB_ISNULL(tg_schema))) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("tg_schema ptr is null", K(ret), "tg_id", table_schema.get_tablegroup_id());
 | 
						|
    } else if (tg_schema->get_binding()) {
 | 
						|
      part_born_method = PartitionBornMethod::PBM_BINDING;
 | 
						|
    } else {
 | 
						|
      part_born_method = PartitionBornMethod::PBM_DIRECTLY_CREATE;
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      start_usec = ObTimeUtility::current_time();
 | 
						|
      if (PartitionBornMethod::PBM_DIRECTLY_CREATE == part_born_method) {
 | 
						|
        if (OB_FAIL(create_table_partitions(
 | 
						|
                table_schema, last_schema_version, create_mode, frozen_version, table_schemas, last_replay_log_id))) {
 | 
						|
          LOG_WARN(
 | 
						|
              "create table partitions failed", K(ret), K(table_schema), K(last_schema_version), K(frozen_version));
 | 
						|
        }
 | 
						|
      } else if (PartitionBornMethod::PBM_BINDING == part_born_method) {
 | 
						|
        if (OB_FAIL(
 | 
						|
                binding_table_partitions(table_schema, tg_schema, last_schema_version, create_mode, table_schemas))) {
 | 
						|
          LOG_WARN(
 | 
						|
              "binding table partitions failed", K(ret), K(table_schema), K(last_schema_version), K(frozen_version));
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("unexpected partition born method", K(ret), K(part_born_method));
 | 
						|
      }
 | 
						|
      end_usec = ObTimeUtility::current_time();
 | 
						|
      cost_usec = end_usec - start_usec;
 | 
						|
      start_usec = end_usec;
 | 
						|
      LOG_INFO("create_table_partitions cost: ", K(cost_usec));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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 obrpc::ObCreateTableMode create_mode, const ObString* ddl_stmt_str, ObMySQLTransaction* sql_trans)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  bool need_process_failed = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else {
 | 
						|
    ObDDLSQLTransaction tmp_trans(schema_service_);
 | 
						|
    ObMySQLTransaction& trans = OB_ISNULL(sql_trans) ? tmp_trans : *sql_trans;
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    if (OB_ISNULL(sql_trans) && OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans, ", KR(ret));
 | 
						|
    } 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, table_schema.get_table_id(), last_schema_version))) {
 | 
						|
        LOG_WARN("failed to insert_ori_schema_version!", KR(ret), K(last_schema_version));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret) && table_schema.has_partition() && OB_CREATE_TABLE_MODE_RESTORE != create_mode) {
 | 
						|
      ObArray<ObTableSchema> schemas;
 | 
						|
      if (OB_FAIL(schemas.push_back(table_schema))) {
 | 
						|
        LOG_WARN("fail to push back table schema", KR(ret));
 | 
						|
      } else if (OB_FAIL(create_table_partitions(
 | 
						|
                     table_schema, last_schema_version, create_mode, frozen_version, schemas))) {
 | 
						|
        need_process_failed = true;
 | 
						|
        LOG_WARN("create table partitions failed", KR(ret), K(table_schema), K(last_schema_version), K(frozen_version));
 | 
						|
      } else {
 | 
						|
        need_process_failed = true;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    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;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret) && need_process_failed) {
 | 
						|
      // push schema version
 | 
						|
      int tmp_ret = OB_SUCCESS;
 | 
						|
      if (OB_SUCCESS != (tmp_ret = process_create_partition_failed(table_schema.get_tenant_id()))) {
 | 
						|
        LOG_ERROR("failed to create partition, may has garbage partition", K(ret), K(tmp_ret), K(table_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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<const ObSimpleTableSchemaV2*> 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 (alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::LOCALITY)) {
 | 
						|
      // After 2.0, the table in the tablegroup will not enter here,
 | 
						|
      // and the relevant code does not need to be changed
 | 
						|
      alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
      common::ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
      common::ObArray<ObZone> zone_list;
 | 
						|
      AlterLocalityType alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
      bool tenant_pools_in_shrinking = false;
 | 
						|
      bool extend_empty_locality = false;
 | 
						|
      if (orig_table_schema.is_sys_table() || !orig_table_schema.has_self_partition()) {
 | 
						|
        ret = OB_OP_NOT_ALLOW;
 | 
						|
        LOG_USER_ERROR(OB_OP_NOT_ALLOW, "change system table or non-partitioned table's locality");
 | 
						|
        LOG_WARN("cannot change system table or non-partitioned table's locality", 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_table_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_table_schema));
 | 
						|
      } else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
        LOG_WARN("fail to get tenant zone list", K(ret));
 | 
						|
      } else 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(parse_and_set_new_locality_options(schema_guard,
 | 
						|
                     orig_table_schema.get_tenant_id(),
 | 
						|
                     new_table_schema,
 | 
						|
                     zone_list,
 | 
						|
                     zone_region_list,
 | 
						|
                     extend_empty_locality))) {
 | 
						|
        LOG_WARN("fail to parse and set new table locality options", K(ret));
 | 
						|
      } else if (OB_FAIL(check_alter_table_locality_type(
 | 
						|
                     schema_guard, orig_table_schema, new_table_schema, alter_locality_type))) {
 | 
						|
        LOG_WARN("fail to check alter tenant locality type", K(ret));
 | 
						|
      } else if (ALTER_LOCALITY_INVALID == alter_locality_type) {
 | 
						|
        ret = OB_OP_NOT_ALLOW;
 | 
						|
        LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter table locality while previous operation is in progress");
 | 
						|
        LOG_WARN("alter table locality not allowed", K(ret), K(orig_table_schema));
 | 
						|
      } else if (ROLLBACK_LOCALITY == alter_locality_type) {
 | 
						|
        if (OB_FAIL(try_rollback_alter_table_locality(alter_table_arg,
 | 
						|
                schema_guard,
 | 
						|
                tenant_schema,
 | 
						|
                new_table_schema,
 | 
						|
                orig_table_schema,
 | 
						|
                zone_region_list,
 | 
						|
                alter_table_arg,
 | 
						|
                alter_locality_op))) {
 | 
						|
          LOG_WARN("fail to try rollback alter table locality", K(ret), K(new_table_schema), K(orig_table_schema));
 | 
						|
        } else {
 | 
						|
        }  // no more to do
 | 
						|
      } else if (TO_NEW_LOCALITY == alter_locality_type) {
 | 
						|
        if (OB_FAIL(try_alter_table_locality(schema_guard,
 | 
						|
                tenant_schema,
 | 
						|
                orig_table_schema,
 | 
						|
                new_table_schema,
 | 
						|
                zone_region_list,
 | 
						|
                alter_table_arg,
 | 
						|
                alter_locality_op))) {
 | 
						|
          LOG_WARN("fail to try alter table locality", K(ret));
 | 
						|
        } else {
 | 
						|
        }  // no more to do
 | 
						|
      } else if (LOCALITY_NOT_CHANGED == alter_locality_type) {
 | 
						|
        alter_locality_op = NOP_LOCALITY_OP;
 | 
						|
        LOG_INFO("set locality to default", K(ret));
 | 
						|
      } else {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("invalid alter locality type", K(ret), K(alter_locality_type));
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        common::ObArray<share::ObResourcePoolName> pool_names;
 | 
						|
        if (new_table_schema.get_locality_str().empty()) {
 | 
						|
          // derived from tenant locality, no need to check
 | 
						|
        } 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_pool_names_of_tenant(new_table_schema.get_tenant_id(), pool_names))) {
 | 
						|
          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_table_schema))) {
 | 
						|
          LOG_WARN("pool unit num is not enough for locality", K(ret));
 | 
						|
        } else {
 | 
						|
        }  // no more to do
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // check primary_zone
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(
 | 
						|
              check_alter_table_replica_options(alter_table_arg, new_table_schema, orig_table_schema, schema_guard))) {
 | 
						|
        LOG_WARN("check replica options failed", K(new_table_schema), K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    bool alter_primary_zone = alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::PRIMARY_ZONE);
 | 
						|
    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, alter_primary_zone))) {
 | 
						|
        LOG_WARN("fail to check table schema in tablegroup", K(ret));
 | 
						|
      } else {
 | 
						|
      }  // good
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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: {
 | 
						|
          ret = new_table_schema.set_primary_zone(alter_table_schema.get_primary_zone());
 | 
						|
          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;
 | 
						|
          ObWorker::CompatMode compat_mode = ObWorker::CompatMode::INVALID;
 | 
						|
          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(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_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
 | 
						|
            if (ObWorker::CompatMode::ORACLE == compat_mode) {
 | 
						|
              is_oracle_mode = true;
 | 
						|
            } else {
 | 
						|
              is_oracle_mode = false;
 | 
						|
            }
 | 
						|
            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);
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
          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 {
 | 
						|
            new_table_schema.set_locality(alter_table_schema.get_locality_str());
 | 
						|
          }
 | 
						|
          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;
 | 
						|
        }
 | 
						|
        default: {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("Unknown option!", K(i));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::generate_single_deployment_tenant_sys_replica_num(
 | 
						|
    const common::ObIArray<share::ObResourcePoolName>& pool_names, int64_t& replica_num)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<share::ObResourcePool> pools;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == unit_mgr_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("unit mgr is null", K(ret));
 | 
						|
  } else if (pool_names.count() <= 0) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret));
 | 
						|
  } else {
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < pool_names.count(); ++i) {
 | 
						|
      const share::ObResourcePoolName& pool_name = pool_names.at(i);
 | 
						|
      share::ObResourcePool this_pool;
 | 
						|
      if (OB_FAIL(unit_mgr_->get_resource_pool_by_name(pool_name, this_pool))) {
 | 
						|
        LOG_WARN("fail to get resource pool by name", K(ret), K(pool_name));
 | 
						|
      } else if (OB_FAIL(pools.push_back(this_pool))) {
 | 
						|
        LOG_WARN("fail to push back", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
      // failed
 | 
						|
    } else if (pools.count() <= 0) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("pools count unexpected", K(ret));
 | 
						|
    } else if (pools.at(0).zone_list_.count() != 1) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("zone list count unexpected", K(ret));
 | 
						|
    } else {
 | 
						|
      replica_num = 0;
 | 
						|
      const common::ObZone& sample_zone = pools.at(0).zone_list_.at(0);
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < pools.count(); ++i) {
 | 
						|
        share::ObResourcePool& this_pool = pools.at(i);
 | 
						|
        if (this_pool.zone_list_.count() != 1) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("zone list count unexpected", K(ret));
 | 
						|
        } else if (this_pool.zone_list_.at(0) != sample_zone) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("pools come from different zone",
 | 
						|
              K(ret),
 | 
						|
              "left_zone",
 | 
						|
              this_pool.zone_list_.at(0),
 | 
						|
              "right_zone",
 | 
						|
              sample_zone);
 | 
						|
        } else {
 | 
						|
          replica_num += this_pool.unit_count_;
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        replica_num = std::min(replica_num, SINGLE_ZONE_DEPLOYMENT_TENANT_SYS_QUORUM);
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::parse_and_set_create_tenant_new_locality_options(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    ObTenantSchema& schema, const common::ObIArray<share::ObResourcePoolName>& pools,
 | 
						|
    const common::ObIArray<common::ObZone>& zone_list,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& zone_region_list)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  char locality_str[MAX_LOCALITY_LENGTH + 1];
 | 
						|
  int64_t pos = 0;
 | 
						|
  ObLocalityDistribution locality_dist;
 | 
						|
  ObArray<ObUnitInfo> 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 <typename T>
 | 
						|
int ObDDLService::parse_and_set_new_locality_options(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const uint64_t tenant_id, T& schema, const common::ObIArray<common::ObZone>& zone_list,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& zone_region_list, bool extend_empty_locality /*=true*/)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  char locality_str[MAX_LOCALITY_LENGTH + 1];
 | 
						|
  int64_t pos = 0;
 | 
						|
  ObLocalityDistribution locality_dist;
 | 
						|
  const ObTenantSchema* tenant_schema = NULL;
 | 
						|
  ObArray<ObUnitInfo> unit_infos;
 | 
						|
  if (schema.get_locality_str().empty() && !extend_empty_locality) {
 | 
						|
    // pass
 | 
						|
  } else 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(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
 | 
						|
    LOG_WARN("fail to get tenant info", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_ISNULL(tenant_schema)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("get invalid tenant schema", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(unit_mgr_->get_active_unit_infos_by_tenant(*tenant_schema, unit_infos))) {
 | 
						|
    LOG_WARN("fail to get unit infos", K(ret), K(tenant_id));
 | 
						|
  } 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(zone_list),
 | 
						|
      K(zone_region_list),
 | 
						|
      K(unit_infos));
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
template <typename T>
 | 
						|
int ObDDLService::set_schema_zone_list(share::schema::ObSchemaGetterGuard& schema_guard, T& schema,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& zone_region_list)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  common::ObArray<share::ObZoneReplicaAttrSet> 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;
 | 
						|
}
 | 
						|
 | 
						|
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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObIndexBuilder index_builder(*this);
 | 
						|
  const ObSArray<ObIndexArg*>& index_arg_list = alter_table_arg.index_arg_list_;
 | 
						|
  AddIndexNameHashSet add_index_name_set;
 | 
						|
  DropIndexNameHashSet drop_index_name_set;
 | 
						|
  RenameIndexNameHashSet rename_ori_index_name_set;
 | 
						|
  RenameIndexNameHashSet rename_new_index_name_set;
 | 
						|
  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<ObIndexArg*>(index_arg_list.at(i));
 | 
						|
    if (OB_ISNULL(index_arg)) {
 | 
						|
      ret = OB_INVALID_ARGUMENT;
 | 
						|
      LOG_WARN("index arg should not be null", K(ret));
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      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<ObCreateIndexArg*>(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_no_pk_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)) {
 | 
						|
          ObTableSchema index_schema;
 | 
						|
          ObArray<ObColumnSchemaV2*> 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.
 | 
						|
            int64_t cur_ts_type = 0;
 | 
						|
            if (OB_FAIL(index_schema.assign(create_index_arg->index_schema_))) {
 | 
						|
              LOG_WARN("fail to assign schema", K(ret));
 | 
						|
            } else if (OB_FAIL(
 | 
						|
                           schema_guard.get_timestamp_service_type(origin_table_schema.get_tenant_id(), cur_ts_type))) {
 | 
						|
              LOG_WARN("fail to get cur ts type", K(ret));
 | 
						|
            } else if (!transaction::is_ts_type_external_consistent(cur_ts_type)) {
 | 
						|
              ret = OB_NOT_SUPPORTED;
 | 
						|
              LOG_WARN("create global index when GTS is off is not supported", K(ret));
 | 
						|
              LOG_USER_ERROR(OB_NOT_SUPPORTED, "create global index when GTS is off");
 | 
						|
            } else if (OB_FAIL(index_schema.set_locality(new_table_schema.get_locality()))) {
 | 
						|
              LOG_WARN("fail to set index_schema locality", K(ret));
 | 
						|
            } else if (OB_FAIL(index_schema.set_primary_zone(new_table_schema.get_primary_zone()))) {
 | 
						|
              LOG_WARN("fail to set index_schema primary_zone", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
          const bool global_index_without_column_info = true;
 | 
						|
          if (OB_FAIL(ret)) {
 | 
						|
          } else if (OB_FAIL(ObIndexBuilderUtil::adjust_expr_index_args(
 | 
						|
                         *create_index_arg, new_table_schema, 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 global index of the main table, and the partition needs to
 | 
						|
            // be built, and the outer insert ori_schema_version
 | 
						|
            if (global_storage && index_schema.has_self_partition()) {
 | 
						|
              int64_t start_usec = ObTimeUtility::current_time();
 | 
						|
              const int64_t last_schema_version = index_schema.get_schema_version();
 | 
						|
              ObArray<ObTableSchema> schemas;
 | 
						|
              if (OB_CREATE_TABLE_MODE_RESTORE == alter_table_arg.create_mode_) {
 | 
						|
                ret = OB_NOT_SUPPORTED;
 | 
						|
                LOG_WARN("alter table add index while tenant restore not support", K(ret));
 | 
						|
              } else if (OB_INVALID_VERSION == last_schema_version) {
 | 
						|
                ret = OB_ERR_UNEXPECTED;
 | 
						|
                LOG_WARN("invalid schema version", K(last_schema_version));
 | 
						|
              } else if (OB_FAIL(schemas.push_back(index_schema))) {
 | 
						|
                LOG_WARN("fail to push back table schema", K(ret));
 | 
						|
              } else if (OB_FAIL(create_table_partitions(index_schema,
 | 
						|
                             last_schema_version,
 | 
						|
                             alter_table_arg.create_mode_,
 | 
						|
                             frozen_version,
 | 
						|
                             schemas))) {
 | 
						|
                LOG_WARN("create table partitions failed",
 | 
						|
                    K(index_schema),
 | 
						|
                    K(last_schema_version),
 | 
						|
                    K(frozen_version),
 | 
						|
                    K(ret));
 | 
						|
              }
 | 
						|
              int64_t cost_usec = ObTimeUtility::current_time() - start_usec;
 | 
						|
              LOG_INFO("create table partitions cost: ", K(cost_usec));
 | 
						|
            }
 | 
						|
            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<ObDropIndexArg*>(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)) {
 | 
						|
          if (OB_FAIL(ddl_operator.alter_table_drop_index(origin_table_schema.get_table_id(),
 | 
						|
                  origin_table_schema.get_database_id(),
 | 
						|
                  *drop_index_arg,
 | 
						|
                  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<ObAlterIndexArg*>(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_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<ObDropForeignKeyArg*>(index_arg);
 | 
						|
        if (OB_FAIL(ddl_operator.alter_table_drop_foreign_key(origin_table_schema, *drop_foreign_key_arg, trans))) {
 | 
						|
          LOG_WARN("failed to alter table drop foreign key", K(ret), K(drop_foreign_key_arg));
 | 
						|
        }
 | 
						|
      } else if (ObIndexArg::ALTER_INDEX_PARALLEL == index_arg->index_action_type_) {
 | 
						|
        ObAlterIndexParallelArg* alter_index_parallel_arg = static_cast<ObAlterIndexParallelArg*>(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 (ddl_operator.alter_index_table_parallel(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<ObRenameIndexArg*>(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 if (OB_FAIL(ddl_operator.alter_table_rename_index(origin_table_schema.get_table_id(),
 | 
						|
                       origin_table_schema.get_database_id(),
 | 
						|
                       *rename_index_arg,
 | 
						|
                       trans))) {
 | 
						|
          LOG_WARN("failed to rename index", K(*rename_index_arg), K(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 {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("Unknown index action type!", K_(index_arg->index_action_type), K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::alter_table_foreign_keys(const share::schema::ObTableSchema& orig_table_schema,
 | 
						|
    share::schema::AlterTableSchema& inc_table_schema, share::schema::ObTableSchema& new_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, new_table_schema, trans))) {
 | 
						|
    LOG_WARN("failed to add table foreign_keys", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::alter_table_constraints(const ObAlterTableArg::AlterConstraintType op_type,
 | 
						|
    const ObTableSchema& orig_table_schema, AlterTableSchema& inc_table_schema, ObTableSchema& new_table_schema,
 | 
						|
    ObDDLOperator& ddl_operator, ObMySQLTransaction& trans)
 | 
						|
{
 | 
						|
  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();
 | 
						|
      for (ObTableSchema::const_constraint_iterator iter_r = iter + 1;
 | 
						|
           OB_SUCC(ret) && iter_r != inc_table_schema.constraint_end();
 | 
						|
           iter_r++) {
 | 
						|
        if ((*iter_r)->get_constraint_name_str() == cst_name) {
 | 
						|
          if (is_oracle_mode) {
 | 
						|
            ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
 | 
						|
          } else {
 | 
						|
            ret = OB_ERR_CONSTRAINT_DUPLICATE;
 | 
						|
          }
 | 
						|
          LOG_WARN("duplicate constraint name", K(ret), K(cst_name));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        ObConstraint* const* 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(); });
 | 
						|
        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) {
 | 
						|
            if (is_oracle_mode) {
 | 
						|
              ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
 | 
						|
            } else {
 | 
						|
              ret = OB_ERR_CONSTRAINT_DUPLICATE;
 | 
						|
            }
 | 
						|
            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<ObConstraint*>(*iter)->set_constraint_id((*res)->get_constraint_id());
 | 
						|
              const_cast<ObConstraint*>(*iter)->set_constraint_type(CONSTRAINT_TYPE_CHECK);
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (obrpc::ObAlterTableArg::ADD_CONSTRAINT == op_type) {
 | 
						|
      if (OB_FAIL(ddl_operator.add_table_constraints(orig_table_schema, 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(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(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::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) {
 | 
						|
    ObSubPartIteratorV2 sub_iter(orig_table_schema, *part, false);
 | 
						|
    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 {
 | 
						|
    ObPartIteratorV2 iter(orig_table_schema, false);
 | 
						|
    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 {
 | 
						|
        ObSubPartIteratorV2 sub_iter(orig_table_schema, *part, false);
 | 
						|
        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_drop_subpart(
 | 
						|
    const ObTableSchema& orig_table_schema, AlterTableSchema& inc_table_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObPartition* inc_part = NULL;
 | 
						|
  if (orig_table_schema.is_sub_part_template()) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("drop subpartition is not supported for nontemplate ", KR(ret));
 | 
						|
  } else 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::get_part_by_id(AlterTableSchema& table_schema, const int64_t part_id, const ObPartition*& partition)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObPartition** partition_array = table_schema.get_part_array();
 | 
						|
  int64_t partition_num = table_schema.get_partition_num();
 | 
						|
  int64_t partition_index = OB_INVALID_INDEX;
 | 
						|
  partition = NULL;
 | 
						|
  if (OB_INVALID_ID == part_id) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", KR(ret), K(part_id));
 | 
						|
  } else if (partition_num < 0) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("partition num is invalid", KR(ret), K(part_id), K(partition_num));
 | 
						|
  } else if (OB_ISNULL(partition_array)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("invalid partition array", KR(ret));
 | 
						|
  } else if (OB_FAIL(table_schema.get_partition_index_loop(part_id, false, partition_index))) {
 | 
						|
    if (OB_ENTRY_NOT_EXIST == ret) {
 | 
						|
      ret = OB_SUCCESS;
 | 
						|
      LOG_TRACE("partition not exist", KR(ret), K(part_id));
 | 
						|
    } else {
 | 
						|
      LOG_WARN("failed to get partition index by id", KR(ret), K(part_id));
 | 
						|
    }
 | 
						|
  } else if (partition_index < partition_num) {
 | 
						|
    partition = partition_array[partition_index];
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_NOT_NULL(partition) && partition->get_part_id() != part_id) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("invalid partition", KR(ret), KPC(partition), K(part_id));
 | 
						|
  } else {
 | 
						|
    // partition maybe null
 | 
						|
  }
 | 
						|
  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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObPartition* inc_part_p = NULL;
 | 
						|
  ObPartition inc_part_set;
 | 
						|
  if (orig_table_schema.is_sub_part_template()) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("drop subpartition is not supported for nontemplate ", KR(ret));
 | 
						|
  } else 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_for_trun_subpart();
 | 
						|
    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_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_max_used_sub_part_id(orig_part->get_max_used_sub_part_id());
 | 
						|
        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<ObPartition*>(inc_part)->add_partition(*orig_subpart))) {
 | 
						|
        LOG_WARN("failed to add partition", K(orig_subpart), KR(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  inc_table_schema.set_part_num(inc_table_schema.get_partition_num());
 | 
						|
  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<ObAuxTableMetaInfo, 16> 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 {
 | 
						|
      int64_t delay_deleted_global_index_count = 0;
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
 | 
						|
        bool is_delay_delete = false;
 | 
						|
        const ObTableSchema* index_table_schema = NULL;
 | 
						|
        if (OB_FAIL(schema_guard.get_table_schema(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_, 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(index_table_schema->get_data_table_id(),
 | 
						|
                    index_table_schema->get_table_id(),
 | 
						|
                    INDEX_STATUS_UNUSABLE,
 | 
						|
                    index_table_schema->get_create_mem_version(),
 | 
						|
                    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,
 | 
						|
                           arg.create_mode_,
 | 
						|
                           &trans,
 | 
						|
                           &is_delay_delete))) {
 | 
						|
              LOG_WARN("ddl_service_ rebuild_index failed", KR(ret));
 | 
						|
            } else {
 | 
						|
              ObSArray<obrpc::ObIndexArg*>& 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));
 | 
						|
                } else if (is_delay_delete) {
 | 
						|
                  delay_deleted_global_index_count++;
 | 
						|
                }
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      // In the case of delayed deletion, it is necessary to determine
 | 
						|
      // whether the sum of rebuilt indexex exceeds the maximum index of the data table.
 | 
						|
      if (OB_SUCC(ret) && 0 < delay_deleted_global_index_count &&
 | 
						|
          orig_table_schema.get_index_tid_count() + delay_deleted_global_index_count > OB_MAX_INDEX_PER_TABLE) {
 | 
						|
        ret = OB_ERR_TOO_MANY_KEYS;
 | 
						|
        LOG_USER_ERROR(OB_ERR_TOO_MANY_KEYS, OB_MAX_INDEX_PER_TABLE);
 | 
						|
        int64_t index_count = orig_table_schema.get_index_tid_count();
 | 
						|
        LOG_WARN("too many index for table", K(OB_MAX_INDEX_PER_TABLE), K(index_count), K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::alter_table_partitions(const ObAlterTableArg::AlterPartitionType op_type,
 | 
						|
    const ObTableSchema& orig_table_schema, AlterTableSchema& inc_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();
 | 
						|
  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::ADD_PARTITION == op_type) {
 | 
						|
    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 (orig_table_schema.is_sub_part_template()) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("drop subpartition is not supported for nontemplate ", 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::PARTITIONED_TABLE == op_type) {
 | 
						|
    ObSplitInfo split_info;
 | 
						|
    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 if (!orig_table_schema.is_sub_part_template()) {
 | 
						|
      ret = OB_NOT_SUPPORTED;
 | 
						|
      LOG_WARN("nontemplate cannot split", KR(ret));
 | 
						|
      LOG_USER_ERROR(OB_NOT_SUPPORTED, "nontemplate cannot split");
 | 
						|
    } else {
 | 
						|
      bool is_alter_tablegroup = false;
 | 
						|
      if (OB_FAIL(ObPartitionSplitHelper::fill_split_info(
 | 
						|
              orig_table_schema, inc_table_schema, new_table_schema, op_type, schema_version, split_info, false))) {
 | 
						|
        LOG_WARN("fail to fill split info", KR(ret));
 | 
						|
      } else if (1 == new_table_schema.get_partition_num() && orig_table_schema.get_binding()) {
 | 
						|
        // Split into only one partition which is binding,
 | 
						|
        // it is the operation of the partition key of the specified table under pg
 | 
						|
        ObPartition* part = new_table_schema.get_part_array()[0];
 | 
						|
        if (OB_ISNULL(part)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("part is null", KR(ret), K(new_table_schema), K(orig_table_schema));
 | 
						|
        } else {
 | 
						|
          // The default is 0, the tablegroup has not been split
 | 
						|
          part->set_mapping_pg_part_id(0);
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(ddl_operator.split_table_partitions(
 | 
						|
                     new_table_schema, orig_table_schema, inc_table_schema, split_info, is_alter_tablegroup, trans))) {
 | 
						|
        LOG_WARN("failed to add table partitions", KR(ret), K(new_table_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (obrpc::ObAlterTableArg::DROP_PARTITION == op_type) {
 | 
						|
    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];
 | 
						|
      bool check_dropped_schema = false;
 | 
						|
      ObPartIteratorV2 iter(orig_table_schema, check_dropped_schema);
 | 
						|
      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));
 | 
						|
          }
 | 
						|
          break;
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_ITER_END == ret) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("part should exists", KR(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      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 (orig_table_schema.is_sub_part_template()) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("drop subpartition is not supported for nontemplate ", KR(ret));
 | 
						|
    } else 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 subpartition_name to subpartition", KR(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::REORGANIZE_PARTITION == op_type ||
 | 
						|
             obrpc::ObAlterTableArg::SPLIT_PARTITION == op_type) {
 | 
						|
    ObSplitInfo split_info;
 | 
						|
    new_table_schema.reset_partition_schema();
 | 
						|
    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 {
 | 
						|
      bool is_alter_tablegroup = false;
 | 
						|
      if (orig_table_schema.is_no_pk_table()) {
 | 
						|
        ret = OB_NOT_SUPPORTED;
 | 
						|
        LOG_WARN("table has no primary key, refuse to split", KR(ret), "table_id", orig_table_schema.get_table_id());
 | 
						|
        LOG_USER_ERROR(OB_NOT_SUPPORTED, "split table or partition without primary key");
 | 
						|
      } else if (OB_FAIL(ObPartitionSplitHelper::fill_split_info(orig_table_schema,
 | 
						|
                     inc_table_schema,
 | 
						|
                     new_table_schema,
 | 
						|
                     op_type,
 | 
						|
                     schema_version,
 | 
						|
                     split_info,
 | 
						|
                     false))) {
 | 
						|
        LOG_WARN("fail to fill split info", KR(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.split_table_partitions(
 | 
						|
                     new_table_schema, orig_table_schema, inc_table_schema, split_info, is_alter_tablegroup, trans))) {
 | 
						|
        LOG_WARN("failed to add table partitions", KR(ret), K(new_table_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == op_type) {
 | 
						|
    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();
 | 
						|
    if (OB_ISNULL(inc_part_array)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("partition array is null", K(ret), K(inc_table_schema));
 | 
						|
    } else {
 | 
						|
      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("partition is null", K(ret), K(i), K(inc_part_num));
 | 
						|
        } else {
 | 
						|
          bool check_dropped_schema = false;
 | 
						|
          ObPartIteratorV2 iter(orig_table_schema, check_dropped_schema);
 | 
						|
          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(inc_part->get_part_name(), part->get_part_name())) {
 | 
						|
              if (OB_FAIL(inc_part->assign(*part))) {
 | 
						|
                LOG_WARN("failed to assign partition", K(ret), K(part), K(inc_part));
 | 
						|
              }
 | 
						|
              break;
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
        if (OB_ITER_END == ret) {
 | 
						|
          ret = OB_PARTITION_NOT_EXIST;
 | 
						|
          LOG_WARN("part should exists", KR(ret), KPC(inc_part));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_FAIL(ddl_operator.truncate_table_partitions(orig_table_schema, inc_table_schema, trans))) {
 | 
						|
      LOG_WARN("failed to truncate partitions", KR(ret));
 | 
						|
    }
 | 
						|
  } else if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == op_type) {
 | 
						|
    if (orig_table_schema.is_sub_part_template()) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "drop subpartition for nontemplate");
 | 
						|
      LOG_WARN("drop subpartition is not supported for nontemplate", KR(ret));
 | 
						|
    } else if (OB_FAIL(ddl_operator.truncate_table_subpartitions(orig_table_schema, inc_table_schema, trans))) {
 | 
						|
      LOG_WARN("failed to drop table partitions", KR(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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 (is_sync_primary_ddl()) {
 | 
						|
      // The ddl synchronized from the standalone cluster is not controlled by the configuration item
 | 
						|
    } else 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;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::alter_table_sess_active_time_in_trans(
 | 
						|
    obrpc::ObAlterTableArg& alter_table_arg, const int64_t frozen_version)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObDatabaseSchema* database_schema = NULL;
 | 
						|
  AlterTableSchema& alter_table_schema = alter_table_arg.alter_table_schema_;
 | 
						|
  ObSEArray<const ObSimpleTableSchemaV2*, 512> table_schemas;
 | 
						|
  if (OB_FAIL(schema_service_->get_schema_guard(schema_guard))) {
 | 
						|
    LOG_WARN("fail to get schema guard", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_table_schemas(table_schemas))) {
 | 
						|
    LOG_WARN("fail to get table schema", K(ret));
 | 
						|
  } 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));
 | 
						|
    }
 | 
						|
    alter_table_schema.set_sess_active_time(ObTimeUtility::current_time());
 | 
						|
    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()) {
 | 
						|
        database_schema = NULL;
 | 
						|
        if (OB_FAIL(schema_guard.get_database_schema(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 (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(alter_table_in_trans(alter_table_arg, frozen_version))) {
 | 
						|
            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));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// fix me :Check whether the newly added index column covers the partition column
 | 
						|
// 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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_UNLIKELY(NULL == root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root balancer is null", K(ret), KP(root_balancer_));
 | 
						|
  } else {
 | 
						|
    bool need_process_failed = false;
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    AlterTableSchema& alter_table_schema = alter_table_arg.alter_table_schema_;
 | 
						|
    ObTableSchema new_table_schema;
 | 
						|
 | 
						|
    // Adding a partition operation will open a separate transaction to update max_used_part_id before this,
 | 
						|
    // so the alter_table_in_trans function needs to get the latest schema_guard again
 | 
						|
    ObSchemaGetterGuard schema_guard;
 | 
						|
    uint64_t tenant_id = alter_table_schema.get_tenant_id();
 | 
						|
    schema_guard.set_session_id(alter_table_arg.session_id_);
 | 
						|
    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 {
 | 
						|
      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 need_update_index_table = false;
 | 
						|
        const ObTableSchema* orig_table_schema = NULL;
 | 
						|
        const ObTenantSchema* tenant_schema = NULL;
 | 
						|
        bool is_db_in_recyclebin = false;
 | 
						|
        AlterLocalityOp alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
        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(
 | 
						|
                       orig_table_schema->get_database_id(), is_db_in_recyclebin))) {
 | 
						|
          LOG_WARN("check database in recyclebin failed", K(ret), 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");
 | 
						|
        } 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));
 | 
						|
        } else {
 | 
						|
          // copy from the old table_schema
 | 
						|
          if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) {
 | 
						|
            LOG_WARN("fail to assign schema", K(ret));
 | 
						|
          } else if (alter_table_arg.is_alter_columns_ ||
 | 
						|
                     (alter_table_arg.is_alter_options_ && alter_table_arg.need_progressive_merge())) {
 | 
						|
            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)) {
 | 
						|
            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(new_table_schema), K(*orig_table_schema), K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
 | 
						|
        ObDDLSQLTransaction trans(schema_service_);
 | 
						|
        if (OB_FAIL(ret)) {
 | 
						|
          // do nothing
 | 
						|
        } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
          LOG_WARN("start transaction failed", K(ret));
 | 
						|
        } else {
 | 
						|
          ObArray<ObTableSchema> 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(ddl_operator.alter_table_column(*orig_table_schema,
 | 
						|
                           alter_table_schema,
 | 
						|
                           new_table_schema,
 | 
						|
                           alter_table_arg.tz_info_wrap_,
 | 
						|
                           alter_table_arg.nls_formats_,
 | 
						|
                           trans,
 | 
						|
                           alter_table_arg.allocator_,
 | 
						|
                           &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, new_table_schema, ddl_operator, trans))) {
 | 
						|
              LOG_WARN("alter table foreign keys failed", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
 | 
						|
          // table indexs
 | 
						|
          if (OB_SUCC(ret) && alter_table_arg.is_alter_indexs_) {
 | 
						|
            need_process_failed = true;
 | 
						|
            if (OB_FAIL(check_restore_point_allow(tenant_id, orig_table_schema->get_table_id()))) {
 | 
						|
              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_))) {
 | 
						|
              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_,
 | 
						|
                    *orig_table_schema,
 | 
						|
                    alter_table_schema,
 | 
						|
                    new_table_schema,
 | 
						|
                    ddl_operator,
 | 
						|
                    trans))) {
 | 
						|
              LOG_WARN("alter table constraints failed", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
          // table partitons
 | 
						|
          ObArray<int64_t> 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->get_table_id()))) {
 | 
						|
              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(alter_table_partitions(alter_table_arg.alter_part_type_,
 | 
						|
                           *orig_table_schema,
 | 
						|
                           alter_table_schema,
 | 
						|
                           new_table_schema,
 | 
						|
                           ddl_operator,
 | 
						|
                           trans))) {
 | 
						|
              LOG_WARN("alter table partitions 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;
 | 
						|
            }
 | 
						|
            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 version and max used column is!", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
 | 
						|
          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) {
 | 
						|
            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) {
 | 
						|
              ObCreateIndexArg* create_index_arg = static_cast<ObCreateIndexArg*>(index_arg);
 | 
						|
              ObTableSchema& index_schema = create_index_arg->index_schema_;
 | 
						|
              if (OB_FAIL(ddl_operator.insert_ori_schema_version(
 | 
						|
                      trans, index_schema.get_table_id(), last_schema_version))) {
 | 
						|
                LOG_WARN("failed to insert_ori_schema_version!", K(ret));
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
 | 
						|
          if (OB_SUCC(ret)) {
 | 
						|
            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_) {
 | 
						|
              need_process_failed = true;
 | 
						|
              ObPartIdsGeneratorForAdd<ObTableSchema> gen(*orig_table_schema, alter_table_schema);
 | 
						|
              if (OB_FAIL(gen.gen(new_partition_ids))) {
 | 
						|
                LOG_WARN("fail to gen partition ids", K(ret));
 | 
						|
              } else if (OB_FAIL(add_partitions_for_add(last_schema_version,
 | 
						|
                             *orig_table_schema,
 | 
						|
                             alter_table_schema,
 | 
						|
                             new_partition_ids,
 | 
						|
                             alter_table_arg.create_mode_))) {
 | 
						|
                LOG_WARN("add partitions failed",
 | 
						|
                    K(last_schema_version),
 | 
						|
                    K(*orig_table_schema),
 | 
						|
                    K(alter_table_schema),
 | 
						|
                    K(frozen_version),
 | 
						|
                    K(ret));
 | 
						|
              }
 | 
						|
            } else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_table_arg.alter_part_type_ &&
 | 
						|
                       alter_table_schema.get_all_part_num() > 1) {
 | 
						|
              need_process_failed = true;
 | 
						|
              if (OB_FAIL(add_partitions_for_split(last_schema_version,
 | 
						|
                      *orig_table_schema,
 | 
						|
                      alter_table_schema,
 | 
						|
                      new_table_schema,
 | 
						|
                      new_partition_ids))) {
 | 
						|
                LOG_WARN("add partitions failed",
 | 
						|
                    K(last_schema_version),
 | 
						|
                    KP(orig_table_schema),
 | 
						|
                    K(alter_table_schema),
 | 
						|
                    K(frozen_version),
 | 
						|
                    K(ret));
 | 
						|
              }
 | 
						|
            } else if (obrpc::ObAlterTableArg::REORGANIZE_PARTITION == alter_table_arg.alter_part_type_ ||
 | 
						|
                       obrpc::ObAlterTableArg::SPLIT_PARTITION == alter_table_arg.alter_part_type_) {
 | 
						|
              ObPartIdsGeneratorForAdd<ObTableSchema> gen(*orig_table_schema, alter_table_schema);
 | 
						|
              need_process_failed = true;
 | 
						|
              if (OB_FAIL(gen.gen(new_partition_ids))) {
 | 
						|
                LOG_WARN("fail to gen partition ids", K(ret));
 | 
						|
              } else if (OB_FAIL(add_partitions_for_split(last_schema_version,
 | 
						|
                             *orig_table_schema,
 | 
						|
                             alter_table_schema,
 | 
						|
                             new_table_schema,
 | 
						|
                             new_partition_ids))) {
 | 
						|
                LOG_WARN("add partitions failed",
 | 
						|
                    K(ret),
 | 
						|
                    K(last_schema_version),
 | 
						|
                    K(alter_table_schema),
 | 
						|
                    K(frozen_version),
 | 
						|
                    "origin_table_schema",
 | 
						|
                    *orig_table_schema);
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
          if (OB_SUCC(ret)) {
 | 
						|
            if (orig_table_schema->get_primary_zone().empty() && new_table_schema.get_primary_zone().empty()) {
 | 
						|
              // bypass
 | 
						|
            } else if (!orig_table_schema->get_primary_zone().empty() && !new_table_schema.get_primary_zone().empty()) {
 | 
						|
              // bypass
 | 
						|
            } else if (!orig_table_schema->get_primary_zone().empty() && new_table_schema.get_primary_zone().empty()) {
 | 
						|
              if (OB_FAIL(
 | 
						|
                      try_modify_tenant_primary_zone_entity_count(trans, schema_guard, false /*sub*/, 1, tenant_id))) {
 | 
						|
                LOG_WARN("fail to try modify primary zone entity count", K(ret));
 | 
						|
              }
 | 
						|
            } else {
 | 
						|
              if (OB_FAIL(
 | 
						|
                      try_modify_tenant_primary_zone_entity_count(trans, schema_guard, true /*add*/, 1, tenant_id))) {
 | 
						|
                LOG_WARN("fail to try modify primary zone entity count", K(ret));
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
          // do rs_job
 | 
						|
          if (OB_SUCC(ret) && alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::LOCALITY)) {
 | 
						|
            if (OB_FAIL(
 | 
						|
                    record_table_locality_event_history(alter_locality_op, alter_table_arg, new_table_schema, trans))) {
 | 
						|
              LOG_WARN("fail to record table 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;
 | 
						|
            }
 | 
						|
          }
 | 
						|
          if (OB_FAIL(ret) && need_process_failed) {
 | 
						|
            // push schema version
 | 
						|
            int tmp_ret = OB_SUCCESS;
 | 
						|
            if (OB_SUCCESS != (tmp_ret = process_create_partition_failed(tenant_id))) {
 | 
						|
              LOG_ERROR("alter table failed, may has garbage partition", K(ret), K(tmp_ret), K(tenant_id));
 | 
						|
            }
 | 
						|
          }
 | 
						|
          if (OB_SUCC(ret) && alter_locality_op == ROLLBACK_ALTER_LOCALITY) {
 | 
						|
            uint64_t tenant_id = alter_table_schema.get_tenant_id();
 | 
						|
            task_mgr_->clear_task(tenant_id, obrpc::ObAdminClearBalanceTaskArg::ALL);
 | 
						|
          }
 | 
						|
          // always notify locality modification whether the above process succeed,
 | 
						|
          // and this shall be invoked alter trans.end(), don't change the sequence.
 | 
						|
          if (!new_table_schema.get_previous_locality_str().empty()) {
 | 
						|
            root_balancer_->notify_locality_modification();
 | 
						|
            root_balancer_->wakeup();
 | 
						|
          }
 | 
						|
 | 
						|
          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/modify column
 | 
						|
          ObAutoincrementService& autoinc_service = ObAutoincrementService::get_instance();
 | 
						|
          if (OB_SUCC(ret)) {
 | 
						|
            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()))) {
 | 
						|
                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;
 | 
						|
}
 | 
						|
 | 
						|
/*
 | 
						|
 * 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
 | 
						|
 */
 | 
						|
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;
 | 
						|
  int alter_column_mode = GCONF._alter_column_mode;
 | 
						|
  const int64_t table_id = orig_table_schema.get_table_id();
 | 
						|
  if (OB_FAIL(check_restore_point_allow(tenant_id, table_id))) {
 | 
						|
    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<AlterColumnSchema*>(*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));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (ALTER_COLUMN_MODE_ALLOWED == alter_column_mode) {
 | 
						|
      // skip
 | 
						|
    } else if (ALTER_COLUMN_MODE_FORBIDDEN == alter_column_mode) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("can't alter column when _alter_column_mode is 0", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter column when _alter_column_mode is 0");
 | 
						|
    } else if (ALTER_COLUMN_MODE_FORBIDDEN_WHEN_MAJOR == alter_column_mode) {
 | 
						|
      int64_t last_merged_version = 0;
 | 
						|
      share::ObSimpleFrozenStatus frozen_status;
 | 
						|
      if (!need_drop_column) {
 | 
						|
        // skip
 | 
						|
      } else if (OB_ISNULL(zone_mgr_) || OB_ISNULL(freeze_info_manager_)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("ptr is null", K(ret), KP_(zone_mgr), KP_(freeze_info_manager), KP_(rpc_proxy), KP_(server_mgr));
 | 
						|
      } else if (OB_FAIL(zone_mgr_->get_global_last_merged_version(last_merged_version))) {
 | 
						|
        LOG_WARN("fail to get last merged version", K(ret));
 | 
						|
      } else if (OB_FAIL(freeze_info_manager_->get_freeze_info(0, frozen_status))) {
 | 
						|
        LOG_WARN("fail to get freeze info", KR(ret));
 | 
						|
      } else if (frozen_status.frozen_version_ != last_merged_version) {
 | 
						|
        ret = OB_OP_NOT_ALLOW;
 | 
						|
        LOG_WARN("can't alter column when major freeze is not finished", K(ret));
 | 
						|
        LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter column when major freeze is not finished");
 | 
						|
      } else {
 | 
						|
        // check all server merge to lastest frozen version
 | 
						|
        if (OB_FAIL(ret)) {
 | 
						|
        } else if (OB_FAIL(check_all_server_frozen_version(frozen_status.frozen_version_))) {
 | 
						|
          LOG_WARN("fail to check all servers' frozen version", K(ret), K(frozen_status));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      // overwrite ret
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
        LOG_WARN("check drop column failed", K(ret));
 | 
						|
        ret = OB_ERR_CHECK_DROP_COLUMN_FAILED;
 | 
						|
        LOG_USER_ERROR(OB_ERR_CHECK_DROP_COLUMN_FAILED);
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("invalid _alter_column_mode", K(ret), K(alter_column_mode));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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 int64_t table_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  bool is_exist = false;
 | 
						|
  if (!is_inner_table(table_id) &&
 | 
						|
      OB_FAIL(get_snapshot_mgr().check_restore_point(get_sql_proxy(), tenant_id, table_id, is_exist))) {
 | 
						|
    LOG_WARN("failed to check restore point", K(ret), K(tenant_id));
 | 
						|
  } else if (is_exist) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("restore point exist, cannot alter ", K(ret), K(tenant_id), K(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<share::ObServerStatus> 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;
 | 
						|
  ObSEArray<ObAuxTableMetaInfo, 16> 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(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_unavailable_index()) {
 | 
						|
        ret = OB_NOT_SUPPORTED;
 | 
						|
        LOG_WARN("alter partitions when index status is unavailable is not supported",
 | 
						|
            KR(ret),
 | 
						|
            "index_tid",
 | 
						|
            simple_index_infos.at(i).table_id_,
 | 
						|
            KPC(index_table_schema));
 | 
						|
        LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter partitions when index status is unavailable");
 | 
						|
      } 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;
 | 
						|
}
 | 
						|
// Check whether the newly added partition key meets the requirements
 | 
						|
// If only the partition can be added, skip the subpartition check;
 | 
						|
// 1. In the case of a primary key exist, the partition key must be part of the primary key.
 | 
						|
//  The local unique index key contains the partition key, that will not be. because
 | 
						|
//  the check has been completed in the resolve phase, and the rs will not recheck
 | 
						|
// 2. In the absence of a primary key, all index tables must include partition key
 | 
						|
int ObDDLService::check_new_partition_key_valid(const share::schema::ObTableSchema& orig_table_schema,
 | 
						|
    const share::schema::AlterTableSchema& alter_table_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (!orig_table_schema.is_valid() || PARTITION_LEVEL_ZERO != orig_table_schema.get_part_level() ||
 | 
						|
      PARTITION_LEVEL_ONE != alter_table_schema.get_part_level()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("table_schema is invalid", K(ret), K(orig_table_schema), K(alter_table_schema));
 | 
						|
  } else if (orig_table_schema.is_no_pk_table()) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("table has no primary key, refuse to split", K(ret), "table_id", orig_table_schema.get_table_id());
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "split partition without primary key");
 | 
						|
  } else {
 | 
						|
    // check whether unique index to meet the demand
 | 
						|
    // In the index check, column array information and partition information are needed,
 | 
						|
    // when non-partitioned table convert to partition table operation,
 | 
						|
    // orig_table_schema lacks partition information
 | 
						|
    // alter_table_schema did not fill in column_name_array information and constraint information
 | 
						|
    // during deserialization, so a new table_schema needs to be constructed
 | 
						|
    ObTableSchema new_table_schema;
 | 
						|
    ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
    if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
 | 
						|
      LOG_WARN("get_index_tid_array failed", K(ret));
 | 
						|
    } else if (OB_FAIL(new_table_schema.assign(orig_table_schema))) {
 | 
						|
      // deep copy all info of orig_table_schema
 | 
						|
      LOG_WARN("fail to assign schema", K(ret));
 | 
						|
    } else if (0 < simple_index_infos.count()) {
 | 
						|
      // alter table schema has not constraint info that is in orig_table_shema.
 | 
						|
      if (OB_FAIL(new_table_schema.assign(alter_table_schema))) {
 | 
						|
        LOG_WARN("failed to assign table schema", K(ret), K(alter_table_schema));
 | 
						|
      } else if (OB_FAIL(new_table_schema.assign_constraint(orig_table_schema))) {
 | 
						|
        LOG_WARN("failed to assign constraint", K(ret), K(orig_table_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    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(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_, 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_TYPE_UNIQUE_LOCAL == index_table_schema->get_index_type() ||
 | 
						|
                 INDEX_TYPE_DOMAIN_CTXCAT == index_table_schema->get_index_type()) {
 | 
						|
        if (OB_FAIL(check_unique_index_cover_partition_column(new_table_schema, *index_table_schema))) {
 | 
						|
          LOG_WARN("failed to check index table cover partition key",
 | 
						|
              K(ret),
 | 
						|
              K(new_table_schema),
 | 
						|
              "index_table",
 | 
						|
              *index_table_schema);
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// Check whether the newly added partition meets the requirements
 | 
						|
// of the unique index index: the index key must contain all partition keys
 | 
						|
int ObDDLService::check_unique_index_cover_partition_column(
 | 
						|
    share::schema::ObTableSchema& table_schema, const share::schema::ObTableSchema& index_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (!table_schema.is_valid() || !index_schema.is_valid()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("index or data table schema is invalid", K(ret), K(table_schema), K(index_schema));
 | 
						|
  } else {
 | 
						|
    obrpc::ObCreateIndexArg index_arg;
 | 
						|
    obrpc::ObColumnSortItem column_item;
 | 
						|
    index_arg.index_type_ = index_schema.get_index_type();
 | 
						|
    // get index column in ObColumnSchemaV2, build ObCreateIndexArg
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < index_schema.get_column_count(); ++i) {
 | 
						|
      const share::schema::ObColumnSchemaV2* col_schema = index_schema.get_column_schema_by_idx(i);
 | 
						|
      if (OB_ISNULL(col_schema)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("failed to get columns", K(ret), K(i), K(index_schema));
 | 
						|
      } else if (col_schema->is_index_column()) {
 | 
						|
        column_item.reset();
 | 
						|
        column_item.column_name_.assign_ptr(
 | 
						|
            col_schema->get_column_name_str().ptr(), col_schema->get_column_name_str().length());
 | 
						|
        if (OB_FAIL(index_arg.index_columns_.push_back(column_item))) {
 | 
						|
          LOG_WARN("failed to push back column item", K(ret), K(column_item), K(col_schema));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
      // nothing
 | 
						|
    } else if (0 >= index_arg.index_columns_.count()) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("failed to find index columns", K(ret), K(index_arg), K(table_schema));
 | 
						|
    } else if (OB_FAIL(sql::ObResolverUtils::check_unique_index_cover_partition_column(table_schema, index_arg))) {
 | 
						|
      LOG_WARN("fail to check unique key cover partition column", K(ret));
 | 
						|
      if (INDEX_TYPE_UNIQUE_LOCAL == 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 = sql::ObDDLResolver::check_uniq_allow(table_schema, 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;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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, const 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;
 | 
						|
  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 (is_new_tablegroup_id(tablegroup_id) && obrpc::ObAlterTableArg::PARTITIONED_TABLE != alter_part_type &&
 | 
						|
             obrpc::ObAlterTableArg::TRUNCATE_PARTITION != alter_part_type) {
 | 
						|
    // after 2.0 do not allow separate additions, deletions, and splitting of tables in tablegroup.
 | 
						|
    // can specify the partition key operation and truncate
 | 
						|
    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));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_part_type) {
 | 
						|
    if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2100) {
 | 
						|
      ret = OB_NOT_SUPPORTED;
 | 
						|
      LOG_WARN("converting non-partitioned table to partition table before version 2.1 not support",
 | 
						|
          K(ret),
 | 
						|
          K(GET_MIN_CLUSTER_VERSION()));
 | 
						|
      LOG_USER_ERROR(OB_NOT_SUPPORTED, "converting non-partitioned table to partition table before version 2.1");
 | 
						|
    } else if (OB_FAIL(check_alter_partition_table(orig_table_schema, alter_table_arg, schema_guard))) {
 | 
						|
      LOG_WARN("failed to check partition table", K(ret), K(orig_table_schema), K(alter_table_arg));
 | 
						|
    }
 | 
						|
    is_split = true;
 | 
						|
  } 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 (obrpc::ObAlterTableArg::REORGANIZE_PARTITION == alter_part_type ||
 | 
						|
             obrpc::ObAlterTableArg::SPLIT_PARTITION == alter_part_type) {
 | 
						|
    if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2100) {
 | 
						|
      ret = OB_NOT_SUPPORTED;
 | 
						|
      LOG_WARN("partitioned table less than 2.1 not support", K(ret), K(GET_MIN_CLUSTER_VERSION()));
 | 
						|
      LOG_USER_ERROR(OB_NOT_SUPPORTED, "partitioned table less than 2.1");
 | 
						|
    } else if (OB_FAIL(check_alter_split_partitions(orig_table_schema, alter_table_arg))) {
 | 
						|
      LOG_WARN("failed to check split paritions", K(ret), K(orig_table_schema), K(alter_table_arg));
 | 
						|
    }
 | 
						|
    is_split = true;
 | 
						|
  } 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;
 | 
						|
}
 | 
						|
// Convert non-partitioned table to partitioned table
 | 
						|
// 1. whether is nonpartition table
 | 
						|
// 2. for table under tablegroup, if the number of partitions is not 1, return error,
 | 
						|
//    and the partition type of other tables under tablegroup shall be the same
 | 
						|
// 3. Currently does not support splitting into level subpartitions
 | 
						|
// 4. The last partition should be default or maxvalue
 | 
						|
int ObDDLService::check_alter_partition_table(const share::schema::ObTableSchema& orig_table_schema,
 | 
						|
    const obrpc::ObAlterTableArg& alter_table_arg, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  AlterTableSchema& alter_table_schema = const_cast<AlterTableSchema&>(alter_table_arg.alter_table_schema_);
 | 
						|
  const ObPartitionLevel part_level = orig_table_schema.get_part_level();
 | 
						|
  const ObPartitionLevel new_part_level = alter_table_schema.get_part_level();
 | 
						|
  const uint64_t tablegroup_id = orig_table_schema.get_tablegroup_id();
 | 
						|
  if (PARTITION_LEVEL_ZERO != part_level) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("can't partitioning a partitioned table", K(ret), K(orig_table_schema));
 | 
						|
  } else if (OB_INVALID_ID != tablegroup_id && is_tablegroup_id(tablegroup_id) &&
 | 
						|
             1 != alter_table_schema.get_all_part_num()) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("can't modify table partition in tablegroup in 2.0", K(ret), K(alter_table_arg));
 | 
						|
    LOG_USER_ERROR(OB_OP_NOT_ALLOW, "can't modify table partition in tablegroup in 2.0");
 | 
						|
  } else if (OB_FAIL(check_split_partition_can_execute())) {
 | 
						|
    LOG_WARN("failed to check split partition can execute", K(ret));
 | 
						|
  } else if (PARTITION_LEVEL_TWO == new_part_level) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "split non partitioned table into two partition table");
 | 
						|
    LOG_WARN("can not split table into two partition table", K(ret), K(alter_table_schema));
 | 
						|
  } else if (ObTableModeFlag::TABLE_MODE_NORMAL != orig_table_schema.get_table_mode_flag()) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("can not split buffer table", K(ret), K(orig_table_schema));
 | 
						|
    LOG_USER_ERROR(OB_OP_NOT_ALLOW, "split buffer table");
 | 
						|
  } else if (OB_FAIL(check_split_table_partition_valid(alter_table_schema))) {
 | 
						|
    LOG_WARN("failed to check partition table valid", K(ret), K(alter_table_schema));
 | 
						|
  } else if (OB_FAIL(check_new_partition_key_valid(orig_table_schema, alter_table_schema, schema_guard))) {
 | 
						|
    LOG_WARN("failed to check new partition key valid", K(ret), K(orig_table_schema), K(alter_table_schema));
 | 
						|
  } else if (OB_INVALID_ID != tablegroup_id && is_tablegroup_id(tablegroup_id) &&
 | 
						|
             1 == alter_table_schema.get_all_part_num()) {
 | 
						|
    // if the table under tablegroup is a single-partition table with a specified partition key,
 | 
						|
    // ensure that the partition type of the table with the specified partition key is the same
 | 
						|
    ObArray<const ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
    const uint64_t tenant_id = orig_table_schema.get_tenant_id();
 | 
						|
    const ObPartitionFuncType part_func_type = alter_table_schema.get_part_option().get_part_func_type();
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schemas_in_tablegroup(tenant_id, tablegroup_id, table_schemas))) {
 | 
						|
      LOG_WARN("failed get table schema in tablegrouop", K(ret), K(tenant_id), K(tablegroup_id));
 | 
						|
    } else {
 | 
						|
      const ObSimpleTableSchemaV2* other_table_schema = NULL;
 | 
						|
      for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); ++i) {
 | 
						|
        // check whether this table is partition table
 | 
						|
        other_table_schema = table_schemas.at(i);
 | 
						|
        if (OB_ISNULL(other_table_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("table schema is null", K(ret), K(i), K(tablegroup_id));
 | 
						|
        } else if (!other_table_schema->has_partition()) {
 | 
						|
          // do nothing
 | 
						|
        } else if (PARTITION_LEVEL_ZERO == other_table_schema->get_part_level()) {
 | 
						|
          // there may be some partition table without partition.
 | 
						|
        } else if (new_part_level != other_table_schema->get_part_level()) {
 | 
						|
          // the level of partition must be same
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("partition table level is not euqal", K(ret), K(new_part_level), K(other_table_schema));
 | 
						|
        } else if (part_func_type == other_table_schema->get_part_option().get_part_func_type()) {
 | 
						|
          // Just find the first partition with the same type
 | 
						|
          break;
 | 
						|
        } else {
 | 
						|
          ret = OB_OP_NOT_ALLOW;
 | 
						|
          LOG_WARN("partition rule diffs from other tables in the tablegroup",
 | 
						|
              K(ret),
 | 
						|
              K(part_func_type),
 | 
						|
              K(other_table_schema));
 | 
						|
          LOG_USER_ERROR(OB_OP_NOT_ALLOW, "partition rule diffs from other tables in the tablegroup");
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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<ObForeignKeyInfo>& 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<ObForeignKeyInfo>& 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) {
 | 
						|
      bool check_dropped_schema = false;
 | 
						|
      ObPartIteratorV2 iter(orig_table_schema, check_dropped_schema);
 | 
						|
      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, "DROP");
 | 
						|
        }
 | 
						|
      } else if (!part->allow_ddl_operator()) {
 | 
						|
        ret = OB_OP_NOT_ALLOW;
 | 
						|
        LOG_WARN("partition can not be drop or partition is doing maintenance operation",
 | 
						|
            K(ret),
 | 
						|
            "source part id",
 | 
						|
            part->get_source_part_ids().at(0));
 | 
						|
        LOG_USER_ERROR(OB_OP_NOT_ALLOW, "drop or truncate partition during partition maintenance");
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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 (orig_table_schema.is_sub_part_template()) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("unsupport management on nontemplate 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;
 | 
						|
}
 | 
						|
 | 
						|
// 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, const obrpc::ObAlterTableArg& alter_table_arg)
 | 
						|
{
 | 
						|
  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();
 | 
						|
  bool is_oracle_mode = false;
 | 
						|
  if (OB_ISNULL(part_array)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("part_array is null", K(ret), K(part_array));
 | 
						|
  } else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
 | 
						|
                 orig_table_schema.get_tenant_id(), 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() + 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);
 | 
						|
  }
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < part_num; ++i) {
 | 
						|
    bool check_dropped_schema = false;
 | 
						|
    ObPartIteratorV2 iter(orig_table_schema, check_dropped_schema);
 | 
						|
    const ObPartition* part = NULL;
 | 
						|
    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())) {
 | 
						|
        ret = OB_ERR_SAME_NAME_PARTITION;
 | 
						|
        LOG_WARN("duplicate partition name", K(ret), K(part_array[i]->get_part_name()));
 | 
						|
        LOG_USER_ERROR(
 | 
						|
            OB_ERR_SAME_NAME_PARTITION, part_array[i]->get_part_name().length(), part_array[i]->get_part_name().ptr());
 | 
						|
      } else if (!orig_table_schema.is_sub_part_template()) {
 | 
						|
        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 {
 | 
						|
          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
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (orig_table_schema.is_range_part()) {
 | 
						|
    const ObRowkey* rowkey_last =
 | 
						|
        &orig_table_schema.get_part_array()[orig_table_schema.get_part_option().get_part_num() - 1]
 | 
						|
             ->get_high_bound_val();
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < part_num; ++i) {
 | 
						|
      const ObRowkey* rowkey_cur = &alter_table_schema.get_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;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } 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<const ObPartition*> 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(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
 | 
						|
                 orig_table_schema.get_tenant_id(), is_oracle_mode))) {
 | 
						|
    LOG_WARN("fail to check is oracle mode", K(ret));
 | 
						|
  } else if (PARTITION_LEVEL_ZERO == part_level || orig_table_schema.is_sub_part_template()) {
 | 
						|
    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) {
 | 
						|
    bool check_dropped_schema = false;
 | 
						|
    ObPartIteratorV2 iter(orig_table_schema, check_dropped_schema);
 | 
						|
    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 (part->get_part_id() != inc_part->get_part_id()) {
 | 
						|
          ret = OB_PARTITION_NOT_EXIST;
 | 
						|
          LOG_WARN("partition is not exist", K(part), K(inc_part), K(ret));
 | 
						|
          LOG_USER_ERROR(OB_PARTITION_NOT_EXIST);
 | 
						|
        } else 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 {
 | 
						|
          orig_parts.push_back(part);
 | 
						|
          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;
 | 
						|
}
 | 
						|
// partition split check
 | 
						|
// 1. Split not supports the split of the subpartition
 | 
						|
// 2. Split only supports list and range partition
 | 
						|
// 3. The count of partition need limit
 | 
						|
// 4. The partition range cannot be changed before and after the split
 | 
						|
// 5. The partition name after the split cannot conflict
 | 
						|
// 6. The partition of split is exist
 | 
						|
int ObDDLService::check_alter_split_partitions(
 | 
						|
    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_;
 | 
						|
  const int64_t part_num = alter_table_schema.get_partition_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();
 | 
						|
  bool is_oracle_mode = false;
 | 
						|
  if (OB_ISNULL(part_array)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("part_array is null", K(ret), K(part_array));
 | 
						|
  } else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
 | 
						|
                 orig_table_schema.get_tenant_id(), 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 (PARTITION_LEVEL_TWO == part_level) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "split subpartition");
 | 
						|
    LOG_WARN("table has subpartition, cannot split", K(ret), K(alter_table_schema));
 | 
						|
  } else if (ObTableModeFlag::TABLE_MODE_NORMAL != orig_table_schema.get_table_mode_flag()) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("can not split buffer table", K(ret), K(orig_table_schema));
 | 
						|
    LOG_USER_ERROR(OB_OP_NOT_ALLOW, "split buffer table");
 | 
						|
  } else if (OB_FAIL(check_split_partition_can_execute())) {
 | 
						|
    LOG_WARN("split table while standby exist not allowed", K(ret));
 | 
						|
  } else if (orig_table_schema.is_no_pk_table()) {
 | 
						|
    // A partitioned table without a primary key cannot be split
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("table has no primary key, refuse to split", K(ret), "table_id", orig_table_schema.get_table_id());
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "split partition without primary key");
 | 
						|
  } else if (!part_option.is_range_part() && !part_option.is_list_part()) {
 | 
						|
    ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION;
 | 
						|
    LOG_WARN("split or reorganize partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg));
 | 
						|
  } else if ((is_oracle_mode && orig_table_schema.get_all_part_num() + part_num - 1 > OB_MAX_PARTITION_NUM_ORACLE) ||
 | 
						|
             (!is_oracle_mode && orig_table_schema.get_all_part_num() + part_num - 1 > OB_MAX_PARTITION_NUM_MYSQL)) {
 | 
						|
    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);
 | 
						|
  }
 | 
						|
  bool equal_split = false;
 | 
						|
  bool check_dropped_schema = false;
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < part_num; ++i) {
 | 
						|
    ObPartIteratorV2 iter(orig_table_schema, check_dropped_schema);
 | 
						|
    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())) {
 | 
						|
        if (!equal_split &&
 | 
						|
            ObCharset::case_insensitive_equal(part->get_part_name(), alter_table_schema.get_split_partition_name())) {
 | 
						|
          equal_split = true;
 | 
						|
        } else {
 | 
						|
          break;
 | 
						|
        }
 | 
						|
        // The partition name can be repeated with the partition being split, not with other partitions
 | 
						|
      }
 | 
						|
    }  // end while
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
      if (OB_ITER_END != ret) {
 | 
						|
        LOG_WARN("iter failed", K(ret));
 | 
						|
      } else {
 | 
						|
        ret = OB_SUCCESS;
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      ret = OB_ERR_SAME_NAME_PARTITION;
 | 
						|
      LOG_WARN("duplicate partition name", K(ret), "part name", part_array[i]->get_part_name());
 | 
						|
      LOG_USER_ERROR(
 | 
						|
          OB_ERR_SAME_NAME_PARTITION, part_array[i]->get_part_name().length(), part_array[i]->get_part_name().ptr());
 | 
						|
    }
 | 
						|
  }  // end for
 | 
						|
  ObPartIteratorV2 iter(orig_table_schema, check_dropped_schema);
 | 
						|
  int64_t index = -1;
 | 
						|
  // split source partition is exist
 | 
						|
  const ObPartition* part = NULL;
 | 
						|
  while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
 | 
						|
    index++;
 | 
						|
    if (OB_ISNULL(part)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("NULL ptr", KP(part), K(ret));
 | 
						|
    } else if (ObCharset::case_insensitive_equal(
 | 
						|
                   part->get_part_name(), alter_table_schema.get_split_partition_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 reorganize not exist",
 | 
						|
          K(ret),
 | 
						|
          "split partition name",
 | 
						|
          alter_table_schema.get_split_partition_name());
 | 
						|
      LOG_USER_ERROR(OB_ERR_DROP_PARTITION_NON_EXISTENT, "reorganize");
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (orig_table_schema.is_range_part()) {
 | 
						|
    if (OB_SUCC(ret) && 0 != index) {
 | 
						|
      const ObRowkey* rowkey_left = &orig_table_schema.get_part_array()[index - 1]->get_high_bound_val();
 | 
						|
      const ObRowkey* rowkey_beg = &alter_table_schema.get_part_array()[0]->get_high_bound_val();
 | 
						|
      if (*rowkey_beg <= *rowkey_left) {
 | 
						|
        ret = OB_ERR_RANGE_NOT_INCREASING_ERROR;
 | 
						|
        LOG_WARN("split must is equal", K(ret), "origin rowkey", *rowkey_left, "new rowkey", *rowkey_beg);
 | 
						|
        const ObString& err_msg = orig_table_schema.get_part_array()[index - 1]->get_part_name();
 | 
						|
        LOG_USER_ERROR(OB_ERR_RANGE_NOT_INCREASING_ERROR, lib::is_oracle_mode() ? err_msg.length() : 0, err_msg.ptr());
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // Other partitions should keep increasing monotonically
 | 
						|
    const ObRowkey* rowkey_last = &alter_table_schema.get_part_array()[part_num - 1]->get_high_bound_val();
 | 
						|
    for (int64_t i = part_num - 2; OB_SUCC(ret) && i >= 0; --i) {
 | 
						|
      const ObRowkey* rowkey_cur = &alter_table_schema.get_part_array()[i]->get_high_bound_val();
 | 
						|
      if (*rowkey_cur >= *rowkey_last) {
 | 
						|
        ret = OB_ERR_RANGE_NOT_INCREASING_ERROR;
 | 
						|
        LOG_WARN("range values should increasing", K(ret), "curr_rowkey", *rowkey_cur, "last_rowkey", *rowkey_last);
 | 
						|
        const ObString& err_msg = alter_table_schema.get_part_array()[i]->get_part_name();
 | 
						|
        LOG_USER_ERROR(OB_ERR_RANGE_NOT_INCREASING_ERROR, lib::is_oracle_mode() ? err_msg.length() : 0, err_msg.ptr());
 | 
						|
      } else {
 | 
						|
        rowkey_last = rowkey_cur;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // There is no data loss after the split, the set before and after the partition should be the same
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      const ObRowkey* rowkey_last = &orig_table_schema.get_part_array()[index]->get_high_bound_val();
 | 
						|
      const ObRowkey* rowkey_cur = &alter_table_schema.get_part_array()[part_num - 1]->get_high_bound_val();
 | 
						|
      // The split point of the largest partition should be consistent with the original partition
 | 
						|
      if (*rowkey_cur != *rowkey_last) {
 | 
						|
        ret = OB_ERR_REORGANIZE_OUTSIDE_RANGE;
 | 
						|
        LOG_WARN("split must is equal", K(ret), "curr_rowkey", *rowkey_cur, "last_rowkey", *rowkey_last);
 | 
						|
        LOG_USER_ERROR(OB_ERR_REORGANIZE_OUTSIDE_RANGE);
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (orig_table_schema.is_list_part()) {
 | 
						|
    // for list partition, you need to ensure that the contents of the partition are not lost before and after the
 | 
						|
    // partition is split
 | 
						|
    if (OB_FAIL(check_split_list_partition_match(alter_table_schema, orig_table_schema, index))) {
 | 
						|
      LOG_WARN("failed to check split partition match", K(ret), K(index), 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<common::ObRowkey> 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<common::ObNewRow>* 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<common::ObNewRow>* 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<common::ObRowkey> 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<common::ObNewRow>* 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<common::ObNewRow>* 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;
 | 
						|
}
 | 
						|
// Ensure that the value of the partition after the split is consistent with the value before the split
 | 
						|
// Before the split, if it was the default partition. judgment is not needed.
 | 
						|
// only need to have the default partition after the split.
 | 
						|
int ObDDLService::check_split_list_partition_match(const share::schema::ObPartitionSchema& new_part,
 | 
						|
    const share::schema::ObPartitionSchema& orig_part, const int64_t split_part_idx)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  int64_t part_num = new_part.get_partition_num();
 | 
						|
  ObPartition** part_array = new_part.get_part_array();
 | 
						|
  common::hash::ObHashSet<common::ObRowkey> list_row_map;
 | 
						|
  ObPartition** orig_part_array = orig_part.get_part_array();
 | 
						|
  common::ObRowkey row_key;
 | 
						|
  if (OB_ISNULL(part_array) || OB_ISNULL(orig_part_array) || split_part_idx >= orig_part.get_partition_num()) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("part array is null",
 | 
						|
        K(ret),
 | 
						|
        KP(part_array),
 | 
						|
        KP(orig_part_array),
 | 
						|
        K(split_part_idx),
 | 
						|
        "partition num",
 | 
						|
        orig_part.get_partition_num());
 | 
						|
  } else 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 < 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<common::ObNewRow>* 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 (int64_t 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_FAIL(list_row_map.set_refactored(row_key))) {
 | 
						|
            LOG_WARN("failed to insert hash map", K(ret), K(row_key));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }  // end for
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_ISNULL(orig_part_array[split_part_idx])) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("part is null", K(ret), K(split_part_idx), K(orig_part));
 | 
						|
    } else {
 | 
						|
      const ObIArray<common::ObNewRow>* orig_list_value = &(orig_part_array[split_part_idx]->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), K(split_part_idx), K(orig_part));
 | 
						|
      } else if (1 == orig_list_value->count() && 1 == orig_list_value->at(0).get_count() &&
 | 
						|
                 orig_list_value->at(0).get_cell(0).is_max_value()) {
 | 
						|
        // Splitting the default partition needs to check whether the value of the newly added partition
 | 
						|
        // conflicts with the value of the previous partition
 | 
						|
        int64_t split_part_id = orig_part_array[split_part_idx]->get_part_id();
 | 
						|
        if (OB_FAIL(check_add_list_partition(orig_part, new_part, split_part_id))) {
 | 
						|
          LOG_WARN("check new add list values failed", K(ret));
 | 
						|
        }
 | 
						|
      } else if (orig_list_value->count() != list_row_map.size()) {
 | 
						|
        ret = OB_ERR_SPLIT_LIST_LESS_VALUE;
 | 
						|
        LOG_WARN("reorganize range is not equal", K(ret), K(new_part), "oig table", orig_part);
 | 
						|
        LOG_USER_ERROR(OB_ERR_SPLIT_LIST_LESS_VALUE);
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        int64_t count = orig_list_value->count();
 | 
						|
        for (int64_t i = 0; OB_SUCC(ret) && i < count; ++i) {
 | 
						|
          row_key.reset();
 | 
						|
          row_key.assign(orig_list_value->at(i).cells_, orig_list_value->at(i).get_count());
 | 
						|
          if (OB_HASH_EXIST != list_row_map.exist_refactored(row_key)) {
 | 
						|
            ret = OB_ERR_SPLIT_LIST_LESS_VALUE;
 | 
						|
            LOG_WARN("reorganize range is not equal", K(ret), K(row_key), K(new_part));
 | 
						|
            LOG_USER_ERROR(OB_ERR_SPLIT_LIST_LESS_VALUE);
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::alter_table(obrpc::ObAlterTableArg& alter_table_arg, const int64_t frozen_version)
 | 
						|
{
 | 
						|
  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();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  }
 | 
						|
  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(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 (alter_table_arg.is_alter_partitions_) {
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        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_list_part()) {
 | 
						|
            if (OB_FAIL(complete_split_partition(*orig_table_schema,
 | 
						|
                    alter_table_arg.alter_table_schema_,
 | 
						|
                    obrpc::ObAlterTableArg::SPLIT_PARTITION == alter_table_arg.alter_part_type_))) {
 | 
						|
              LOG_WARN("failed to complete split partition", 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));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      // update partition_cnt_within_partition_table if needed
 | 
						|
      if (OB_SUCC(ret) && -1 == orig_table_schema->get_part_option().get_partition_cnt_within_partition_table()) {
 | 
						|
        const uint64_t table_id = orig_table_schema->get_table_id();
 | 
						|
        if (OB_FAIL(update_partition_cnt_within_partition_table(*orig_table_schema))) {
 | 
						|
          LOG_WARN("update failed", K(*orig_table_schema), K(ret));
 | 
						|
          // get new table schema again
 | 
						|
        } 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(table_id, orig_table_schema))) {
 | 
						|
          LOG_WARN("fail to get table schema", K(ret), K(table_id));
 | 
						|
        } else if (NULL == orig_table_schema) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("NULL ptr", K(orig_table_schema), K(table_id), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      // create partitions (meta table)
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        const bool is_add = (obrpc::ObAlterTableArg::ADD_PARTITION == alter_table_arg.alter_part_type_ ||
 | 
						|
                             obrpc::ObAlterTableArg::ADD_SUB_PARTITION == 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_ ||
 | 
						|
                             obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_ ||
 | 
						|
                             obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_ ||
 | 
						|
                             (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_table_arg.alter_part_type_ &&
 | 
						|
                                 alter_table_schema.get_all_part_num() > 1));
 | 
						|
        if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_table_arg.alter_part_type_ ||
 | 
						|
            obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) {
 | 
						|
          if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_ &&
 | 
						|
              OB_FAIL(gen_inc_table_schema_for_trun_subpart(
 | 
						|
                  *orig_table_schema, const_cast<AlterTableSchema&>(alter_table_schema)))) {
 | 
						|
            LOG_WARN("fail to subpartition_name to subpartition", KR(ret));
 | 
						|
          } else if (OB_FAIL(update_max_used_subpart_id_if_needed(is_add, *orig_table_schema, alter_table_schema))) {
 | 
						|
            LOG_WARN("update_max_used_part_id_if_needed failed",
 | 
						|
                K(is_add),
 | 
						|
                K(*orig_table_schema),
 | 
						|
                K(alter_table_schema),
 | 
						|
                K(ret));
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          if (OB_FAIL(update_max_used_part_id_if_needed(is_add, *orig_table_schema, alter_table_schema))) {
 | 
						|
            LOG_WARN("update_max_used_part_id_if_needed failed",
 | 
						|
                K(is_add),
 | 
						|
                K(*orig_table_schema),
 | 
						|
                K(alter_table_schema),
 | 
						|
                K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // check foreign key info and check constraint
 | 
						|
    // and put foreign_key_info into table schema
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      const ObTableSchema* table_schema = NULL;
 | 
						|
      bool is_oracle_mode = false;
 | 
						|
      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(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
 | 
						|
                     table_schema->get_tenant_id(), is_oracle_mode))) {
 | 
						|
        LOG_WARN("fail to check is oracle mode", K(ret));
 | 
						|
      }
 | 
						|
      // check check constraint info
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (is_oracle_mode && alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::ADD_CONSTRAINT) {
 | 
						|
        ObTableSchema::const_constraint_iterator iter = alter_table_arg.alter_table_schema_.constraint_begin();
 | 
						|
        if (iter + 1 != alter_table_arg.alter_table_schema_.constraint_end()) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("alter table could only add one constraint once", 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(), 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()));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      // 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_, 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<ObForeignKeyInfo>& 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 (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_ = 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)) {
 | 
						|
            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_;
 | 
						|
          }
 | 
						|
          // 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
 | 
						|
    }    // check foreign key info end.
 | 
						|
  }
 | 
						|
 | 
						|
  // 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))) {
 | 
						|
        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(alter_table_in_trans(alter_table_arg, frozen_version))) {
 | 
						|
      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));
 | 
						|
  }
 | 
						|
 | 
						|
  // refresh table schema
 | 
						|
  int tmp_ret = OB_SUCCESS;
 | 
						|
  int64_t orig_schema_version = OB_INVALID_VERSION;
 | 
						|
  int64_t cur_schema_version = OB_INVALID_VERSION;
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_SUCCESS !=
 | 
						|
             (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(tenant_id, orig_schema_version))) {
 | 
						|
    LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(tenant_id));
 | 
						|
  } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
 | 
						|
    LOG_WARN("publish_schema failed", K(tmp_ret));
 | 
						|
  } else if (OB_CREATE_TABLE_MODE_RESTORE == alter_table_arg.create_mode_) {
 | 
						|
    // skip
 | 
						|
  } else if (OB_SUCCESS !=
 | 
						|
             (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(tenant_id, cur_schema_version))) {
 | 
						|
    LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(tenant_id));
 | 
						|
  } else if (cur_schema_version != orig_schema_version) {
 | 
						|
    end_usec = ObTimeUtility::current_time();
 | 
						|
    cost_usec = end_usec - start_usec;
 | 
						|
    start_usec = end_usec;
 | 
						|
    LOG_INFO("publish_schema cost: ", K(cost_usec));
 | 
						|
    if (alter_table_arg.is_alter_partitions_ &&
 | 
						|
        (obrpc::ObAlterTableArg::ADD_PARTITION == alter_table_arg.alter_part_type_ ||
 | 
						|
            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_ ||
 | 
						|
            obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_ ||
 | 
						|
            obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_)) {
 | 
						|
      ObSchemaGetterGuard new_schema_guard;
 | 
						|
      const uint64_t table_id = orig_table.get_table_id();
 | 
						|
      const ObTableSchema* new_table_schema = NULL;
 | 
						|
      if (OB_SUCCESS != (tmp_ret = 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_SUCCESS != (tmp_ret = new_schema_guard.get_table_schema(table_id, new_table_schema))) {
 | 
						|
        LOG_WARN("fail to get table schema", K(tmp_ret), K(table_id));
 | 
						|
      } else if (NULL == new_table_schema) {
 | 
						|
        tmp_ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("NULL ptr", K(table_id), K(tmp_ret));
 | 
						|
      } else {
 | 
						|
        if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_table_schema(*new_table_schema))) {
 | 
						|
          LOG_WARN("fail to check need wait leader by schema", KR(tmp_ret), "new_table_schema", *new_table_schema);
 | 
						|
        }
 | 
						|
        end_usec = ObTimeUtility::current_time();
 | 
						|
        cost_usec = end_usec - start_usec;
 | 
						|
        start_usec = end_usec;
 | 
						|
        LOG_INFO("waiter.wait cost: ", K(cost_usec));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  // submit async build index task
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
    LOG_WARN("publish_schema failed", K(ret));
 | 
						|
  } else if (OB_CREATE_TABLE_MODE_RESTORE == alter_table_arg.create_mode_) {
 | 
						|
    // skip
 | 
						|
  } else {
 | 
						|
    ObIndexBuilder index_builder(*this);
 | 
						|
    const ObSArray<ObIndexArg*>& 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<ObIndexArg*>(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 (ObIndexArg::ADD_INDEX == index_arg->index_action_type_ ||
 | 
						|
                 ObIndexArg::REBUILD_INDEX == index_arg->index_action_type_) {
 | 
						|
        ObCreateIndexArg* create_index_arg = static_cast<ObCreateIndexArg*>(index_arg);
 | 
						|
        ObTableSchema& index_schema = create_index_arg->index_schema_;
 | 
						|
        if ((INDEX_TYPE_NORMAL_GLOBAL == create_index_arg->index_type_ ||
 | 
						|
                INDEX_TYPE_UNIQUE_GLOBAL == create_index_arg->index_type_)) {
 | 
						|
          start_usec = ObTimeUtility::current_time();
 | 
						|
          if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_table_schema(index_schema))) {
 | 
						|
            LOG_WARN("fail to check need wait leader by schema", KR(tmp_ret), K(index_schema));
 | 
						|
          }
 | 
						|
          cost_usec = ObTimeUtility::current_time() - start_usec;
 | 
						|
          LOG_INFO("index partition waiter.wait cost: ", K(cost_usec));
 | 
						|
          // submit build global index task
 | 
						|
          if (OB_FAIL(index_builder.submit_build_global_index_task(index_schema))) {
 | 
						|
            LOG_WARN("fail to submit build global index task", K(ret));
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          if (OB_FAIL(index_builder.submit_build_local_index_task(index_schema))) {
 | 
						|
            LOG_WARN("failt to submit build local index task", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ret = tmp_ret;
 | 
						|
  }
 | 
						|
 | 
						|
  // just for debug
 | 
						|
  if (OB_SUCC(ret) && false == is_alter_sess_active_time) {
 | 
						|
    int tmp_ret = OB_SUCCESS;
 | 
						|
    ObSchemaGetterGuard new_schema_guard;
 | 
						|
    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;
 | 
						|
    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(table_id, new_table_schema))) {
 | 
						|
      LOG_WARN("fail to get table schema", K(tmp_ret), 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;
 | 
						|
}
 | 
						|
 | 
						|
// TODO: () whether need create partition, depends on table_schema's part_num.
 | 
						|
//
 | 
						|
int ObDDLService::create_table_partitions(const ObTableSchema& table_schema, const int64_t schema_version,
 | 
						|
    obrpc::ObCreateTableMode create_mode, const int64_t frozen_version, const ObIArray<ObTableSchema>& schemas,
 | 
						|
    const uint64_t last_replay_log_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(frozen_version);
 | 
						|
  RS_TRACE(create_table_partitions_begin);
 | 
						|
  ObTablePartitionAddr table_addr;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } 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 if (OB_ISNULL(root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_balancer_ is null", K(ret));
 | 
						|
  } else if (OB_FAIL(root_balancer_->alloc_partitions_for_create(table_schema, create_mode, table_addr))) {
 | 
						|
    LOG_WARN("alloc partition address failed", K(ret), K(table_schema));
 | 
						|
  } else if (OB_FAIL(create_partitions_for_create(
 | 
						|
                 table_schema, schema_version, table_addr, schemas, create_mode, last_replay_log_id))) {
 | 
						|
    LOG_WARN("create table partitions failed", K(ret), K(table_addr));
 | 
						|
  }
 | 
						|
  RS_TRACE(create_table_partitions_end);
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// Only consider standalone partition, global index, and no need to consider local index
 | 
						|
int ObDDLService::create_table_partitions_for_physical_restore(const obrpc::ObRestorePartitionsArg& arg,
 | 
						|
    const common::hash::ObHashSet<int64_t>& base_part_id_set, const share::ObSimpleFrozenStatus frozen_status,
 | 
						|
    const int64_t last_schema_version, ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  RS_TRACE(create_table_partitions_begin);
 | 
						|
  const uint64_t table_id = arg.schema_id_;
 | 
						|
  const ObCreateTableMode create_mode = ObCreateTableMode::OB_CREATE_TABLE_MODE_LOOSE;
 | 
						|
  const ObTableSchema* table_schema = NULL;
 | 
						|
  ObTablePartitionAddr table_addr;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_ISNULL(root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_balancer_ is null", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_table_schema(table_id, table_schema))) {
 | 
						|
    LOG_WARN("fail to get table", K(ret), K(table_id));
 | 
						|
  } else if (OB_ISNULL(table_schema)) {
 | 
						|
    ret = OB_TABLE_NOT_EXIST;
 | 
						|
    LOG_WARN("table not exist. table may be droppped concurrently in physical restore", KR(ret), K(table_id));
 | 
						|
  } else if (OB_FAIL(root_balancer_->alloc_partitions_for_create(*table_schema, create_mode, table_addr))) {
 | 
						|
    LOG_WARN("alloc partition address failed", K(ret), KPC(table_schema));
 | 
						|
  } else if (OB_FAIL(create_partitions_for_physical_restore(
 | 
						|
                 schema_guard, *table_schema, arg, base_part_id_set, frozen_status, last_schema_version, table_addr))) {
 | 
						|
    LOG_WARN("fail to create restore partitions", K(ret), K(table_id));
 | 
						|
  }
 | 
						|
  RS_TRACE(create_table_partitions_end);
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::add_tablegroup_partitions_for_add(const ObTablegroupSchema& tablegroup_schema,
 | 
						|
    const ObTablegroupSchema& inc_tablegroup_schema, const common::ObIArray<int64_t>& partition_ids,
 | 
						|
    const ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::ObSimpleFrozenStatus frozen_status;
 | 
						|
  common::ObArray<ObPartitionAddr> tablegroup_addr;
 | 
						|
  int64_t tenant_id = tablegroup_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (create_mode <= obrpc::OB_CREATE_TABLE_MODE_INVALID || create_mode >= obrpc::OB_CREATE_TABLE_MODE_MAX) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(create_mode));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == freeze_info_manager_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("freeze info manager_ is null", K(ret));
 | 
						|
  } else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id, frozen_status))) {
 | 
						|
    LOG_WARN("fail to get frozen status", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root balancer ptr is null", K(ret));
 | 
						|
  } else if (OB_FAIL(root_balancer_->alloc_partitions_for_add(
 | 
						|
                 tablegroup_schema, inc_tablegroup_schema, create_mode, tablegroup_addr))) {
 | 
						|
    LOG_WARN("fail to alloc tablegroup partitions for create", K(ret));
 | 
						|
  } else if (OB_FAIL(create_tablegroup_partitions_for_create(
 | 
						|
                 inc_tablegroup_schema, tablegroup_addr, partition_ids, frozen_status, create_mode))) {
 | 
						|
    LOG_WARN("fail to create tablegroup partitions for create", K(ret));
 | 
						|
  } else if (OB_FAIL(check_need_wait_leader_by_tablegroup_schema(inc_tablegroup_schema))) {
 | 
						|
    LOG_WARN("fail to check need wati leader by schema", KR(ret), K(inc_tablegroup_schema));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// Ensure that the split partition and the source partition are on the same machine
 | 
						|
int ObDDLService::add_tablegroup_partitions_for_split(const ObTablegroupSchema& tablegroup_schema,
 | 
						|
    const ObTablegroupSchema& new_tablegroup_schema, const common::ObIArray<int64_t>& partition_ids,
 | 
						|
    const ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::ObSimpleFrozenStatus frozen_status;
 | 
						|
  common::ObArray<ObPartitionAddr> tablegroup_addr;
 | 
						|
  int64_t tenant_id = tablegroup_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_CREATE_TABLE_MODE_STRICT != create_mode) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(create_mode));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == freeze_info_manager_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("freeze info manager_ is null", K(ret));
 | 
						|
  } else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id, frozen_status))) {
 | 
						|
    LOG_WARN("fail to get frozen status", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root balancer ptr is null", K(ret));
 | 
						|
  } else if (OB_FAIL(root_balancer_->alloc_partitions_for_split(
 | 
						|
                 tablegroup_schema, new_tablegroup_schema, tablegroup_addr))) {
 | 
						|
    LOG_WARN("fail to alloc tablegroup partitions for create", K(ret), K(new_tablegroup_schema));
 | 
						|
  } else if (OB_FAIL(create_tablegroup_partitions_for_create(
 | 
						|
                 new_tablegroup_schema, tablegroup_addr, partition_ids, frozen_status, create_mode))) {
 | 
						|
    LOG_WARN("fail to create tablegroup partitions for create", K(ret), K(new_tablegroup_schema));
 | 
						|
  } else if (OB_FAIL(check_need_wait_leader_by_tablegroup_schema(new_tablegroup_schema))) {
 | 
						|
    LOG_WARN("fail to check need wait leader by schema", KR(ret), K(new_tablegroup_schema));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::binding_add_partitions_for_add(const int64_t schema_version,
 | 
						|
    const share::schema::ObTableSchema& new_table_schema, const share::schema::ObTableSchema& orig_table_schema,
 | 
						|
    const share::schema::ObTablegroupSchema& tablegroup_schema, obrpc::ObCreateTableMode& create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObTableSchema> schemas;
 | 
						|
  share::schema::ObSchemaGetterGuard schema_guard;
 | 
						|
  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 if (OB_FAIL(schemas.push_back(new_table_schema))) {
 | 
						|
    LOG_WARN("failed to push back", K(ret), K(new_table_schema));
 | 
						|
  } 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 {
 | 
						|
    ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
    const ObTableSchema* index_schema = NULL;
 | 
						|
    if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
 | 
						|
      STORAGE_LOG(WARN, "fail to get index tid array", K(ret));
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
 | 
						|
      if (OB_FAIL(schema_guard.get_table_schema(simple_index_infos.at(i).table_id_, index_schema))) {
 | 
						|
        STORAGE_LOG(WARN, "fail to get table schema", K(ret));
 | 
						|
      } else if (OB_ISNULL(index_schema)) {
 | 
						|
        ret = OB_TABLE_NOT_EXIST;
 | 
						|
        STORAGE_LOG(WARN, "error unexpected, index schema must not be NULL", K(ret));
 | 
						|
      } else if (OB_FAIL(schemas.push_back(*index_schema))) {
 | 
						|
        STORAGE_LOG(WARN, "fail to push back index schema", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_FAIL(binding_table_partitions(
 | 
						|
                 new_table_schema, &tablegroup_schema, schema_version, create_mode, schemas))) {
 | 
						|
    LOG_WARN("failed to binding table partitions", K(ret), K(new_table_schema));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::binding_table_partitions(const share::schema::ObTableSchema& table_schema,
 | 
						|
    const share::schema::ObTablegroupSchema* tg_schema, const int64_t schema_version,
 | 
						|
    obrpc::ObCreateTableMode create_mode, const common::ObIArray<share::schema::ObTableSchema>& schemas)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::schema::ObSchemaGetterGuard schema_guard;
 | 
						|
  common::ObArray<ObTableSchema> available_schemas;
 | 
						|
  const uint64_t tenant_id = table_schema.get_tenant_id();
 | 
						|
  ObPartitionCreator creator(*rpc_proxy_, *pt_operator_, server_mgr_, true /* binding */);
 | 
						|
  share::ObSimpleFrozenStatus frozen_status;
 | 
						|
  int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  int64_t non_paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  PartitionInfoMap partition_info_map;
 | 
						|
  common::ObArray<common::ObPGKey> invalid_part_info_pgk_array;
 | 
						|
  common::ObArray<common::ObPartitionKey> invalid_part_info_pk_array;
 | 
						|
  const int64_t PART_INFO_MAP_SIZE = 2 * (std::max(OB_MAX_PARTITION_NUM_MYSQL, OB_MAX_PARTITION_NUM_ORACLE));
 | 
						|
  share::ObSplitPartition split_info;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == tg_schema)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), KP(tg_schema));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == pt_operator_ || nullptr == freeze_info_manager_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("pt_operator ptr or freeze info manager ptr is null", 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(calc_schema_replica_num(
 | 
						|
                 schema_guard, *tg_schema, paxos_replica_num, non_paxos_replica_num, create_mode))) {
 | 
						|
    LOG_WARN("fail to calc schema replica num", K(ret));
 | 
						|
  } else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id, frozen_status))) {
 | 
						|
    LOG_WARN("fail to get maxs frozen status", K(ret));
 | 
						|
  } else if (OB_FAIL(partition_info_map.create(PART_INFO_MAP_SIZE, ObModIds::OB_RS_PARTITION_TABLE_TEMP))) {
 | 
						|
    LOG_WARN("fail to create info map", K(ret));
 | 
						|
  } else if (table_schema.is_in_splitting()) {
 | 
						|
    if (OB_FAIL(ObPartitionSplitHelper::build_split_info(table_schema, split_info))) {
 | 
						|
      LOG_WARN("failed to build split info", K(ret), K(split_info), K(table_schema));
 | 
						|
    } else if (!split_info.is_valid()) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("split info is invalid", K(ret), K(split_info), K(table_schema));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObTableSchema tmp_schema;
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < schemas.count(); ++i) {
 | 
						|
      const share::schema::ObTableSchema& table_schema = schemas.at(i);
 | 
						|
      if (0 == i ||
 | 
						|
          (is_available_index_status(table_schema.get_index_status()) && !table_schema.has_self_partition())) {
 | 
						|
        tmp_schema.reset();
 | 
						|
        if (OB_FAIL(tmp_schema.assign(table_schema))) {
 | 
						|
          LOG_WARN("fail to assign schema", K(ret));
 | 
						|
        } else if (FALSE_IT(tmp_schema.reset_partition_schema())) {
 | 
						|
          // will never be here
 | 
						|
        } else if (OB_FAIL(available_schemas.push_back(tmp_schema))) {
 | 
						|
          LOG_WARN("fail to push back", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObTablePartitionIterator iter;
 | 
						|
    if (OB_FAIL(iter.init(tg_schema->get_tablegroup_id(), schema_guard, *pt_operator_))) {
 | 
						|
      LOG_WARN("fail to init table partition iterator", K(ret));
 | 
						|
    } else {
 | 
						|
      ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
 | 
						|
      share::ObPartitionInfo pg_info;
 | 
						|
      pg_info.set_allocator(&allocator);
 | 
						|
      while (OB_SUCC(ret) && OB_SUCC(iter.next(pg_info))) {
 | 
						|
        const share::ObPartitionReplica* leader_replica = nullptr;
 | 
						|
        int64_t part_cnt = -1;
 | 
						|
        common::ObPGKey pg_key;
 | 
						|
        if (OB_FAIL(pg_info.get_partition_cnt(part_cnt))) {
 | 
						|
          if (OB_PARTITION_NOT_EXIST == ret) {
 | 
						|
            ret = OB_SUCCESS;
 | 
						|
            // because of don't wait leader, here maybe all replica is FLAG_REPLICA
 | 
						|
          } else {
 | 
						|
            LOG_WARN("fail to get partition cnt", KR(ret));
 | 
						|
          }
 | 
						|
        } else if (OB_FAIL(pg_key.init(pg_info.get_table_id(), pg_info.get_partition_id(), part_cnt))) {
 | 
						|
          LOG_WARN("fail to init pg key", K(ret));
 | 
						|
        } else if (OB_FAIL(pg_info.find_leader_v2(leader_replica))) {
 | 
						|
          if (OB_ENTRY_NOT_EXIST == ret) {
 | 
						|
            ret = OB_SUCCESS;  // overwrite
 | 
						|
          } else {
 | 
						|
          }  // failed,
 | 
						|
        } else {
 | 
						|
          if (OB_FAIL(partition_info_map.set_refactored(pg_key, pg_info, 0 /* not overwrite */))) {
 | 
						|
            LOG_WARN("fail to set refactored", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
        pg_info.reuse();
 | 
						|
        allocator.reuse();
 | 
						|
      }
 | 
						|
      if (OB_ITER_END == ret) {
 | 
						|
        ret = OB_SUCCESS;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    bool check_dropped_schema = false;
 | 
						|
    ObTablePgKeyIter iter(table_schema, tg_schema->get_tablegroup_id(), check_dropped_schema);
 | 
						|
    if (OB_FAIL(iter.init())) {
 | 
						|
      LOG_WARN("fail to iter init", K(ret));
 | 
						|
    } else {
 | 
						|
      common::ObPartitionKey pkey;
 | 
						|
      common::ObPGKey pgkey;
 | 
						|
      obrpc::ObCreatePartitionArg arg;
 | 
						|
      while (OB_SUCC(ret) && OB_SUCC(iter.next(pkey, pgkey))) {
 | 
						|
        arg.reset();
 | 
						|
        const share::ObPartitionInfo* pg_info = nullptr;
 | 
						|
        const share::ObPartitionReplica* leader_replica = nullptr;
 | 
						|
        if (OB_UNLIKELY(!pkey.is_valid() || !pgkey.is_valid())) {
 | 
						|
          ret = OB_INVALID_ARGUMENT;
 | 
						|
          LOG_WARN("invalid pkey or pgkey", K(ret), K(pgkey));
 | 
						|
        } else if (nullptr == (pg_info = partition_info_map.get(pgkey))) {
 | 
						|
          if (OB_FAIL(invalid_part_info_pgk_array.push_back(pgkey))) {
 | 
						|
            LOG_WARN("fail to push back", K(ret));
 | 
						|
          } else if (OB_FAIL(invalid_part_info_pk_array.push_back(pkey))) {
 | 
						|
            LOG_WARN("fail to push back", K(ret));
 | 
						|
          }
 | 
						|
        } else if (OB_FAIL(pg_info->find_leader_v2(leader_replica))) {
 | 
						|
          LOG_WARN("fail to find leader v2", K(ret));
 | 
						|
        } else if (OB_UNLIKELY(nullptr == leader_replica)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("leader replica ptr is null", K(ret));
 | 
						|
        } else if (OB_FAIL(fill_create_binding_partition_arg(pkey,
 | 
						|
                       pgkey,
 | 
						|
                       leader_replica->zone_,
 | 
						|
                       available_schemas,
 | 
						|
                       paxos_replica_num,
 | 
						|
                       non_paxos_replica_num,
 | 
						|
                       schema_version,
 | 
						|
                       frozen_status.frozen_version_,
 | 
						|
                       frozen_status.frozen_timestamp_,
 | 
						|
                       create_mode,
 | 
						|
                       arg))) {
 | 
						|
          LOG_WARN("fail to fill create binding partition arg", K(ret));
 | 
						|
        } else if (OB_FAIL(arg.split_info_.assign(split_info))) {
 | 
						|
          LOG_WARN("failed to assign split info", K(ret), K(split_info), K(arg));
 | 
						|
        } else if (OB_FAIL(creator.add_create_partition_arg(leader_replica->server_, arg))) {
 | 
						|
          LOG_WARN("fail to add argument", K(ret));
 | 
						|
        } else {
 | 
						|
          pkey.reset();
 | 
						|
          pgkey.reset();
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_ITER_END == ret) {
 | 
						|
        ret = OB_SUCCESS;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (invalid_part_info_pk_array.count() != invalid_part_info_pgk_array.count()) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("array count not match",
 | 
						|
          K(ret),
 | 
						|
          "left_cnt",
 | 
						|
          invalid_part_info_pk_array.count(),
 | 
						|
          "right_cnt",
 | 
						|
          invalid_part_info_pgk_array.count());
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
 | 
						|
    share::ObPartitionInfo pg_info;
 | 
						|
    pg_info.set_allocator(&allocator);
 | 
						|
    obrpc::ObCreatePartitionArg arg;
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < invalid_part_info_pgk_array.count(); ++i) {
 | 
						|
      const common::ObPGKey& pgkey = invalid_part_info_pgk_array.at(i);
 | 
						|
      const common::ObPartitionKey& pkey = invalid_part_info_pk_array.at(i);
 | 
						|
      while (OB_SUCC(ret) && THIS_WORKER.get_timeout_remain() > 0) {
 | 
						|
        const share::ObPartitionReplica* leader_replica = nullptr;
 | 
						|
        pg_info.reuse();
 | 
						|
        allocator.reuse();
 | 
						|
        arg.reset();
 | 
						|
        if (OB_UNLIKELY(!pgkey.is_valid() || !pkey.is_valid())) {
 | 
						|
          ret = OB_INVALID_ARGUMENT;
 | 
						|
          LOG_WARN("invalid pkey or pgkey", K(ret), K(pgkey));
 | 
						|
        } else if (OB_SUCCESS != pt_operator_->get(  // attention: do not modify ret value
 | 
						|
                                     pgkey.get_tablegroup_id(),
 | 
						|
                                     pgkey.get_partition_id(),
 | 
						|
                                     pg_info)) {
 | 
						|
          if (THIS_WORKER.get_timeout_remain() <= 0) {
 | 
						|
            ret = OB_TIMEOUT;
 | 
						|
            LOG_WARN("get pg leader timeout", K(ret));
 | 
						|
          } else {
 | 
						|
            usleep(100 * 1000);  // sleep 100ms and retry
 | 
						|
          }
 | 
						|
        } else if (OB_SUCCESS != pg_info.find_leader_v2(leader_replica)) {  // do not modify ret
 | 
						|
          if (THIS_WORKER.get_timeout_remain() <= 0) {
 | 
						|
            ret = OB_TIMEOUT;
 | 
						|
            LOG_WARN("get pg leader timeout", K(ret));
 | 
						|
          } else {
 | 
						|
            usleep(100 * 1000);  // sleep 100ms and retry
 | 
						|
          }
 | 
						|
        } else if (OB_UNLIKELY(nullptr == leader_replica)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("leader replica ptr is null", K(ret));
 | 
						|
        } else if (OB_FAIL(fill_create_binding_partition_arg(pkey,
 | 
						|
                       pgkey,
 | 
						|
                       leader_replica->zone_,
 | 
						|
                       available_schemas,
 | 
						|
                       paxos_replica_num,
 | 
						|
                       non_paxos_replica_num,
 | 
						|
                       schema_version,
 | 
						|
                       frozen_status.frozen_version_,
 | 
						|
                       frozen_status.frozen_timestamp_,
 | 
						|
                       create_mode,
 | 
						|
                       arg))) {
 | 
						|
          LOG_WARN("fail to fill create binding partition arg", K(ret));
 | 
						|
        } else if (OB_FAIL(arg.split_info_.assign(split_info))) {
 | 
						|
          LOG_WARN("fail to assign split info", K(ret), K(arg), K(split_info));
 | 
						|
        } else if (OB_FAIL(creator.add_create_partition_arg(leader_replica->server_, arg))) {
 | 
						|
          LOG_WARN("fail to add argument", K(ret));
 | 
						|
        } else {
 | 
						|
          break;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (OB_FAIL(creator.execute())) {
 | 
						|
      LOG_WARN("fail to execute", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::update_partition_cnt_within_partition_table(const ObTableSchema& orig_table_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = orig_table_schema.get_tenant_id();
 | 
						|
 | 
						|
  int64_t partition_cnt_within_partition_table =
 | 
						|
      orig_table_schema.get_part_option().get_partition_cnt_within_partition_table();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (partition_cnt_within_partition_table != -1) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("unexpected parttion cnt", K(ret), K(partition_cnt_within_partition_table));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans, ", K(ret));
 | 
						|
    } else {
 | 
						|
      ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
      ObTableSchema new_table_schema;
 | 
						|
      if (OB_FAIL(new_table_schema.assign(orig_table_schema))) {
 | 
						|
        LOG_WARN("fail to assign schema", K(ret));
 | 
						|
      } else {
 | 
						|
        partition_cnt_within_partition_table = new_table_schema.get_all_part_num();
 | 
						|
        new_table_schema.get_part_option().set_partition_cnt_within_partition_table(
 | 
						|
            partition_cnt_within_partition_table);
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(ddl_operator.update_partition_cnt_within_partition_table(new_table_schema, trans))) {
 | 
						|
        LOG_WARN("update_partition_cnt_within_partition_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_FAIL(publish_schema(tenant_id))) {
 | 
						|
        LOG_WARN("publish schema failed", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::update_max_used_part_id_if_needed(
 | 
						|
    const bool is_add, const ObTableSchema& table_schema, const ObTableSchema& inc_table_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = table_schema.get_tenant_id();
 | 
						|
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  }
 | 
						|
 | 
						|
  // update max_used_part_id with a separate transaction first
 | 
						|
  // should always update, no matter add or delete partition!
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans, ", K(ret));
 | 
						|
    } else {
 | 
						|
      ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
      if (OB_FAIL(ddl_operator.update_max_used_part_id(is_add, table_schema, inc_table_schema, trans))) {
 | 
						|
        LOG_WARN("update max used part id 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::update_max_used_subpart_id_if_needed(
 | 
						|
    const bool is_add, const ObTableSchema& table_schema, const ObTableSchema& inc_table_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = table_schema.get_tenant_id();
 | 
						|
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_SUCC(ret)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans, ", K(ret));
 | 
						|
    } else {
 | 
						|
      ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
      if (OB_FAIL(ddl_operator.update_max_used_subpart_id(is_add, table_schema, inc_table_schema, trans))) {
 | 
						|
        LOG_WARN("update max used part id 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::add_partitions_for_add(const int64_t schema_version, const ObTableSchema& table_schema,
 | 
						|
    const ObTableSchema& inc_table_schema, const ObIArray<int64_t>& partition_ids, const ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_ISNULL(root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_balancer_ is null", K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObTablePartitionAddr table_addr;
 | 
						|
    int64_t inc_partition_count = inc_table_schema.get_partition_num();
 | 
						|
    if (PARTITION_LEVEL_TWO == table_schema.get_part_level() && !table_schema.is_sub_part_template()) {
 | 
						|
      if (OB_ISNULL(inc_table_schema.get_part_array())) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("part array is null", K(ret));
 | 
						|
      } else {
 | 
						|
        inc_partition_count = 0;
 | 
						|
        for (int64_t i = 0; i < inc_table_schema.get_partition_num(); i++) {
 | 
						|
          if (OB_ISNULL(inc_table_schema.get_part_array()[i])) {
 | 
						|
            ret = OB_ERR_UNEXPECTED;
 | 
						|
            LOG_WARN("part_array[i] is null", K(ret), K(i));
 | 
						|
          } else {
 | 
						|
            inc_partition_count += inc_table_schema.get_part_array()[i]->get_subpartition_num();
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    } else if (PARTITION_LEVEL_TWO == table_schema.get_part_level() && table_schema.is_sub_part_template()) {
 | 
						|
      inc_partition_count *= table_schema.get_sub_part_option().get_part_num();
 | 
						|
    }
 | 
						|
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_FAIL(
 | 
						|
                   root_balancer_->alloc_partitions_for_add(table_schema, inc_table_schema, create_mode, table_addr))) {
 | 
						|
      LOG_WARN("alloc partition address failed", K(ret), K(table_schema), K(inc_table_schema), K(create_mode));
 | 
						|
    } else if (OB_FAIL(create_partitions_for_add(
 | 
						|
                   schema_version, table_schema, inc_partition_count, table_addr, partition_ids, create_mode))) {
 | 
						|
      LOG_WARN("add partitions failed", K(ret), K(schema_version), K(table_addr));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::add_partitions_for_split(const int64_t schema_version, const ObTableSchema& table_schema,
 | 
						|
    const ObTableSchema& inc_table_schema, const ObTableSchema& new_table_schema, ObIArray<int64_t>& partition_ids)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_ISNULL(root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_balancer_ is null", K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObTablePartitionAddr table_addr;
 | 
						|
    // partition split need use OB_CREATE_TABLE_MODE_STRICT mode
 | 
						|
    ObCreateTableMode create_mode = OB_CREATE_TABLE_MODE_STRICT;
 | 
						|
    int64_t inc_partition_count = inc_table_schema.get_part_option().get_part_num();
 | 
						|
    if (OB_FAIL(root_balancer_->alloc_partitions_for_split(table_schema, new_table_schema, table_addr))) {
 | 
						|
      LOG_WARN("alloc partition address failed", K(ret), K(table_schema), K(inc_table_schema));
 | 
						|
    } else if (OB_FAIL(create_partitions_for_split(schema_version,
 | 
						|
                   table_schema,
 | 
						|
                   inc_partition_count,
 | 
						|
                   table_addr,
 | 
						|
                   new_table_schema,
 | 
						|
                   partition_ids,
 | 
						|
                   create_mode))) {
 | 
						|
      LOG_WARN("add partitions failed", K(ret), K(schema_version), K(table_addr));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
    uint64_t sequence_id = OB_INVALID_ID;
 | 
						|
    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();
 | 
						|
      ObWorker::CompatMode compat_mode = ObWorker::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 = ObWorker::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 (ObWorker::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_);
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed", K(ret));
 | 
						|
      } else {
 | 
						|
        // todo use array to replace hashmap and hashset @hualong
 | 
						|
        // record table already be renamed in the schema mgr
 | 
						|
        common::hash::ObPlacementHashSet<ObTableItem, 32> delete_table_set;
 | 
						|
        // record new table name set
 | 
						|
        // table_item -> table_id
 | 
						|
        common::hash::ObHashMap<ObTableItem, uint64_t> 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<ObSchemaType> conflict_schema_types;
 | 
						|
              if (ObString(OB_RECYCLEBIN_SCHEMA_NAME) == to_table_item.database_name_ ||
 | 
						|
                  ObSchemaUtils::is_public_database(to_table_item.database_name_, is_oracle_mode)) {
 | 
						|
                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,
 | 
						|
                             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 (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(
 | 
						|
                                 from_table_schema->get_database_id(), is_db_in_recyclebin))) {
 | 
						|
                    LOG_WARN("check database in recyclebin failed", K(ret));
 | 
						|
                  } 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(table_id, from_table_schema))) {
 | 
						|
                  LOG_WARN("get_table_schema failed", K(ret), 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)) {
 | 
						|
              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 {
 | 
						|
                  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(from_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 (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_, *sql_proxy_);
 | 
						|
                    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));
 | 
						|
                    }
 | 
						|
                  }
 | 
						|
                }
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }  // for
 | 
						|
      }    // 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::adjust_partition_id_to_continuous(ObTableSchema& table)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
 | 
						|
  if ((PARTITION_LEVEL_ONE == table.get_part_level() || PARTITION_LEVEL_TWO == table.get_part_level()) &&
 | 
						|
      table.is_range_part()) {
 | 
						|
    const int64_t part_num = table.get_part_option().get_part_num();
 | 
						|
    ObPartition** part_array = table.get_part_array();
 | 
						|
    if (OB_ISNULL(part_array)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("NULL ptr", K(ret));
 | 
						|
    } else {
 | 
						|
      table.get_part_option().set_max_used_part_id(part_num - 1);
 | 
						|
      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(ret));
 | 
						|
        } else {
 | 
						|
          part_array[i]->set_part_id(i);
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::truncate_table_in_trans(const ObTableSchema& orig_table_schema,
 | 
						|
    ObIArray<ObTableSchema>& table_schemas, const ObIArray<ObRecycleObject>& index_recycle_objs,
 | 
						|
    ObSchemaGetterGuard& schema_guard, ObMySQLTransaction& trans, const ObString* ddl_stmt_str,
 | 
						|
    const bool to_recyclebin, obrpc::ObCreateTableMode create_mode, const ObString& database_name)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else {
 | 
						|
    bool need_process_failed = false;
 | 
						|
    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();
 | 
						|
    ObArray<const ObObjPriv*> orig_obj_privs_pointer_ora;
 | 
						|
    ObArray<ObObjPriv> orig_obj_privs_ora;
 | 
						|
    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));
 | 
						|
    }
 | 
						|
    // 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<uint64_t>(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 {
 | 
						|
      // 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_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<ObForeignKeyInfo>& 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 && 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 (is_table_in_recyclebin &&
 | 
						|
                     OB_FAIL(ddl_operator.create_index_in_recyclebin(tmp_schema, schema_guard, trans, NULL))) {
 | 
						|
            LOG_WARN("failed to create index schema", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
        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()));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        const int64_t last_schema_version = ddl_operator.get_last_operation_schema_version();
 | 
						|
        if (OB_CREATE_TABLE_MODE_RESTORE == create_mode) {
 | 
						|
          ret = OB_NOT_SUPPORTED;
 | 
						|
          LOG_WARN("truncate table while restore tenant not supported", K(ret));
 | 
						|
        } else {
 | 
						|
          bool gts_on = false;
 | 
						|
          int64_t cur_ts_type = 0;
 | 
						|
          if (OB_FAIL(schema_guard.get_timestamp_service_type(tenant_id, cur_ts_type))) {
 | 
						|
            LOG_WARN("fail to get cur ts type", K(ret));
 | 
						|
          } else {
 | 
						|
            gts_on = transaction::is_ts_type_external_consistent(cur_ts_type);
 | 
						|
          }
 | 
						|
          for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
 | 
						|
            const share::schema::ObTableSchema& this_table = table_schemas.at(i);
 | 
						|
            uint64_t table_id = this_table.get_table_id();
 | 
						|
            bool has_partition = false;
 | 
						|
            if (!is_inner_table(table_id)) {
 | 
						|
              has_partition = this_table.has_partition();
 | 
						|
            } else {
 | 
						|
              has_partition = is_inner_table_with_partition(table_id);
 | 
						|
            }
 | 
						|
            if (!has_partition) {
 | 
						|
            } else if (this_table.is_global_index_table() && !gts_on) {
 | 
						|
              ret = OB_NOT_SUPPORTED;
 | 
						|
              LOG_USER_ERROR(OB_NOT_SUPPORTED, "create global index when gts off");
 | 
						|
            } else if (OB_FAIL(create_or_bind_tables_partitions(schema_guard,
 | 
						|
                           this_table,
 | 
						|
                           last_schema_version,
 | 
						|
                           create_mode,
 | 
						|
                           this_table.get_create_mem_version(),
 | 
						|
                           table_schemas))) {
 | 
						|
              need_process_failed = true;
 | 
						|
              LOG_WARN("fail to create table partitions", K(ret));
 | 
						|
            } else {
 | 
						|
              need_process_failed = true;
 | 
						|
            }
 | 
						|
            if (OB_SUCC(ret)) {
 | 
						|
              if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans, table_id, last_schema_version))) {
 | 
						|
                LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(table_id), K(last_schema_version));
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (to_recyclebin && !is_inner_table(orig_table_schema.get_table_id())) {
 | 
						|
        if (orig_table_schema.get_primary_zone().empty()) {
 | 
						|
          // bypass, truncate a table primary zone derive from upper level
 | 
						|
        } else {
 | 
						|
          if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(
 | 
						|
                  trans, schema_guard, true /*add*/, 1, orig_table_schema.get_tenant_id()))) {
 | 
						|
            LOG_WARN("fail to try modify tenant primary zone entity count", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      } else { /* Did not enter the recycle bin, drop one, create one. equal */
 | 
						|
      }
 | 
						|
    }
 | 
						|
    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) && need_process_failed) {
 | 
						|
      // push schema version
 | 
						|
      int tmp_ret = OB_SUCCESS;
 | 
						|
      if (OB_SUCCESS != (tmp_ret = process_create_partition_failed(tenant_id))) {
 | 
						|
        LOG_ERROR("truncate partition failed, may has garbage partition", K(ret), K(tmp_ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::restore_obj_priv_after_truncation(ObDDLOperator& ddl_operator, ObMySQLTransaction& trans,
 | 
						|
    ObIArray<ObObjPriv>& 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;
 | 
						|
  ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
  UNUSED(table_type);
 | 
						|
  if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
 | 
						|
    LOG_WARN("get_aux_tid_array failed", K(ret));
 | 
						|
  }
 | 
						|
  int64_t N = simple_index_infos.count();
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
 | 
						|
    const ObTableSchema* aux_table_schema = NULL;
 | 
						|
    uint64_t tid = simple_index_infos.at(i).table_id_;
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schema(tid, aux_table_schema))) {
 | 
						|
      LOG_WARN("get_table_schema failed", "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 = %ld",
 | 
						|
        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 = %ld AND %s <> %ld",
 | 
						|
        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_SUCCESS != 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::truncate_table(const ObTruncateTableArg& arg, const int64_t frozen_version)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  ObSchemaService* schema_service = NULL;
 | 
						|
  ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  uint64_t tenant_id = arg.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(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("failed to start trans, ", K(ret));
 | 
						|
  } 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(
 | 
						|
                     orig_table_schema->get_database_id(), is_db_in_recyclebin))) {
 | 
						|
        LOG_WARN("check database in recyclebin failed", K(ret), 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()) {
 | 
						|
        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 {  // 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<ObForeignKeyInfo>& 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<char*>(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<int32_t>(pos), static_cast<int32_t>(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<ObForeignKeyInfo>& 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<char*>(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<int32_t>(pos), static_cast<int32_t>(pos), err_desciption_buf);
 | 
						|
                    LOG_USER_ERROR(OB_ERR_TRUNCATE_ILLEGAL_FK,
 | 
						|
                        truncate_table_err_desciption.length(),
 | 
						|
                        truncate_table_err_desciption.ptr());
 | 
						|
                  }
 | 
						|
                }
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
 | 
						|
        if (OB_SUCC(ret) && orig_table_schema->is_oracle_tmp_table()) {
 | 
						|
          // oracle temp table truncate table is delete where __session_id
 | 
						|
          if (OB_FAIL(truncate_oracle_temp_table(
 | 
						|
                  arg.database_name_, arg.table_name_, arg.tenant_id_, arg.session_id_, 0))) {
 | 
						|
            LOG_WARN("truncate oracle temporary table failed", K(ret));
 | 
						|
          } else { /*do nothing*/
 | 
						|
          }
 | 
						|
        } else if (OB_SUCC(ret)) {
 | 
						|
          ObSArray<ObTableSchema> table_schemas;
 | 
						|
          ObSArray<ObRecycleObject> 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(orig_table_schema->get_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);
 | 
						|
            new_table_schema.set_create_mem_version(frozen_version);
 | 
						|
            if (OB_FAIL(adjust_partition_id_to_continuous(new_table_schema))) {
 | 
						|
              LOG_WARN("failed to adjust_partition_id_to_continuous", K(new_table_schema), K(ret));
 | 
						|
            } else if (OB_FAIL(new_table_schema.clear_dropped_partition())) {
 | 
						|
              LOG_WARN("failed to clear_dropped_partition", K(new_table_schema), K(ret));
 | 
						|
            } else 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<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
            if (OB_FAIL(orig_table_schema->get_simple_index_infos_without_delay_deleted_tid(simple_index_infos))) {
 | 
						|
              LOG_WARN("get simple_index_infos without delay_deleted_tid 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(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_, 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(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_create_mem_version(frozen_version);
 | 
						|
                    // 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<ObRecycleObject> recycle_objs;
 | 
						|
                      uint64_t tenant_id = index_table_schema->get_tenant_id();
 | 
						|
                      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);
 | 
						|
                        index_recycle_objs.push_back(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
 | 
						|
          }
 | 
						|
 | 
						|
          // check table
 | 
						|
          if (OB_SUCC(ret)) {
 | 
						|
            bool to_recyclebin = arg.to_recyclebin_;
 | 
						|
            if (OB_FAIL(truncate_table_in_trans(*orig_table_schema,
 | 
						|
                    table_schemas,
 | 
						|
                    index_recycle_objs,
 | 
						|
                    schema_guard,
 | 
						|
                    trans,
 | 
						|
                    &arg.ddl_stmt_str_,
 | 
						|
                    to_recyclebin,
 | 
						|
                    arg.create_mode_,
 | 
						|
                    arg.database_name_))) {
 | 
						|
              LOG_WARN("truncate table in trans failed", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
          int tmp_ret = OB_SUCCESS;
 | 
						|
          int64_t orig_schema_version = OB_INVALID_VERSION;
 | 
						|
          int64_t cur_schema_version = OB_INVALID_VERSION;
 | 
						|
          if (OB_FAIL(ret)) {
 | 
						|
          } else if (OB_SUCCESS !=
 | 
						|
                     (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(tenant_id, orig_schema_version))) {
 | 
						|
            LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(tenant_id));
 | 
						|
          } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
 | 
						|
            LOG_WARN("publish_schema failed", K(tmp_ret));
 | 
						|
          } else if (OB_CREATE_TABLE_MODE_RESTORE == arg.create_mode_) {
 | 
						|
            // skip
 | 
						|
          } else if (OB_SUCCESS !=
 | 
						|
                     (tmp_ret = schema_service_->get_tenant_refreshed_schema_version(tenant_id, cur_schema_version))) {
 | 
						|
            LOG_WARN("fail to get tenant refreshed schema version", K(tmp_ret), K(tenant_id));
 | 
						|
          } else if (cur_schema_version != orig_schema_version) {
 | 
						|
            if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_table_schema(new_table_schema))) {
 | 
						|
              LOG_WARN("fail to check need wait elect", KR(tmp_ret), K(new_table_schema), K(tenant_id));
 | 
						|
            }
 | 
						|
          }
 | 
						|
          if (OB_SUCC(ret)) {
 | 
						|
            ret = tmp_ret;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }  // else-end
 | 
						|
    }
 | 
						|
  }
 | 
						|
  allocator.clear();
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
/*
 | 
						|
 *  @description:
 | 
						|
 *  According to the schema information, check whether the partition creation needs to
 | 
						|
 *  wait for the leader to report to the meta table
 | 
						|
 *    need wait, wait
 | 
						|
 *    no need wait, skip
 | 
						|
 *  @param[in] table_schema
 | 
						|
 *  @return code
 | 
						|
 */
 | 
						|
int ObDDLService::check_need_wait_leader_by_table_schema(const ObTableSchema& schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", KR(ret));
 | 
						|
  } else {
 | 
						|
    ObLeaderElectionWaiter waiter(*pt_operator_, stopped_);
 | 
						|
    bool has_self_partition = false;
 | 
						|
    if (!is_inner_table(schema.get_table_id())) {
 | 
						|
      has_self_partition = schema.has_self_partition();
 | 
						|
    } else {
 | 
						|
      has_self_partition = is_inner_table_with_partition(schema.get_table_id());
 | 
						|
    }
 | 
						|
    if (!is_sys_table(schema.get_table_id())) {
 | 
						|
      // user table don't wait leader
 | 
						|
    } else if (!has_self_partition) {
 | 
						|
      // user table without partition don't wait leader
 | 
						|
    } else {
 | 
						|
      int64_t timeout_us = THIS_WORKER.get_timeout_remain();
 | 
						|
      const int64_t conf_ddl_timeout = GCONF._ob_ddl_timeout;
 | 
						|
      timeout_us = std::min(conf_ddl_timeout, timeout_us);
 | 
						|
      if (OB_FAIL(waiter.wait(schema, timeout_us))) {
 | 
						|
        LOG_WARN("wait elect leader failed", K(schema), K(timeout_us), KR(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// now all tablegroup no need wait leader, because sys tablegroup don't have partition
 | 
						|
int ObDDLService::check_need_wait_leader_by_tablegroup_schema(const ObTablegroupSchema& tablegroup_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", KR(ret));
 | 
						|
  } else {
 | 
						|
    ObLeaderElectionWaiter waiter(*pt_operator_, stopped_);
 | 
						|
    const uint64_t tablegroup_id = extract_pure_id(tablegroup_schema.get_tablegroup_id());
 | 
						|
    if (!tablegroup_schema.has_self_partition()) {
 | 
						|
      // table without partition, don't wait
 | 
						|
    } else if (tablegroup_id >= OB_USER_TABLEGROUP_ID) {
 | 
						|
      // user tablgroup no need wait
 | 
						|
    } else {
 | 
						|
      int64_t timeout_us = THIS_WORKER.get_timeout_remain();
 | 
						|
      const int64_t conf_ddl_timeout = GCONF._ob_ddl_timeout;
 | 
						|
      timeout_us = std::min(conf_ddl_timeout, timeout_us);
 | 
						|
      if (OB_FAIL(waiter.wait(tablegroup_schema, timeout_us))) {
 | 
						|
        LOG_WARN("wait elect leader failed", K(tablegroup_schema), K(timeout_us), KR(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
/*
 | 
						|
 *  @description:
 | 
						|
 *  According to the table_id and partition_id information, determine whether the partition needs to
 | 
						|
 *  wait for the leader to report to the meta table
 | 
						|
 *    need wait, wait
 | 
						|
 *    no need wait, skip
 | 
						|
 *  @param[in] table_id/partition_id
 | 
						|
 *  @return error code
 | 
						|
 */
 | 
						|
int ObDDLService::check_need_wait_leader_by_id(const uint64_t table_id, const int64_t partition_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", KR(ret));
 | 
						|
  } else if (table_id == OB_INVALID_ID || partition_id == OB_INVALID_ID) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("table_id or partition_id is invalid", KR(ret), K(table_id), K(partition_id));
 | 
						|
  } else {
 | 
						|
    ObLeaderElectionWaiter waiter(*pt_operator_, stopped_);
 | 
						|
    int64_t timeout_us = THIS_WORKER.get_timeout_remain();
 | 
						|
    const int64_t conf_ddl_timeout = GCONF._ob_ddl_timeout;
 | 
						|
    timeout_us = std::min(conf_ddl_timeout, timeout_us);
 | 
						|
    if (OB_FAIL(waiter.wait(table_id, partition_id, timeout_us))) {
 | 
						|
      LOG_WARN("wait elect leader failed", K(table_id), K(timeout_us), K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
 | 
						|
  if (OB_FAIL(schema_guard.get_table_schema(foreign_key_info.child_table_id_, child_table_schema))) {
 | 
						|
    LOG_WARN("fail to get child table schema", K(ret), 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(child_table_schema->get_database_id(), child_db_schema))) {
 | 
						|
    LOG_WARN("failed to get database", K(ret), 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(parent_table_schema.get_database_id(), parent_db_schema))) {
 | 
						|
    SHARE_SCHEMA_LOG(WARN, "failed to get database", K(ret), 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_scheams 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 share::schema::ObTableType table_type_, ObSchemaService& schema_service, const uint64_t frozen_version,
 | 
						|
    ObIArray<ObTableSchema>& new_scheams, ObArenaAllocator& allocator)
 | 
						|
{
 | 
						|
  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;
 | 
						|
  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");
 | 
						|
  } 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_create_mem_version(frozen_version);
 | 
						|
    new_table_schema.reset_foreign_key_infos();
 | 
						|
    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());
 | 
						|
    }
 | 
						|
    if (OB_FAIL(adjust_partition_id_to_continuous(new_table_schema))) {
 | 
						|
      LOG_WARN("failed to adjust_partition_id_to_continuous", K(new_table_schema), K(ret));
 | 
						|
    } else if (OB_FAIL(new_scheams.push_back(new_table_schema))) {
 | 
						|
      LOG_WARN("failed to add table schema!", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    // reconstruct index schema
 | 
						|
    ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
    if (OB_FAIL(new_table_schema.get_simple_index_infos_without_delay_deleted_tid(simple_index_infos))) {
 | 
						|
      LOG_WARN("get simple_index_infos without delay_deleted_tid 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(simple_index_infos.at(i).table_id_, index_table_schema))) {
 | 
						|
        LOG_WARN("get_table_schema failed", K(ret), "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;
 | 
						|
        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());
 | 
						|
            new_index_schema.set_create_mem_version(frozen_version);
 | 
						|
            // create table like, index always is valid
 | 
						|
            new_index_schema.set_index_status(INDEX_STATUS_AVAILABLE);
 | 
						|
            if (OB_FAIL(new_scheams.push_back(new_index_schema))) {
 | 
						|
              LOG_WARN("failed to add table schema!", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }  // end for
 | 
						|
  }
 | 
						|
 | 
						|
  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_;
 | 
						|
  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(
 | 
						|
                     orig_table_schema->get_database_id(), is_db_in_recyclebin))) {
 | 
						|
        LOG_WARN("check database in recyclebin failed", K(ret), 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
 | 
						|
        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<ObTableSchema> table_schemas;
 | 
						|
        if (OB_FAIL(rebuild_table_schema_with_new_id(*orig_table_schema,
 | 
						|
                *new_db_schema,
 | 
						|
                arg.new_table_name_,
 | 
						|
                arg.create_host_,
 | 
						|
                arg.table_type_,
 | 
						|
                *schema_service,
 | 
						|
                frozen_version,
 | 
						|
                table_schemas,
 | 
						|
                allocator))) {
 | 
						|
          LOG_WARN("failed to rebuild table schema with new id");
 | 
						|
        } else if (OB_FAIL(create_user_tables(arg.if_not_exist_,
 | 
						|
                       arg.ddl_stmt_str_,
 | 
						|
                       table_schemas,
 | 
						|
                       frozen_version,
 | 
						|
                       arg.create_mode_,
 | 
						|
                       schema_guard))) {
 | 
						|
          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, bool* is_delay_delete /*NULL*/)
 | 
						|
{
 | 
						|
  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;
 | 
						|
 | 
						|
    if (OB_ISNULL(sql_trans) && OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", KR(ret));
 | 
						|
    } 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));
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (to_recyclebin && !table_schema.is_index_table() && !is_inner_table(table_schema.get_table_id())) {
 | 
						|
        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_FAIL(ret)) {
 | 
						|
          // failed
 | 
						|
        } else if (table_schema.get_primary_zone().empty()) {
 | 
						|
          // bypass, derive from upper layer
 | 
						|
        } else {
 | 
						|
          if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(
 | 
						|
                  trans, schema_guard, false /*sub*/, 1, table_schema.get_tenant_id()))) {
 | 
						|
            LOG_WARN("fail to try modify tenant primary zone entity count", 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,
 | 
						|
                                is_delay_delete))) {
 | 
						|
          LOG_WARN("ddl_operator drop_table failed", K(table_schema), KR(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    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;
 | 
						|
  if (USER_INDEX != table_type) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("table type is invalide", K(ret));
 | 
						|
  } else {
 | 
						|
    ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
    if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) {
 | 
						|
      LOG_WARN("get_aux_tid_array failed", K(ret));
 | 
						|
    }
 | 
						|
    int64_t N = simple_index_infos.count();
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
 | 
						|
      const ObTableSchema* aux_table_schema = NULL;
 | 
						|
      uint64_t tid = simple_index_infos.at(i).table_id_;
 | 
						|
      if (OB_FAIL(schema_guard.get_table_schema(tid, aux_table_schema))) {
 | 
						|
        LOG_WARN("get_table_schema failed", "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 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
 | 
						|
        if (to_recyclebin && !is_inner_table(table_schema.get_table_id())) {
 | 
						|
          if (aux_table_schema->is_in_recyclebin()) {
 | 
						|
            LOG_INFO("aux table is already in recyclebin");
 | 
						|
          } else if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(
 | 
						|
                         *aux_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(*aux_table_schema, trans))) {
 | 
						|
          LOG_WARN("ddl_operator drop_table failed", K(*aux_table_schema), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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));
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin(
 | 
						|
                table_schema, trans, new_db_id, new_table_name, &ddl_stmt_str, guard))) {
 | 
						|
          LOG_WARN("flashback 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;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
  UNUSED(table_type);
 | 
						|
  ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
  if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) {
 | 
						|
    LOG_WARN("get_aux_tid_array failed", K(ret));
 | 
						|
  }
 | 
						|
  int64_t N = simple_index_infos.count();
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
 | 
						|
    const ObTableSchema* aux_table_schema = NULL;
 | 
						|
    uint64_t tid = simple_index_infos.at(i).table_id_;
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schema(tid, aux_table_schema))) {
 | 
						|
      LOG_WARN("get_table_schema failed", "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,
 | 
						|
                   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;
 | 
						|
}
 | 
						|
 | 
						|
// 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<int>(origin_table_name.length()),
 | 
						|
            origin_table_name.ptr(),
 | 
						|
            (OB_SYS_TENANT_ID == tenant_id) ? database_id : extract_pure_id(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, ObString& object_name,
 | 
						|
    common::ObIAllocator* allocator, common::ObMySQLProxy* sql_proxy)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSqlString sql;
 | 
						|
  HEAP_VAR(ObMySQLProxy::MySQLResult, res)
 | 
						|
  {
 | 
						|
    common::sqlclient::ObMySQLResult* result = NULL;
 | 
						|
 | 
						|
    if (OB_FAIL(sql.append_fmt(
 | 
						|
            "select database_id, object_name from oceanbase.__all_recyclebin where original_name = '%.*s' \
 | 
						|
         and database_id = %lu order by gmt_create desc limit 1",
 | 
						|
            static_cast<int>(origin_table_name.length()),
 | 
						|
            origin_table_name.ptr(),
 | 
						|
            (OB_SYS_TENANT_ID == tenant_id) ? database_id : extract_pure_id(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_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,
 | 
						|
                 combine_id(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, object_name, &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,
 | 
						|
                   combine_id(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;
 | 
						|
    share::ObWorker::CompatMode compat_mode = share::ObWorker::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;
 | 
						|
      bool is_oracle_mode = false;
 | 
						|
      ObWorker::CompatMode compat_mode = ObWorker::CompatMode::MYSQL;
 | 
						|
      if (is_virtual_tenant_id(tenant_id) || OB_SYS_TENANT_ID == tenant_id) {
 | 
						|
        compat_mode = ObWorker::CompatMode::MYSQL;
 | 
						|
      } 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 (ObWorker::CompatMode::ORACLE == compat_mode) {
 | 
						|
          is_oracle_mode = true;
 | 
						|
        } else {
 | 
						|
          is_oracle_mode = false;
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (ObString(OB_RECYCLEBIN_SCHEMA_NAME) == arg.new_db_name_ ||
 | 
						|
                 ObSchemaUtils::is_public_database(arg.new_db_name_, is_oracle_mode)) {
 | 
						|
        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) && share::ObWorker::CompatMode::ORACLE == compat_mode) {
 | 
						|
        ObArray<ObSchemaType> 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,
 | 
						|
                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) && share::ObWorker::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_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) && share::ObWorker::CompatMode::ORACLE == compat_mode) {
 | 
						|
        ObArray<ObSchemaType> 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,
 | 
						|
                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()) {
 | 
						|
        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,
 | 
						|
                 combine_id(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(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;
 | 
						|
      bool is_oracle_mode = false;
 | 
						|
      ObWorker::CompatMode compat_mode = ObWorker::CompatMode::MYSQL;
 | 
						|
      if (is_virtual_tenant_id(tenant_id) || OB_SYS_TENANT_ID == tenant_id) {
 | 
						|
        compat_mode = ObWorker::CompatMode::MYSQL;
 | 
						|
      } 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 (ObWorker::CompatMode::ORACLE == compat_mode) {
 | 
						|
          is_oracle_mode = true;
 | 
						|
        } else {
 | 
						|
          is_oracle_mode = false;
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(schema_guard.get_database_schema(data_table_schema->get_database_id(), database_schema))) {
 | 
						|
        LOG_WARN("get database schema failed", K(ret));
 | 
						|
      } 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() ||
 | 
						|
                 ObSchemaUtils::is_public_database(database_schema->get_database_name(), is_oracle_mode)) {
 | 
						|
        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_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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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 {
 | 
						|
      if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin(
 | 
						|
              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_;
 | 
						|
  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,
 | 
						|
                 combine_id(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) {
 | 
						|
    ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
 | 
						|
    LOG_WARN("table is not in recyclebin", K(arg), K(ret));
 | 
						|
  } else if (!table_schema->is_index_table()) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("purge index failed, the table is not index", K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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, int64_t& pz_count, ObMySQLTransaction* pr_trans)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObTableSchema* 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,
 | 
						|
                 combine_id(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) {
 | 
						|
    ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
 | 
						|
    LOG_WARN("table is not in recyclebin", K(arg), K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    if (OB_ISNULL(pr_trans) && OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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));
 | 
						|
    } else if (table_schema->get_primary_zone().empty()) {
 | 
						|
      // derive from upper level
 | 
						|
    } else if (nullptr != pr_trans) {
 | 
						|
      pz_count = 1;
 | 
						|
    } else {
 | 
						|
      if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(
 | 
						|
              trans, schema_guard, false /*sub*/, 1, table_schema->get_tenant_id()))) {
 | 
						|
        LOG_WARN("fail to try modify primary zone entity count", 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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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, int64_t& pz_count, ObMySQLTransaction* pr_trans)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObDatabaseSchema* database_schema = NULL;
 | 
						|
  const uint64_t tenant_id = arg.tenant_id_;
 | 
						|
  ObArray<uint64_t> table_ids;
 | 
						|
  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 if (OB_FAIL(
 | 
						|
                 schema_guard.get_table_ids_in_database(tenant_id, database_schema->get_database_id(), table_ids))) {
 | 
						|
    LOG_WARN("fail to get table ids in database", K(ret), K(arg));
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    pz_count = 0;
 | 
						|
    if (database_schema->get_primary_zone().empty()) {
 | 
						|
      // derive from upper level
 | 
						|
    } else {
 | 
						|
      ++pz_count;
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < table_ids.count(); ++i) {
 | 
						|
      const ObSimpleTableSchemaV2* schema = nullptr;
 | 
						|
      if (OB_FAIL(schema_guard.get_table_schema(table_ids.at(i), schema))) {
 | 
						|
        LOG_WARN("fail to get table schema", K(ret), "table_id", table_ids.at(i));
 | 
						|
      } else if (nullptr == schema) {
 | 
						|
        // bypass
 | 
						|
      } else if (schema->get_primary_zone().empty()) {
 | 
						|
        // bypass
 | 
						|
      } else if (USER_INDEX == schema->get_table_type()) {
 | 
						|
        // bypass, hope some day, pz_value can be removed from oceanbase
 | 
						|
      } else {
 | 
						|
        ++pz_count;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_ISNULL(pr_trans) && OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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));
 | 
						|
    } else if (0 == pz_count) {
 | 
						|
      // no need to do pz_count
 | 
						|
    } else if (nullptr != pr_trans) {
 | 
						|
      // bypass
 | 
						|
    } else {
 | 
						|
      if (OB_FAIL(
 | 
						|
              try_modify_tenant_primary_zone_entity_count(trans, schema_guard, false /*sub*/, pz_count, tenant_id))) {
 | 
						|
        LOG_WARN("fail to try modify primary zone entity count", 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<ObRecycleObject> 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<ObRecycleObject>& 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_);
 | 
						|
  trans.set_end_tenant_id(tenant_id);
 | 
						|
  int64_t accu_pz_count = 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(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("start transaction failed", K(ret));
 | 
						|
  } else {
 | 
						|
    for (int i = 0; OB_SUCC(ret) && i < recycle_objs.count() && purged_objects < arg.purge_num_; ++i) {
 | 
						|
      ddl_stmt.reset();
 | 
						|
      int64_t this_pz_count = 0;
 | 
						|
      const ObRecycleObject& recycle_obj = recycle_objs.at(i);
 | 
						|
      switch (recycle_obj.get_type()) {
 | 
						|
        case ObRecycleObject::INDEX: {
 | 
						|
          // Indexes and VP tables do not enter the recycle bin separately,
 | 
						|
          // and the purge table will synchronize purge index/aux_vp,
 | 
						|
          // 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.table_name_ = recycle_obj.get_object_name();
 | 
						|
            purge_table_arg.ddl_stmt_str_ = ddl_stmt.string();
 | 
						|
            if (OB_FAIL(purge_table(purge_table_arg, this_pz_count, &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, this_pz_count, &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));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      accu_pz_count += this_pz_count;
 | 
						|
      LOG_INFO("purge expire recycle object execpt tenant finished", K(recycle_obj), K(arg), K(ret), K(accu_pz_count));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (accu_pz_count > 0 && OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(
 | 
						|
              trans, schema_guard, false /*SUB*/, accu_pz_count, tenant_id))) {
 | 
						|
        LOG_WARN("fail to try sub tenant primary zone entity count", 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<ObRecycleObject>& 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))) {
 | 
						|
      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)
 | 
						|
{
 | 
						|
  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));
 | 
						|
  }
 | 
						|
 | 
						|
  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<int>(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 list!", 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<uint64_t> 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(mv_ids.at(i), mv))) {
 | 
						|
        LOG_WARN("fail to get table schema", K(ret), 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_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 (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)
 | 
						|
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<const ObSimpleTableSchemaV2*, 512> table_schemas;
 | 
						|
  ObDropTableArg& drop_table_arg = const_cast<ObDropTableArg&>(const_drop_table_arg);
 | 
						|
  if (0 == drop_table_arg.session_id_) {
 | 
						|
    // do nothing, in case of alter system drop tables in session 0
 | 
						|
  } else if (OB_FAIL(schema_service_->get_schema_guard(schema_guard))) {
 | 
						|
    LOG_WARN("fail to get schema guard", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_table_schemas(table_schemas))) {
 | 
						|
    LOG_WARN("fail to get table schema", K(ret));
 | 
						|
  } 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) {
 | 
						|
        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(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 (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));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::force_drop_tablegroup_and_partitions(const uint64_t tablegroup_id,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const common::ObIArray<int64_t>& partition_ids,
 | 
						|
    ObMySQLTransaction& trans)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  bool is_only_delete_part = true;
 | 
						|
  const uint64_t tenant_id = extract_tenant_id(tablegroup_id);
 | 
						|
  const ObTablegroupSchema* orig_tablegroup_schema = NULL;
 | 
						|
  ObTablegroupSchema new_tablegroup_schema;
 | 
						|
  ObTablegroupSchema alter_tablegroup_schema;
 | 
						|
  ObSArray<int64_t> part_ids_for_delay_delete_part;
 | 
						|
  ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
  ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_);
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (partition_ids.count() > 0) {
 | 
						|
      // The partition was deleted with delay, and the tablegroup was not delay delete
 | 
						|
      is_only_delete_part = true;
 | 
						|
    } else {
 | 
						|
      // tablegroup was delay delete
 | 
						|
      is_only_delete_part = false;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_FAIL(schema_guard.get_tablegroup_schema(tablegroup_id, orig_tablegroup_schema))) {
 | 
						|
    LOG_WARN("get tablegroup schema failed", K(ret), K(tablegroup_id));
 | 
						|
  } else if (OB_ISNULL(orig_tablegroup_schema)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("orig_tablegroup_schema is null", K(ret), K(tablegroup_id));
 | 
						|
  } else if (is_only_delete_part) {
 | 
						|
    part_ids_for_delay_delete_part.reset();
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < partition_ids.count(); ++i) {
 | 
						|
      if (OB_FAIL(part_ids_for_delay_delete_part.push_back(partition_ids.at(i)))) {
 | 
						|
        LOG_WARN("push back failed", K(ret), K(partition_ids.at(i)));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_FAIL(new_tablegroup_schema.assign(*orig_tablegroup_schema))) {
 | 
						|
      LOG_WARN("fail to assign schema", K(ret));
 | 
						|
    } else if (!is_new_tablegroup_id(orig_tablegroup_schema->get_tablegroup_id())) {
 | 
						|
      ret = OB_NOT_SUPPORTED;
 | 
						|
      LOG_WARN("shouldn't force drop tablegroup while tablegroup is created before ver 2.0",
 | 
						|
          K(ret),
 | 
						|
          KPC(orig_tablegroup_schema));
 | 
						|
    } else if (OB_FAIL(ddl_operator.add_dropped_part_to_partition_schema(
 | 
						|
                   part_ids_for_delay_delete_part, *orig_tablegroup_schema, alter_tablegroup_schema))) {
 | 
						|
      LOG_WARN("fail to add dropped part to partition schema", K(ret));
 | 
						|
    } else if (orig_tablegroup_schema->get_binding() /*PG need force_drop*/
 | 
						|
               && OB_FAIL(helper.batch_force_drop_table_parts(
 | 
						|
                      tenant_id, schema_guard, orig_tablegroup_schema, alter_tablegroup_schema, trans))) {
 | 
						|
      LOG_WARN("fail to batch force drop table parts", K(ret));
 | 
						|
    } else if (OB_FAIL(ddl_operator.drop_tablegroup_partitions_for_inspection(
 | 
						|
                   *orig_tablegroup_schema, alter_tablegroup_schema, new_tablegroup_schema, trans))) {
 | 
						|
      LOG_WARN("failed to drop table partitions", K(ret));
 | 
						|
    }
 | 
						|
  } else {  // !is_only_delete_part
 | 
						|
    ObArray<uint64_t> table_ids;
 | 
						|
    if (OB_FAIL(schema_guard.get_table_ids_in_tablegroup(
 | 
						|
            tenant_id, orig_tablegroup_schema->get_tablegroup_id(), table_ids))) {
 | 
						|
      LOG_WARN("get tables in database failed", K(ret), K(tenant_id), K(orig_tablegroup_schema->get_tablegroup_id()));
 | 
						|
    } else if (table_ids.count() > 0) {
 | 
						|
      // In order to prevent the tablegroup from being deleted,
 | 
						|
      // if the table in the tablegroup is still there, you need to wait for the table to be deleted first.
 | 
						|
      ret = OB_EAGAIN;
 | 
						|
      LOG_WARN("should delete table schema first", K(ret), K(orig_tablegroup_schema->get_tablegroup_id()));
 | 
						|
    } else if (OB_FAIL(ddl_operator.drop_tablegroup_for_inspection(*orig_tablegroup_schema, trans))) {
 | 
						|
      LOG_WARN("failed to drop tablegroup for inspection", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::force_drop_schema(const obrpc::ObForceDropSchemaArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const uint64_t schema_id = arg.schema_id_;
 | 
						|
  const uint64_t tenant_id = extract_tenant_id(schema_id);
 | 
						|
  ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_INVALID_ID == schema_id) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid arugment", K(schema_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 with version in inner table", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("start transaction failed", K(ret));
 | 
						|
  } else if (TABLE_SCHEMA == arg.type_ &&
 | 
						|
             OB_FAIL(ddl_operator.force_drop_table_and_partitions(
 | 
						|
                 schema_id, schema_guard, arg.partition_ids_, arg.subpartition_ids_, trans))) {
 | 
						|
    LOG_WARN("fail to force drop table and partitions", K(ret));
 | 
						|
  } else if (TABLEGROUP_SCHEMA == arg.type_ &&
 | 
						|
             OB_FAIL(force_drop_tablegroup_and_partitions(schema_id, schema_guard, arg.partition_ids_, trans))) {
 | 
						|
    LOG_WARN("fail to force drop tablegroup and partitions", K(ret));
 | 
						|
  } else if (DATABASE_SCHEMA == arg.type_ &&
 | 
						|
             OB_FAIL(ddl_operator.force_drop_database(schema_id, schema_guard, trans))) {
 | 
						|
    LOG_WARN("fail to force drop database", 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("fail to publish schema", K(ret), K(tenant_id));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  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)
 | 
						|
{
 | 
						|
  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_;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("check_inner_stat error", 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 {
 | 
						|
    bool is_db_in_recyclebin = false;
 | 
						|
    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));
 | 
						|
    }
 | 
						|
    DropTableIdHashSet drop_table_set;
 | 
						|
    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;
 | 
						|
      is_db_in_recyclebin = false;
 | 
						|
      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");
 | 
						|
      } else if (check_tmp_table_only && false == table_schema->is_tmp_table()) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("table_schema should be temporary");
 | 
						|
      } else if (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(
 | 
						|
                     schema_guard.check_database_in_recyclebin(table_schema->get_database_id(), is_db_in_recyclebin))) {
 | 
						|
        LOG_WARN("check database in recyclebin failed", K(ret));
 | 
						|
      } else if (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));
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        ObString ddl_stmt_str;
 | 
						|
        ObSqlString drop_sql;
 | 
						|
        ObWorker::CompatMode compat_mode = ObWorker::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 (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,
 | 
						|
                       ObWorker::CompatMode::ORACLE == compat_mode))) {
 | 
						|
          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)) {
 | 
						|
          bool is_cascade_constrains = false;
 | 
						|
          ObWorker::CompatMode compat_mode = ObWorker::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 (ObWorker::CompatMode::ORACLE == compat_mode) {
 | 
						|
            // oracle cascade constarints use if_exist_ flag
 | 
						|
            is_cascade_constrains = drop_table_arg.if_exist_;
 | 
						|
          }
 | 
						|
          // 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()) {
 | 
						|
            if (table_schema->is_child_table()) {
 | 
						|
              const ObIArray<ObForeignKeyInfo>& foreign_key_infos = 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);
 | 
						|
                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_) {
 | 
						|
                  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
 | 
						|
              const ObIArray<ObForeignKeyInfo>& foreign_key_infos = 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);
 | 
						|
                if (OB_HASH_EXIST == drop_table_set.exist_refactored(foreign_key_info.child_table_id_)) {
 | 
						|
                  // do-nothing
 | 
						|
                } else {
 | 
						|
                  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 {
 | 
						|
            // 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_) {
 | 
						|
            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,
 | 
						|
                    *table_schema,
 | 
						|
                    false,
 | 
						|
                    USER_INDEX == drop_table_arg.table_type_,
 | 
						|
                    to_recyclebin,
 | 
						|
                    &ddl_stmt_str,
 | 
						|
                    NULL,
 | 
						|
                    &drop_table_set))) {
 | 
						|
              LOG_WARN("ddl_service_ drop_table failed", K(table_item), K(tenant_id), K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
        if (OB_SUCC(ret)) {
 | 
						|
          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));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      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;
 | 
						|
        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_SUCC(ret)) {
 | 
						|
      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(err_table_list), K(ret));
 | 
						|
              }
 | 
						|
              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));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    // 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
 | 
						|
    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;
 | 
						|
  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_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(table_schema->get_database_id(), is_db_in_recyclebin))) {
 | 
						|
    LOG_WARN("check database in recyclebin failed", KR(ret));
 | 
						|
  } 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;
 | 
						|
      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, arg.create_mode_, NULL))) {
 | 
						|
        LOG_WARN("ddl_service_ rebuild_index failed", K(tenant_id), KR(ret));
 | 
						|
      } else if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
        LOG_WARN("publish schema failed", KR(ret));
 | 
						|
      } else if (OB_CREATE_TABLE_MODE_RESTORE == arg.create_mode_) {
 | 
						|
        // skip
 | 
						|
      } else if (OB_FAIL(index_builder.submit_build_global_index_task(new_table_schema))) {
 | 
						|
        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();
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  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, const obrpc::ObCreateTableMode create_mode,
 | 
						|
    ObMySQLTransaction* sql_trans, bool* is_delay_delete /*NULL*/)
 | 
						|
{
 | 
						|
  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;
 | 
						|
  if (OB_ISNULL(schema_service)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema_service must not null", KR(ret));
 | 
						|
  } else if (OB_ISNULL(sql_trans) && OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("start transaction failed", KR(ret));
 | 
						|
  } else if (OB_FAIL(drop_table_in_trans(
 | 
						|
                 schema_guard, index_schema, true, true, false, ddl_stmt_str, &trans, NULL, is_delay_delete))) {
 | 
						|
    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(create_table_in_trans(index_schema, frozen_version, create_mode, ddl_stmt_str, &trans))) {
 | 
						|
    LOG_WARN("create_table_in_trans failed", K(index_schema), KR(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  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, const int64_t create_mem_version)
 | 
						|
{
 | 
						|
  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 = extract_tenant_id(table_id);
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_INVALID_ID == table_id || create_mem_version <= 0 || 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), K(create_mem_version));
 | 
						|
  } 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(table_id, table))) {
 | 
						|
    LOG_WARN("get table schema failed", KR(ret), 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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", KR(ret));
 | 
						|
    } else if (OB_FAIL(ddl_operator.update_index_status(
 | 
						|
                   table->get_data_table_id(), table_id, new_status, create_mem_version, 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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObPartitionUnitsArray partition_units_array;
 | 
						|
  ObArray<ObZone> zones;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else {
 | 
						|
    table_schema.set_table_id(combine_id(OB_SYS_TENANT_ID, table_schema.get_table_id()));
 | 
						|
    if (OB_FAIL(create_table_in_trans(table_schema, 0, obrpc::OB_CREATE_TABLE_MODE_INVALID, NULL, NULL))) {
 | 
						|
      LOG_WARN("create_table_in_trans failed", K(table_schema), K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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))) {
 | 
						|
    LOG_WARN("drop table in transaction failed", K(ret), K(table_schema));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::construct_partitions_for_standby(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const common::ObIArray<ObPartitionKey>& keys, const int64_t schema_id, ObPartitionCreator& creator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const ObCreateTableMode create_mode = ObCreateTableMode::OB_CREATE_TABLE_MODE_LOOSE;
 | 
						|
  common::ObArray<ObPartitionAddr> addrs;
 | 
						|
  obrpc::ObCreatePartitionArg arg;
 | 
						|
  share::ObSimpleFrozenStatus frozen_status;
 | 
						|
  frozen_status.frozen_version_ = ObFreezeInfoManager::ORIGIN_FROZEN_VERSION;
 | 
						|
  frozen_status.frozen_timestamp_ = ObFreezeInfoManager::ORIGIN_FROZEN_TIMESTAMP;
 | 
						|
  const int64_t non_paxos_replica_num = 0;
 | 
						|
  int64_t paxos_replica_num = 0;
 | 
						|
  const ObPartitionSchema* schema = NULL;
 | 
						|
  const ObTablegroupSchema* tg_schema = NULL;
 | 
						|
  const ObTableSchema* table_schema = NULL;
 | 
						|
  ObArray<ObTableSchema> available_schemas;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_ISNULL(root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_balancer_ is null", K(ret));
 | 
						|
  } else if (is_new_tablegroup_id(schema_id)) {
 | 
						|
    if (OB_FAIL(schema_guard.get_tablegroup_schema(schema_id, tg_schema))) {
 | 
						|
      LOG_WARN("failed to get tablegroup schema", K(ret), K(schema_id));
 | 
						|
    } else if (OB_ISNULL(tg_schema)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("tablegroup schema is null", KR(ret), K(schema_id));
 | 
						|
    } else {
 | 
						|
      schema = tg_schema;
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    ObTableSchema new_table_schema;
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schema(schema_id, table_schema))) {
 | 
						|
      LOG_WARN("failed to get table schema", KR(ret), K(schema_id));
 | 
						|
    } else if (OB_ISNULL(table_schema)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("table schema is null", KR(ret), K(schema_id));
 | 
						|
    } else if (OB_FAIL(new_table_schema.assign(*table_schema))) {
 | 
						|
      LOG_WARN("failed to assign table schema", KR(ret), KPC(table_schema));
 | 
						|
    } else {
 | 
						|
      new_table_schema.reset_partition_schema();
 | 
						|
      if (OB_FAIL(available_schemas.push_back(new_table_schema))) {
 | 
						|
        LOG_WARN("failed to push back", K(ret), K(new_table_schema));
 | 
						|
      } else {
 | 
						|
        schema = table_schema;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_ISNULL(schema)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema is null", KR(ret), K(schema_id));
 | 
						|
  } else if (!schema->has_self_partition()) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema must have self partition", K(ret));
 | 
						|
  } else if (is_new_tablegroup_id(schema_id)) {
 | 
						|
    if (OB_FAIL(root_balancer_->alloc_tablegroup_partitions_for_standby(
 | 
						|
            *tg_schema, keys, create_mode, addrs, schema_guard))) {
 | 
						|
      LOG_WARN("fail to alloc standby tablegroup partitions for create", K(ret));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    if (OB_FAIL(root_balancer_->alloc_table_partitions_for_standby(
 | 
						|
            *table_schema, keys, create_mode, addrs, schema_guard))) {
 | 
						|
      LOG_WARN("fail to alloc standby table partitions for create", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_FAIL(schema->get_paxos_replica_num(schema_guard, paxos_replica_num))) {
 | 
						|
    LOG_WARN("failed to get paxos replica num", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(keys.count() != addrs.count())) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("addr count not equal to partition key count",
 | 
						|
        K(ret),
 | 
						|
        "addr_count",
 | 
						|
        addrs.count(),
 | 
						|
        "partition_key_count",
 | 
						|
        keys.count());
 | 
						|
  } else {
 | 
						|
    const int64_t schema_version = schema->get_schema_version();
 | 
						|
    const int64_t last_replay_log_id = 0;
 | 
						|
    const bool is_standby = true;
 | 
						|
    const bool is_bootstrap = false;
 | 
						|
    const ObCreateTableMode create_mode = OB_CREATE_TABLE_MODE_LOOSE;
 | 
						|
    obrpc::ObSetMemberListArg member_list_arg;
 | 
						|
    share::ObSplitPartition split_info;
 | 
						|
    const int64_t restore = REPLICA_RESTORE_STANDBY;
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < addrs.count(); ++i) {
 | 
						|
      const ObPartitionKey& key = keys.at(i);
 | 
						|
      const ObPartitionAddr& part_addr = addrs.at(i);
 | 
						|
      if (OB_FAIL(construct_create_partition_creator(key,
 | 
						|
              available_schemas,
 | 
						|
              paxos_replica_num,
 | 
						|
              non_paxos_replica_num,
 | 
						|
              schema_version,
 | 
						|
              last_replay_log_id,
 | 
						|
              create_mode,
 | 
						|
              restore,
 | 
						|
              part_addr,
 | 
						|
              split_info,
 | 
						|
              frozen_status,
 | 
						|
              is_standby,
 | 
						|
              is_bootstrap,
 | 
						|
              creator))) {
 | 
						|
        LOG_WARN("failed to construct create partition arg", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_partitions_for_create(const ObTableSchema& table, const int64_t schema_version,
 | 
						|
    const ObITablePartitionAddr& table_addr, const ObIArray<ObTableSchema>& schemas,
 | 
						|
    const ObCreateTableMode create_mode, const uint64_t last_replay_log_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::schema::ObSchemaGetterGuard schema_guard;
 | 
						|
  int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  int64_t non_paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  const uint64_t tenant_id = table.get_tenant_id();
 | 
						|
  bool is_standby = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("check inner stat failed", K(ret));
 | 
						|
  } else if (!table.is_valid() || table_addr.empty() || schemas.count() < 1) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(table), "addr count", table_addr.count(), K(schemas.count()));
 | 
						|
  } 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_->get_tenant_schema_guard(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(
 | 
						|
                 calc_schema_replica_num(schema_guard, table, paxos_replica_num, non_paxos_replica_num, create_mode))) {
 | 
						|
    LOG_WARN("fail to calc schema replica_num",
 | 
						|
        K(ret),
 | 
						|
        "table_id",
 | 
						|
        table.get_table_id(),
 | 
						|
        K(paxos_replica_num),
 | 
						|
        K(non_paxos_replica_num));
 | 
						|
  } else if (OB_UNLIKELY((!is_standby && paxos_replica_num <= 0) || non_paxos_replica_num < 0)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("paxos_replica_num or non_paxos_replica_num is invalid",
 | 
						|
        K(ret),
 | 
						|
        K(paxos_replica_num),
 | 
						|
        K(non_paxos_replica_num));
 | 
						|
  } else {
 | 
						|
    const bool is_bind = false;
 | 
						|
    const bool need_repeat_create = is_standby && is_sys_table(table.get_table_id());
 | 
						|
    ObPartitionCreator creator(*rpc_proxy_, *pt_operator_, server_mgr_, is_bind, need_repeat_create, sql_proxy_);
 | 
						|
    ObArray<int64_t> part_ids;
 | 
						|
    ObPartIdsGenerator gen(table);
 | 
						|
    share::ObSimpleFrozenStatus frozen_status;
 | 
						|
    int64_t restore = REPLICA_NOT_RESTORE;
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id, frozen_status))) {
 | 
						|
      LOG_WARN("fail to get freeze info", K(ret), K(frozen_status));
 | 
						|
    } else if (OB_FAIL(gen.gen(part_ids))) {
 | 
						|
      LOG_WARN("generate part ids failed", K(ret));
 | 
						|
    } else if (OB_FAIL(prepare_create_partitions(creator,
 | 
						|
                   table,
 | 
						|
                   table.get_table_id(),
 | 
						|
                   schema_version,
 | 
						|
                   table.get_all_part_num(),
 | 
						|
                   table.get_partition_cnt(),
 | 
						|
                   paxos_replica_num,
 | 
						|
                   non_paxos_replica_num,
 | 
						|
                   part_ids,
 | 
						|
                   table_addr,
 | 
						|
                   schemas,
 | 
						|
                   false,
 | 
						|
                   is_standby,
 | 
						|
                   create_mode,
 | 
						|
                   restore,
 | 
						|
                   frozen_status,
 | 
						|
                   last_replay_log_id))) {
 | 
						|
      LOG_WARN("prepare create partitions failed", K(ret));
 | 
						|
    } else if (OB_FAIL(creator.execute())) {
 | 
						|
      LOG_WARN("execute create partition failed", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_partitions_for_add(const int64_t schema_version, const ObTableSchema& table,
 | 
						|
    const int64_t inc_partition_cnt, const ObITablePartitionAddr& table_addr, const ObIArray<int64_t>& partition_ids,
 | 
						|
    const ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  ObTableSchema new_schema;
 | 
						|
  return create_partitions_for_split(
 | 
						|
      schema_version, table, inc_partition_cnt, table_addr, new_schema, partition_ids, create_mode);
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_partitions_for_split(const int64_t schema_version, const ObTableSchema& table,
 | 
						|
    const int64_t inc_partition_cnt, const ObITablePartitionAddr& table_addr, const ObTableSchema& new_table,
 | 
						|
    const ObIArray<int64_t>& partition_ids, const ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  int64_t non_paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  const uint64_t tenant_id = table.get_tenant_id();
 | 
						|
  bool is_standby = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("check inner stat failed", K(ret));
 | 
						|
  } else if (!table.is_valid() || table_addr.empty()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(table), "addr count", table_addr.count());
 | 
						|
  } else if (OB_FAIL(get_is_standby_cluster(is_standby))) {
 | 
						|
    LOG_WARN("failed to get is standby cluster", 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(
 | 
						|
                 calc_schema_replica_num(schema_guard, table, paxos_replica_num, non_paxos_replica_num, create_mode))) {
 | 
						|
    LOG_WARN("fail to calc schema replica_num",
 | 
						|
        K(ret),
 | 
						|
        "table_id",
 | 
						|
        table.get_table_id(),
 | 
						|
        K(paxos_replica_num),
 | 
						|
        K(non_paxos_replica_num));
 | 
						|
  } else if (OB_UNLIKELY((!is_standby && paxos_replica_num <= 0) || non_paxos_replica_num < 0)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("paxos_replica_num or non_paxos_replica_num is invalid",
 | 
						|
        K(ret),
 | 
						|
        K(paxos_replica_num),
 | 
						|
        K(non_paxos_replica_num));
 | 
						|
  } else {
 | 
						|
    ObArray<ObTableSchema> schemas;
 | 
						|
    ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
 | 
						|
    const ObTableSchema* index_schema = NULL;
 | 
						|
 | 
						|
    if (OB_FAIL(schemas.push_back(table))) {
 | 
						|
      STORAGE_LOG(WARN, "fail to push back main table", K(ret));
 | 
						|
    } else if (OB_FAIL(table.get_simple_index_infos(simple_index_infos))) {
 | 
						|
      STORAGE_LOG(WARN, "fail to get index tid array", K(ret));
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
 | 
						|
      if (OB_FAIL(schema_guard.get_table_schema(simple_index_infos.at(i).table_id_, index_schema))) {
 | 
						|
        STORAGE_LOG(WARN, "fail to get table schema", K(ret));
 | 
						|
      } else if (OB_ISNULL(index_schema)) {
 | 
						|
        ret = OB_TABLE_NOT_EXIST;
 | 
						|
        STORAGE_LOG(WARN, "error unexpected, index schema must not be NULL", K(ret));
 | 
						|
      } else if (OB_FAIL(schemas.push_back(*index_schema))) {
 | 
						|
        STORAGE_LOG(WARN, "fail to push back index schema", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    share::ObSimpleFrozenStatus frozen_status;
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id, frozen_status))) {
 | 
						|
      LOG_WARN("fail to get freeze info", K(ret), K(frozen_status));
 | 
						|
    }
 | 
						|
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      ObPartitionCreator creator(*rpc_proxy_, *pt_operator_, server_mgr_);
 | 
						|
 | 
						|
      // all_part_num contains the number of all partitions before and after the split;
 | 
						|
      const int64_t all_part_num = table.get_all_part_num() + inc_partition_cnt;
 | 
						|
      int64_t restore = REPLICA_NOT_RESTORE;
 | 
						|
      if (OB_FAIL(prepare_create_partitions(creator,
 | 
						|
              new_table,
 | 
						|
              table.get_table_id(),
 | 
						|
              schema_version,
 | 
						|
              all_part_num,
 | 
						|
              table.get_partition_cnt(),
 | 
						|
              paxos_replica_num,
 | 
						|
              non_paxos_replica_num,
 | 
						|
              partition_ids,
 | 
						|
              table_addr,
 | 
						|
              schemas,
 | 
						|
              false,
 | 
						|
              is_standby,
 | 
						|
              create_mode,
 | 
						|
              restore,
 | 
						|
              frozen_status))) {
 | 
						|
        LOG_WARN("prepare create partitions failed", K(ret));
 | 
						|
      } else if (OB_FAIL(creator.execute())) {
 | 
						|
        LOG_WARN("execute create partition failed", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::prepare_create_partitions(ObPartitionCreator& creator, const uint64_t table_id,
 | 
						|
    const int64_t schema_version, const int64_t partition_num, const int64_t partition_cnt,
 | 
						|
    const int64_t paxos_replica_num, const int64_t non_paxos_replica_num, const ObIArray<int64_t>& partition_ids,
 | 
						|
    const ObITablePartitionAddr& table_addr, const ObIArray<ObTableSchema>& schemas, const bool is_bootstrap,
 | 
						|
    const bool is_standby, ObCreateTableMode create_mode, const int64_t restore,
 | 
						|
    const ObSimpleFrozenStatus& frozen_status, const uint64_t last_replay_log_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObTableSchema* new_schema = NULL;
 | 
						|
  ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
 | 
						|
  if (OB_FAIL(ObSchemaUtils::alloc_schema(allocator, new_schema))) {
 | 
						|
    LOG_WARN("fail to alloc schema", KR(ret));
 | 
						|
  } else if (OB_FAIL(prepare_create_partitions(creator,
 | 
						|
                 *new_schema,
 | 
						|
                 table_id,
 | 
						|
                 schema_version,
 | 
						|
                 partition_num,
 | 
						|
                 partition_cnt,
 | 
						|
                 paxos_replica_num,
 | 
						|
                 non_paxos_replica_num,
 | 
						|
                 partition_ids,
 | 
						|
                 table_addr,
 | 
						|
                 schemas,
 | 
						|
                 is_bootstrap,
 | 
						|
                 is_standby,
 | 
						|
                 create_mode,
 | 
						|
                 restore,
 | 
						|
                 frozen_status,
 | 
						|
                 last_replay_log_id))) {
 | 
						|
    LOG_WARN("fail to prepare create partition", KR(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// In order to reuse the code, this function and sub-functions have been modified
 | 
						|
// so that it does not care whether the upper layer is create table or alter table add partition
 | 
						|
// 1. both create table and alter table add partition use this interface
 | 
						|
// 2. The upper layer needs to generate the corresponding data,
 | 
						|
//  this function does not care whether the upper layer is create or add
 | 
						|
// 3. partition_num is the number of all partition
 | 
						|
int ObDDLService::prepare_create_partitions(ObPartitionCreator& creator, const ObTableSchema& new_schema,
 | 
						|
    const uint64_t table_id, const int64_t schema_version, const int64_t partition_num, const int64_t partition_cnt,
 | 
						|
    const int64_t paxos_replica_num, const int64_t non_paxos_replica_num, const ObIArray<int64_t>& partition_ids,
 | 
						|
    const ObITablePartitionAddr& table_addr, const ObIArray<ObTableSchema>& schemas, const bool is_bootstrap,
 | 
						|
    const bool is_standby, ObCreateTableMode create_mode, const int64_t restore,
 | 
						|
    const ObSimpleFrozenStatus& frozen_status, const uint64_t last_replay_log_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(partition_num < 1) || schemas.count() < 1) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid partition num", K(ret), K(table_addr.count()), K(partition_num), K(partition_ids.count()));
 | 
						|
  } else if (!frozen_status.is_valid()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid frozen status", K(ret), K(frozen_status));
 | 
						|
  } else if (new_schema.get_part_option().get_part_num() != table_addr.count() &&
 | 
						|
             partition_ids.count() != table_addr.count()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid partition num",
 | 
						|
        K(ret),
 | 
						|
        K(table_addr.count()),
 | 
						|
        K(new_schema.get_part_option()),
 | 
						|
        K(partition_ids.count()));
 | 
						|
  } 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 {
 | 
						|
    ObArray<ObTableSchema> available_schemas;
 | 
						|
    ObTableSchema tmp_schema;
 | 
						|
    LOG_DEBUG("create partitions",
 | 
						|
        K(table_addr),
 | 
						|
        K(new_schema),
 | 
						|
        K(partition_num),
 | 
						|
        K(partition_cnt),
 | 
						|
        K(table_id),
 | 
						|
        K(paxos_replica_num),
 | 
						|
        K(partition_ids),
 | 
						|
        K(schema_version));
 | 
						|
    if (USER_INDEX == new_schema.get_table_type() && new_schema.is_global_index_table()) {
 | 
						|
      if (OB_FAIL(tmp_schema.assign(new_schema))) {
 | 
						|
        LOG_WARN("fail to assign schema", K(ret));
 | 
						|
      } else {
 | 
						|
        tmp_schema.reset_partition_schema();
 | 
						|
      }
 | 
						|
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(available_schemas.push_back(tmp_schema))) {
 | 
						|
        LOG_WARN("fail to push back", K(ret));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < schemas.count(); ++i) {
 | 
						|
        if (is_sys_table(schemas.at(i).get_data_table_id())) {
 | 
						|
          // index of system table don't build partition
 | 
						|
        } else if (0 == i || (is_available_index_status(schemas.at(i).get_index_status()) &&
 | 
						|
                                 !schemas.at(i).has_self_partition())) {
 | 
						|
          // don't add in global index
 | 
						|
          tmp_schema.reset();
 | 
						|
          if (OB_FAIL(tmp_schema.assign(schemas.at(i)))) {
 | 
						|
            LOG_WARN("fail to assign schema", K(ret));
 | 
						|
          } else {
 | 
						|
            tmp_schema.reset_partition_schema();
 | 
						|
          }
 | 
						|
          uint64_t table_id = tmp_schema.get_table_id();
 | 
						|
          if (OB_FAIL(ret)) {
 | 
						|
          } else if (OB_FAIL(available_schemas.push_back(tmp_schema))) {
 | 
						|
            STORAGE_LOG(WARN, "fail to push back schema", K(ret));
 | 
						|
          } else if (OB_FAIL(add_sys_table_index(table_id, available_schemas))) {
 | 
						|
            LOG_WARN("fail to add sys table index", K(ret), K(table_id));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    // if (OB_FAIL(ret) || is_bootstrap) {
 | 
						|
    //  if (OB_FAIL(freeze_info_manager_->get_freeze_info(1, frozen_status))) {
 | 
						|
    //    LOG_WARN("fail to get freeze info", KR(ret));
 | 
						|
    //  }
 | 
						|
    //} else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id,
 | 
						|
    //                                                                                frozen_status))) {
 | 
						|
    //  LOG_WARN("fail to get freeze info", K(ret), K(frozen_status));
 | 
						|
    //}
 | 
						|
    RS_TRACE(fill_flag_replica_begin);
 | 
						|
    share::ObSplitPartition split_info;
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (new_schema.is_in_splitting()) {
 | 
						|
      if (OB_FAIL(ObPartitionSplitHelper::build_split_info(new_schema, split_info))) {
 | 
						|
        LOG_WARN("failed to build split info", K(ret), K(split_info), K(new_schema));
 | 
						|
      } else if (!split_info.is_valid()) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("split info is invalid", K(ret), K(split_info), K(new_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < table_addr.count(); ++i) {
 | 
						|
      const ObPartitionAddr& part_addr = table_addr.at(i);
 | 
						|
      int64_t partition_id = OB_INVALID_PARTITION_ID;
 | 
						|
      if (partition_ids.count() > 0) {
 | 
						|
        partition_id = partition_ids.at(i);
 | 
						|
        const ObPartition* partition = NULL;
 | 
						|
        bool check_dropped_partition = false;
 | 
						|
        // Because subpartition split is not supported, partition_id is equivalent to part_id, which needs to be
 | 
						|
        // modified later
 | 
						|
        if (!new_schema.is_in_splitting()) {
 | 
						|
          // nothing todo
 | 
						|
        } else if (OB_FAIL(new_schema.get_partition_by_part_id(partition_id, check_dropped_partition, partition))) {
 | 
						|
          LOG_WARN("fail to get partition", K(ret), K(partition_id));
 | 
						|
        } else if (OB_ISNULL(partition)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("get invalid partition", K(ret), K(partition));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      } else if (OB_ISNULL(new_schema.get_part_array()) || OB_ISNULL(new_schema.get_part_array()[i])) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("invalid new schema", K(ret), K(new_schema));
 | 
						|
      } else {
 | 
						|
        // FIXME:() Can you ensure that the secondary partition does not reach this logic???
 | 
						|
        partition_id = new_schema.get_part_array()[i]->get_part_id();
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        ObPartitionKey pkey;
 | 
						|
        if (OB_FAIL(pkey.init(table_id, partition_id, partition_cnt))) {
 | 
						|
          LOG_WARN("failed to init pkey", K(ret), K(table_id), K(partition_cnt));
 | 
						|
        } else if (OB_FAIL(construct_create_partition_creator(pkey,
 | 
						|
                       available_schemas,
 | 
						|
                       paxos_replica_num,
 | 
						|
                       non_paxos_replica_num,
 | 
						|
                       schema_version,
 | 
						|
                       last_replay_log_id,
 | 
						|
                       create_mode,
 | 
						|
                       restore,
 | 
						|
                       part_addr,
 | 
						|
                       split_info,
 | 
						|
                       frozen_status,
 | 
						|
                       is_standby,
 | 
						|
                       is_bootstrap,
 | 
						|
                       creator))) {
 | 
						|
          LOG_WARN("failed to construct create partition arg", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    RS_TRACE(fill_flag_replica_end);
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::construct_create_partition_creator(const common::ObPartitionKey& pkey,
 | 
						|
    const common::ObIArray<share::schema::ObTableSchema>& schemas, const int64_t paxos_replica_num,
 | 
						|
    const int64_t non_paxos_replica_num, const int64_t schema_version, const int64_t last_replay_log_id,
 | 
						|
    const obrpc::ObCreateTableMode create_mode, const int64_t restore, const ObPartitionAddr& part_addr,
 | 
						|
    const share::ObSplitPartition& split_info, const share::ObSimpleFrozenStatus& frozen_status, const bool is_standby,
 | 
						|
    const bool is_bootstrap, ObPartitionCreator& creator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else {
 | 
						|
    obrpc::ObCreatePartitionArg arg;
 | 
						|
    obrpc::ObSetMemberListArg member_list_arg;
 | 
						|
    ObPartitionReplica flag_replica;
 | 
						|
    arg.schema_version_ = schema_version;
 | 
						|
    arg.last_replay_log_id_ = last_replay_log_id;
 | 
						|
    const int64_t now = ObTimeUtility::current_time();
 | 
						|
    bool partition_already_exist = true;
 | 
						|
    const int64_t table_id = pkey.get_table_id();
 | 
						|
    const int64_t partition_id = pkey.get_partition_id();
 | 
						|
    const int64_t partition_cnt = pkey.get_partition_cnt();
 | 
						|
    // Check whether the partition already has a persistent member list. If it already exists,
 | 
						|
    // it indicates that the partition has been created successfully and does not need to be recreated
 | 
						|
    if (OB_FAIL(creator.check_partition_already_exist(
 | 
						|
            pkey.get_table_id(), pkey.get_partition_id(), paxos_replica_num, partition_already_exist))) {
 | 
						|
      LOG_WARN("failed to check need create partition", K(ret), K(pkey));
 | 
						|
    } else if (partition_already_exist) {
 | 
						|
      LOG_INFO("partition already exist", K(pkey), K(paxos_replica_num));
 | 
						|
    } else {
 | 
						|
      LOG_DEBUG("prepare create partition", "partkey", pkey, "addr", part_addr);
 | 
						|
      if (OB_FAIL(arg.table_schemas_.assign(schemas))) {
 | 
						|
        LOG_WARN("fail to assign table schemas", K(ret));
 | 
						|
      } else if (OB_FAIL(fill_partition_member_list(part_addr, now, paxos_replica_num, arg))) {
 | 
						|
        LOG_WARN("fail to fill create partition arg memberlist", KR(ret), K(arg), K(paxos_replica_num));
 | 
						|
      } else {
 | 
						|
        // OB_CREATE_TABLE_MODE_PHYSICAL_RESTORE is only used to control the initial restore state of the physical
 | 
						|
        // restoration system table partition, Its path should be consistent with OB_CREATE_TABLE_MODE_LOOSE
 | 
						|
        obrpc::ObCreateTableMode mode =
 | 
						|
            OB_CREATE_TABLE_MODE_PHYSICAL_RESTORE == create_mode ? OB_CREATE_TABLE_MODE_LOOSE : create_mode;
 | 
						|
        creator.set_create_mode(mode);
 | 
						|
        if (OB_CREATE_TABLE_MODE_PHYSICAL_RESTORE == create_mode) {
 | 
						|
          // restore tenant, need reset sys table member_list
 | 
						|
          if (REPLICA_NOT_RESTORE != restore) {
 | 
						|
            arg.member_list_.reset();
 | 
						|
          }
 | 
						|
        } else if (creator.is_ignore_member_list()) {
 | 
						|
          if (OB_FAIL(member_list_arg.init(
 | 
						|
                  table_id, partition_id, partition_cnt, arg.member_list_, paxos_replica_num, now, arg.leader_))) {
 | 
						|
            LOG_WARN("failed to init member list", K(ret), K(partition_id), K(table_id), K(arg));
 | 
						|
          } else if (OB_FAIL(creator.build_member_list_map(member_list_arg))) {
 | 
						|
            LOG_WARN("failed to push back arg", K(ret), K(member_list_arg));
 | 
						|
          } else {
 | 
						|
            arg.member_list_.reset();
 | 
						|
            arg.ignore_member_list_ = true;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      FOREACH_CNT_X(a, part_addr, OB_SUCC(ret))
 | 
						|
      {
 | 
						|
        if (OB_ISNULL(a) || OB_ISNULL(rpc_proxy_)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("a or rpc_proxy_ is null", K(ret));
 | 
						|
        } else if (OB_INVALID_INDEX == partition_id) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("generate a invalid partition id", K(ret));
 | 
						|
        } else if (is_standby && !ObMultiClusterUtil::is_cluster_private_table(table_id) &&
 | 
						|
                   !common::ObReplicaTypeCheck::is_paxos_replica_V2(a->replica_type_)) {
 | 
						|
          // No need to create a non-paxos copy of the standby database,
 | 
						|
          // there is no persistent member list, standby_restore will have problems
 | 
						|
        } else {
 | 
						|
          common::ObRole role = FOLLOWER;
 | 
						|
          int64_t new_restore = restore;
 | 
						|
          if (OB_CREATE_TABLE_MODE_RESTORE == create_mode &&
 | 
						|
              ObReplicaTypeCheck::is_replica_with_ssstore(a->replica_type_)) {
 | 
						|
            // logical restore
 | 
						|
            new_restore = REPLICA_LOGICAL_RESTORE_DATA;
 | 
						|
          }
 | 
						|
          if (OB_FAIL(set_flag_role(a->initial_leader_, is_standby, new_restore, table_id, role))) {
 | 
						|
            LOG_WARN("fail to set flag role", KR(ret), K(is_standby), K(restore), K(new_restore), K(table_id), K(role));
 | 
						|
          } else if (OB_FAIL(fill_create_partition_arg(table_id,
 | 
						|
                         partition_cnt,
 | 
						|
                         paxos_replica_num,
 | 
						|
                         non_paxos_replica_num,
 | 
						|
                         partition_id,
 | 
						|
                         *a,
 | 
						|
                         now,
 | 
						|
                         is_bootstrap,
 | 
						|
                         is_standby,
 | 
						|
                         new_restore,
 | 
						|
                         frozen_status,
 | 
						|
                         arg))) {
 | 
						|
            LOG_WARN("fail to fill ObCreatePartitionArg",
 | 
						|
                K(ret),
 | 
						|
                K(table_id),
 | 
						|
                K(partition_id),
 | 
						|
                K(partition_cnt),
 | 
						|
                K(frozen_status),
 | 
						|
                K(paxos_replica_num),
 | 
						|
                "replica_addr",
 | 
						|
                *a,
 | 
						|
                "timestamp",
 | 
						|
                now);
 | 
						|
          } else if (OB_FAIL(fill_flag_replica(table_id, partition_cnt, partition_id, arg, *a, flag_replica))) {
 | 
						|
            LOG_WARN("fail to fill flag replica",
 | 
						|
                K(ret),
 | 
						|
                K(table_id),
 | 
						|
                K(partition_id),
 | 
						|
                K(partition_cnt),
 | 
						|
                K(arg),
 | 
						|
                "replica addr",
 | 
						|
                *a);
 | 
						|
          } else if (OB_FAIL(creator.add_flag_replica(flag_replica))) {
 | 
						|
            LOG_WARN("add flag replica to partition creator failed", K(ret), K(flag_replica));
 | 
						|
          } else if (OB_FAIL(arg.split_info_.assign(split_info))) {
 | 
						|
            LOG_WARN("failed to assign split pair", K(ret), K(split_info), K(arg));
 | 
						|
          } else if (OB_FAIL(creator.add_create_partition_arg(a->addr_, arg))) {
 | 
						|
            LOG_WARN("add create partition argument failed", K(ret));
 | 
						|
          } else {
 | 
						|
            LOG_DEBUG("add create partition arg", K(arg), K(a->addr_));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
/*
 | 
						|
//FIXME: part_id is not equal to part_idx, try not to use this function
 | 
						|
int ObDDLService::generate_partition_id(
 | 
						|
    const share::schema::ObTableSchema &table,
 | 
						|
    const int64_t partition_idx,
 | 
						|
    int64_t &partition_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_UNLIKELY(partition_idx < 0)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(partition_idx));
 | 
						|
  } else {
 | 
						|
    const int64_t sub_part_num = table.get_sub_part_num();
 | 
						|
    const ObPartitionLevel part_level = table.get_part_level();
 | 
						|
    int64_t part_id = partition_idx;
 | 
						|
    int64_t sub_part_id = 0;
 | 
						|
    if (PARTITION_LEVEL_TWO == part_level && 0 != sub_part_num) {
 | 
						|
      part_id = partition_idx / sub_part_num;
 | 
						|
      sub_part_id = partition_idx % sub_part_num;
 | 
						|
    }
 | 
						|
    partition_id = generate_phy_part_id(part_id, sub_part_id, part_level);
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}*/
 | 
						|
 | 
						|
int ObDDLService::fill_create_binding_partition_arg(const common::ObPartitionKey& pkey, const common::ObPGKey& pgkey,
 | 
						|
    const common::ObZone& zone, const common::ObIArray<share::schema::ObTableSchema>& schemas,
 | 
						|
    const int64_t paxos_replica_num, const int64_t non_paxos_replica_num, const int64_t schema_version,
 | 
						|
    const int64_t frozen_version, const int64_t frozen_timestamp, const obrpc::ObCreateTableMode create_mode,
 | 
						|
    obrpc::ObCreatePartitionArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_UNLIKELY(!pkey.is_valid() || !pgkey.is_valid())) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(pkey), K(pgkey));
 | 
						|
  } else if (OB_CREATE_TABLE_MODE_PHYSICAL_RESTORE == create_mode) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("create mode not supported yet", K(ret), K(create_mode));
 | 
						|
  } else {
 | 
						|
    arg.partition_key_ = pkey;
 | 
						|
    arg.pg_key_ = pgkey;
 | 
						|
    arg.zone_ = zone;
 | 
						|
    arg.replica_num_ = paxos_replica_num;
 | 
						|
    arg.memstore_version_ = frozen_version + 1;
 | 
						|
    arg.lease_start_ = ObTimeUtility::current_time();  // dummy
 | 
						|
    arg.logonly_replica_num_ = 0;
 | 
						|
    arg.backup_replica_num_ = 0;
 | 
						|
    arg.readonly_replica_num_ = 0;
 | 
						|
    // TODO: create binding partition is partition level, not replica level,
 | 
						|
    // The copy type/memstore percent should not be filled in. Due to implementation problems, fill in FULL/100% first
 | 
						|
    arg.replica_type_ = ObReplicaType::REPLICA_TYPE_FULL;
 | 
						|
    (void)arg.set_memstore_percent(100 /* dummy memstore percent */);
 | 
						|
    arg.last_submit_timestamp_ = arg.lease_start_;
 | 
						|
    arg.non_paxos_replica_num_ = non_paxos_replica_num;
 | 
						|
    arg.frozen_timestamp_ = frozen_timestamp;
 | 
						|
    arg.restore_ = 0;
 | 
						|
    arg.schema_version_ = schema_version;
 | 
						|
    arg.last_replay_log_id_ = 0;  // dummy
 | 
						|
    if (OB_FAIL(arg.table_schemas_.assign(schemas))) {
 | 
						|
      LOG_WARN("fail to assign schemas", K(ret));
 | 
						|
    }
 | 
						|
    if (OB_CREATE_TABLE_MODE_RESTORE == create_mode) {
 | 
						|
      // logical restore
 | 
						|
      arg.restore_ = REPLICA_LOGICAL_RESTORE_DATA;
 | 
						|
    } else {
 | 
						|
      arg.restore_ = REPLICA_NOT_RESTORE;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::fill_create_partition_arg(const uint64_t table_id, const int64_t partition_cnt,
 | 
						|
    const int64_t paxos_replica_num, const int64_t non_paxos_replica_num, const int64_t partition_id,
 | 
						|
    const ObReplicaAddr& replica_addr, const int64_t lease_start_ts, const bool is_bootstrap, const bool is_standby,
 | 
						|
    const int64_t restore, const share::ObSimpleFrozenStatus& frozen_status, obrpc::ObCreatePartitionArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_UNLIKELY(OB_INVALID_INDEX == partition_id) || OB_UNLIKELY(lease_start_ts < 0) ||
 | 
						|
      OB_UNLIKELY(non_paxos_replica_num < 0) || !frozen_status.is_valid()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument",
 | 
						|
        K(ret),
 | 
						|
        K(partition_id),
 | 
						|
        K(paxos_replica_num),
 | 
						|
        K(non_paxos_replica_num),
 | 
						|
        K(lease_start_ts),
 | 
						|
        K(frozen_status));
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
    // nothing todo
 | 
						|
  } else {
 | 
						|
    arg.zone_ = replica_addr.zone_;
 | 
						|
    arg.replica_num_ = paxos_replica_num;
 | 
						|
    arg.memstore_version_ = frozen_status.frozen_version_ + 1;
 | 
						|
    // 1. for table created from observer before 1.4.3(exclude 1.4.3), partiton_cnt will be the cnt when table created
 | 
						|
    // 2. other, partition_cnt will be 0
 | 
						|
    arg.partition_key_.init(table_id, partition_id, partition_cnt);
 | 
						|
    arg.pg_key_.init(table_id, partition_id, partition_cnt);
 | 
						|
    // lease start time may be overwrite in ObPartitionCreator
 | 
						|
    arg.lease_start_ = lease_start_ts;
 | 
						|
    // logonly, backup, readonly replica num should be load from schema afterwards
 | 
						|
    // set them to zero temporarily
 | 
						|
    arg.logonly_replica_num_ = 0;
 | 
						|
    arg.backup_replica_num_ = 0;
 | 
						|
    arg.readonly_replica_num_ = 0;
 | 
						|
    arg.replica_type_ = replica_addr.replica_type_;
 | 
						|
    if (OB_FAIL(arg.set_memstore_percent(replica_addr.get_memstore_percent()))) {
 | 
						|
      LOG_WARN("fail to set memstore percent", K(ret));
 | 
						|
    } else {
 | 
						|
      if (restore >= share::REPLICA_RESTORE_DATA) {
 | 
						|
        // The last_submit_ts in the table creation request during the physical recovery period is set to 0
 | 
						|
        // to avoid being larger than the value in clog
 | 
						|
        arg.last_submit_timestamp_ = 0;
 | 
						|
      } else {
 | 
						|
        arg.last_submit_timestamp_ = lease_start_ts;
 | 
						|
      }
 | 
						|
      arg.non_paxos_replica_num_ = non_paxos_replica_num;
 | 
						|
      arg.restore_ = restore;
 | 
						|
      if (is_bootstrap && !is_standby) {
 | 
						|
        arg.frozen_timestamp_ = 1;
 | 
						|
      } else if (share::REPLICA_RESTORE_STANDBY == restore) {
 | 
						|
        // The copy of the restore state cannot set frozen_timestamp, which may be larger than the source
 | 
						|
        arg.frozen_timestamp_ = 1;
 | 
						|
      } else {
 | 
						|
        arg.frozen_timestamp_ = frozen_status.frozen_timestamp_;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::set_flag_role(const bool initial_leader, const bool is_standby, const int64_t restore,
 | 
						|
    const uint64_t table_id, common::ObRole& role)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", KR(ret));
 | 
						|
  } else {
 | 
						|
    if (initial_leader && !is_standby && REPLICA_NOT_RESTORE == restore && !is_sys_table(table_id)) {
 | 
						|
      // When setting the flag role, only the non-restore leader of the primary cluster is considered,
 | 
						|
      // and the rest are reported normally
 | 
						|
      role = LEADER;
 | 
						|
    } else {
 | 
						|
      role = FOLLOWER;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::fill_flag_replica(const uint64_t table_id, const int64_t partition_cnt, const int64_t partition_id,
 | 
						|
    const ObCreatePartitionArg& arg, const ObReplicaAddr& replica_addr, share::ObPartitionReplica& flag_replica)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(partition_id < 0) || OB_UNLIKELY(partition_cnt < 0) || OB_UNLIKELY(!arg.is_valid())) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(partition_id), K(partition_cnt), K(arg));
 | 
						|
  } else {
 | 
						|
    flag_replica.reset();
 | 
						|
    if (OB_FAIL(pt_operator_->gen_flag_replica(table_id,
 | 
						|
            partition_id,
 | 
						|
            partition_cnt,
 | 
						|
            replica_addr.addr_,
 | 
						|
            replica_addr.zone_,
 | 
						|
            replica_addr.unit_id_,
 | 
						|
            replica_addr.replica_type_,
 | 
						|
            replica_addr.get_memstore_percent(),
 | 
						|
            arg.replica_num_,
 | 
						|
            flag_replica))) {
 | 
						|
      LOG_WARN("gen flag replica failed",
 | 
						|
          K(ret),
 | 
						|
          "table id",
 | 
						|
          table_id,
 | 
						|
          "phy_partition_id",
 | 
						|
          partition_id,
 | 
						|
          "partition_cnt",
 | 
						|
          partition_cnt,
 | 
						|
          "addr",
 | 
						|
          replica_addr.addr_,
 | 
						|
          "zone",
 | 
						|
          replica_addr.zone_,
 | 
						|
          "unit_id",
 | 
						|
          replica_addr.unit_id_,
 | 
						|
          "replica_type",
 | 
						|
          replica_addr.replica_type_);
 | 
						|
    } else {
 | 
						|
      flag_replica.is_restore_ = arg.restore_;
 | 
						|
      if (arg.ignore_member_list_) {
 | 
						|
        // not fill member list
 | 
						|
      } else if (REPLICA_RESTORE_DATA == flag_replica.is_restore_ ||
 | 
						|
                 REPLICA_RESTORE_ARCHIVE_DATA == flag_replica.is_restore_) {
 | 
						|
        // Physically restored copy, no member list
 | 
						|
      } else {
 | 
						|
        ObMember member;
 | 
						|
        for (int64_t index = 0; OB_SUCC(ret) && index < arg.member_list_.get_member_number(); ++index) {
 | 
						|
          if (OB_FAIL(arg.member_list_.get_member_by_index(index, member))) {
 | 
						|
            LOG_WARN("get member by index failed", K(ret), K(index));
 | 
						|
          } else if (OB_FAIL(flag_replica.member_list_.push_back(
 | 
						|
                         ObPartitionReplica::Member(member.get_server(), member.get_timestamp())))) {
 | 
						|
            LOG_WARN("add member list failed", K(ret));
 | 
						|
          } else {
 | 
						|
          }  // do nothing
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::fill_partition_member_list(const ObPartitionAddr& part_addr, const int64_t timestamp,
 | 
						|
    const int64_t paxos_replica_count, obrpc::ObCreatePartitionArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_UNLIKELY(timestamp < 0) || OB_UNLIKELY(OB_INVALID_COUNT == paxos_replica_count)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", KR(ret), K(timestamp), K(paxos_replica_count));
 | 
						|
  } else {
 | 
						|
    ObAddr leader;
 | 
						|
    FOREACH_CNT_X(a, part_addr, OB_SUCC(ret))
 | 
						|
    {
 | 
						|
      if (OB_ISNULL(a)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("a is null", K(ret));
 | 
						|
      } else if (common::REPLICA_TYPE_FULL == a->replica_type_ || common::REPLICA_TYPE_LOGONLY == a->replica_type_) {
 | 
						|
        if (OB_FAIL(arg.member_list_.add_member(ObMember(a->addr_, timestamp)))) {
 | 
						|
          LOG_WARN("fail to add server to list", K(ret), "addr", *a);
 | 
						|
        } else {
 | 
						|
        }  // do nothing
 | 
						|
      } else if (common::REPLICA_TYPE_READONLY == a->replica_type_) {
 | 
						|
        // readonly replica is not in memberlist, do nothing
 | 
						|
      } else {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("invalid replica type", K(ret), "replica_type", a->replica_type_);
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else {
 | 
						|
        if (a->initial_leader_) {
 | 
						|
          leader = a->addr_;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (!leader.is_valid()) {
 | 
						|
      ret = OB_INVALID_ARGUMENT;
 | 
						|
      LOG_WARN("no initial leader", K(ret), K(part_addr));
 | 
						|
    } else {
 | 
						|
      arg.leader_ = leader;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (majority(paxos_replica_count) > arg.member_list_.get_member_number()) {
 | 
						|
      ret = OB_REPLICA_NUM_NOT_ENOUGH;
 | 
						|
      LOG_WARN("partition count less than majority", KR(ret), K(arg), K(paxos_replica_count));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObDDLService::is_zone_exist(const ObArray<ObZone>& 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 schema_status(
 | 
						|
          OB_SYS_TENANT_ID, OB_INVALID_TIMESTAMP, OB_INVALID_VERSION, 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
 | 
						|
      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_refresh_schema_status(schema_status))) {
 | 
						|
        LOG_WARN("init tenant schema status failed", K(ret), K(schema_status));
 | 
						|
      } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
      } else if (OB_FAIL(init_tenant_sys_params(arg, tenant_schema, sys_variable))) {
 | 
						|
        LOG_WARN("fail to init tenant sys params", K(ret), K(tenant_schema));
 | 
						|
      } else if (OB_FAIL(ddl_operator.finish_schema_split_v2(trans, OB_SYS_TENANT_ID))) {
 | 
						|
        LOG_WARN("fail to log ddl operation", K(ret));
 | 
						|
      } 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, trans))) {
 | 
						|
        LOG_WARN("init tenant env failed", K(tenant_schema), K(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, ObTenantSchema& tenant_schema, const int64_t frozen_version)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  ObSchemaService* schema_service = NULL;
 | 
						|
  bool tenant_exist = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (NULL == (schema_service = schema_service_->get_schema_service()) || nullptr == zone_mgr_) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema service is null", 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(check_tenant_schema(arg.pool_list_, tenant_schema, schema_guard))) {
 | 
						|
    LOG_WARN("check tenant schema failed", K(arg.pool_list_), K(tenant_schema), K(ret));
 | 
						|
  } else if (NULL != schema_guard.get_tenant_info(tenant_schema.get_tenant_name())) {
 | 
						|
    tenant_exist = true;
 | 
						|
  } else if (arg.is_restore_) {
 | 
						|
    // Drop tenant recycling internal tables temporarily only supports physical backup and recovery
 | 
						|
    // Since the job recovery of the restore tenant is asynchronous,
 | 
						|
    // it is necessary to ensure that the tenant with the same name is not created during the period.
 | 
						|
    ObSEArray<RestoreJob, 10> job_infos;
 | 
						|
    ObRestoreTableOperator restore_op;
 | 
						|
    if (OB_FAIL(restore_op.init(sql_proxy_))) {
 | 
						|
      LOG_WARN("fail init", K(ret));
 | 
						|
    } else if (OB_FAIL(restore_op.get_jobs(job_infos))) {
 | 
						|
      LOG_WARN("fail to get jobs", K(ret));
 | 
						|
    } else {
 | 
						|
      FOREACH_CNT_X(job_info, job_infos, !tenant_exist)
 | 
						|
      {
 | 
						|
        if (OB_ISNULL(job_info)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("job_info is null", K(ret));
 | 
						|
        } else if (job_info->tenant_name_ == tenant_schema.get_tenant_name()) {
 | 
						|
          tenant_exist = true;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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_schema.get_tenant_name_str()));
 | 
						|
      LOG_INFO("tenant already exists, not need to create", "tenant_name", tenant_schema.get_tenant_name(), K(ret));
 | 
						|
    } else {
 | 
						|
      ret = OB_TENANT_EXIST;
 | 
						|
      LOG_USER_ERROR(OB_TENANT_EXIST, to_cstring(tenant_schema.get_tenant_name_str()));
 | 
						|
      LOG_WARN("tenant already exists", "tenant_name", tenant_schema.get_tenant_name(), K(ret));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    // tenant not exist, create it
 | 
						|
    uint64_t new_tenant_id = tenant_schema.get_tenant_id();
 | 
						|
    if (OB_FAIL(schema_service->fetch_new_tenant_id(new_tenant_id))) {
 | 
						|
      LOG_WARN("fetch_new_tenant_id failed", K(ret));
 | 
						|
    } else if (OB_INVALID_ID == new_tenant_id) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("tenant id is invalid", K(ret), K(new_tenant_id), K(tenant_schema));
 | 
						|
    } else {
 | 
						|
      common::ObRegion primary_region;
 | 
						|
      common::ObZone first_pz;
 | 
						|
      if (tenant_schema.get_primary_zone_array().count() <= 0) {
 | 
						|
        primary_region.reset();  // don't have primary region
 | 
						|
      } else if (FALSE_IT(first_pz = tenant_schema.get_primary_zone_array().at(0).zone_)) {
 | 
						|
        // share never be here
 | 
						|
      } else if (OB_FAIL(zone_mgr_->get_region(first_pz, primary_region))) {
 | 
						|
        LOG_WARN("fail to get region", K(first_pz));
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        ObSchemaStatusProxy* schema_status_proxy = GCTX.schema_status_proxy_;
 | 
						|
        ObRefreshSchemaStatus schema_status(
 | 
						|
            new_tenant_id, OB_INVALID_TIMESTAMP, OB_INVALID_VERSION, OB_INVALID_VERSION);
 | 
						|
        tenant_schema.set_tenant_id(new_tenant_id);
 | 
						|
        // 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
 | 
						|
        if (OB_ISNULL(schema_status_proxy)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("schema_status_proxy is null", K(ret));
 | 
						|
          // TODO:when create tenant, ensure schema_statua>snapshot_gc_ts
 | 
						|
        } else if (OB_FAIL(schema_status_proxy->set_refresh_schema_status(schema_status))) {
 | 
						|
          LOG_WARN("init tenant schema status failed", K(ret), K(schema_status));
 | 
						|
        } else if (OB_FAIL(create_tenant_env(
 | 
						|
                       schema_guard, arg, primary_region, tenant_schema, frozen_version, &arg.ddl_stmt_str_))) {
 | 
						|
          LOG_WARN("create tenant env failed", K(arg), K(tenant_schema), K(ret));
 | 
						|
        }
 | 
						|
        if (OB_SUCC(ret)) {
 | 
						|
          if (OB_FAIL(root_balancer_->create_unit_replica_counter(new_tenant_id))) {
 | 
						|
            LOG_WARN("fail to create unit replica counter", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_tenant_end(const uint64_t tenant_id)
 | 
						|
{
 | 
						|
  LOG_INFO("receieve create tenant end request", K(tenant_id));
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObTenantSchema* tenant_schema = NULL;
 | 
						|
  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 cluster", K(ret));
 | 
						|
  } 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_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 trans(schema_service_);
 | 
						|
    trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    int64_t refreshed_schema_version = OB_INVALID_VERSION;
 | 
						|
    if (!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;
 | 
						|
      int64_t sys_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
 | 
						|
      if (OB_ISNULL(schema_service_impl)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("schema_service_impl is null", K(ret));
 | 
						|
      } else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(tenant_id, refreshed_schema_version))) {
 | 
						|
        LOG_WARN("fail to get tenant schema version", K(ret));
 | 
						|
      } else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(OB_SYS_TENANT_ID, sys_schema_version))) {
 | 
						|
        LOG_WARN("fail to get tenant schema version", K(ret));
 | 
						|
      } 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(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(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*/
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    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 (OB_SUCC(ret)) {
 | 
						|
      if (OB_SUCCESS != (temp_ret = publish_schema(OB_SYS_TENANT_ID))) {
 | 
						|
        LOG_WARN("publish schema failed", K(temp_ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(!arg.is_valid())) {
 | 
						|
    LOG_WARN("invalid argument", 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_id_, orig_tenant_schema))) {
 | 
						|
    ret = OB_TENANT_NOT_EXIST;
 | 
						|
    LOG_WARN("tenant not exist", K(ret), "tenant_id", arg.tenant_id_);
 | 
						|
  } else if (OB_UNLIKELY(NULL == orig_tenant_schema)) {
 | 
						|
    ret = OB_TENANT_NOT_EXIST;
 | 
						|
    LOG_WARN("tenant not exist", K(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",
 | 
						|
        K(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 {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("fail to start transaction", K(ret));
 | 
						|
    } else {
 | 
						|
      ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
      ObTenantSchema new_tenant_schema = *orig_tenant_schema;
 | 
						|
      new_tenant_schema.reset_sysvars();
 | 
						|
      if (OB_FAIL(new_tenant_schema.set_previous_locality(ObString::make_string("")))) {
 | 
						|
        LOG_WARN("fail to set previous locality", K(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant_schema, trans))) {
 | 
						|
        LOG_WARN("fail to alter tenant", K(ret), K(new_tenant_schema));
 | 
						|
      } else {  // 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", K(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", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::commit_alter_tablegroup_locality(const rootserver::ObCommitAlterTablegroupLocalityArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObTablegroupSchema* orig_tablegroup_schema = NULL;
 | 
						|
  const uint64_t tenant_id = extract_tenant_id(arg.tablegroup_id_);
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(!arg.is_valid())) {
 | 
						|
    LOG_WARN("invalid argument", 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_schema(arg.tablegroup_id_, orig_tablegroup_schema))) {
 | 
						|
    ret = OB_TENANT_NOT_EXIST;
 | 
						|
    LOG_WARN("tenant not exist", K(ret), "tenant_id", arg.tablegroup_id_);
 | 
						|
  } else if (OB_UNLIKELY(NULL == orig_tablegroup_schema)) {
 | 
						|
    ret = OB_TENANT_NOT_EXIST;
 | 
						|
    LOG_WARN("tablegroup not exist", K(ret), "tablegroup_id", arg.tablegroup_id_);
 | 
						|
  } else if (OB_UNLIKELY(orig_tablegroup_schema->get_locality_str().empty()) ||
 | 
						|
             OB_UNLIKELY(orig_tablegroup_schema->get_previous_locality_str().empty())) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("tablegroup locality status error",
 | 
						|
        K(ret),
 | 
						|
        "tablegroup_id",
 | 
						|
        orig_tablegroup_schema->get_tablegroup_id(),
 | 
						|
        "tablegroup locality",
 | 
						|
        orig_tablegroup_schema->get_locality_str(),
 | 
						|
        "tablegroup previous locality",
 | 
						|
        orig_tablegroup_schema->get_previous_locality_str());
 | 
						|
  } else {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("fail to start transaction", K(ret));
 | 
						|
    } else {
 | 
						|
      ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
      ObTablegroupSchema new_tablegroup_schema = *orig_tablegroup_schema;
 | 
						|
      if (OB_FAIL(new_tablegroup_schema.set_previous_locality(ObString::make_string("")))) {
 | 
						|
        LOG_WARN("fail to set previous locality", K(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.alter_tablegroup(new_tablegroup_schema, trans))) {
 | 
						|
        LOG_WARN("fail to alter tablegroup", K(ret), K(new_tablegroup_schema));
 | 
						|
      } else {
 | 
						|
        // complete the RS JOB
 | 
						|
        ObRsJobInfo job_info;
 | 
						|
        if (OB_SUCC(RS_JOB_FIND(job_info,
 | 
						|
                trans,
 | 
						|
                "job_type",
 | 
						|
                "ALTER_TABLEGROUP_LOCALITY",
 | 
						|
                "job_status",
 | 
						|
                "INPROGRESS",
 | 
						|
                "tenant_id",
 | 
						|
                tenant_id,
 | 
						|
                "tablegroup_id",
 | 
						|
                arg.tablegroup_id_))) {
 | 
						|
          // good, find job
 | 
						|
        } else if (OB_SUCC(RS_JOB_FIND(job_info,
 | 
						|
                       trans,
 | 
						|
                       "job_type",
 | 
						|
                       "ROLLBACK_ALTER_TABLEGROUP_LOCALITY",
 | 
						|
                       "job_status",
 | 
						|
                       "INPROGRESS",
 | 
						|
                       "tenant_id",
 | 
						|
                       tenant_id,
 | 
						|
                       "tablegroup_id",
 | 
						|
                       arg.tablegroup_id_))) {
 | 
						|
          // good, find job
 | 
						|
        } else {
 | 
						|
          LOG_WARN("failed to find rs job", K(ret), K(arg.tablegroup_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(tenant_id))) {
 | 
						|
          LOG_WARN("fail to publish schema", K(ret));
 | 
						|
        } else {
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::commit_alter_table_locality(const rootserver::ObCommitAlterTableLocalityArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const share::schema::ObTableSchema* orig_table_schema = NULL;
 | 
						|
  const uint64_t tenant_id = extract_tenant_id(arg.table_id_);
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not inited", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(!arg.is_valid())) {
 | 
						|
    LOG_WARN("invalid argument", 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", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_table_schema(arg.table_id_, orig_table_schema))) {
 | 
						|
    ret = OB_TABLE_NOT_EXIST;
 | 
						|
    LOG_WARN("table not exist", K(ret), "table_id", arg.table_id_);
 | 
						|
  } else if (OB_UNLIKELY(NULL == orig_table_schema)) {
 | 
						|
    ret = OB_TABLE_NOT_EXIST;
 | 
						|
    LOG_WARN("table not exist", K(ret), "table_id", arg.table_id_);
 | 
						|
  } else if (OB_UNLIKELY(orig_table_schema->get_locality_str().empty()) ||
 | 
						|
             OB_UNLIKELY(orig_table_schema->get_previous_locality_str().empty())) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("table locality status error",
 | 
						|
        K(ret),
 | 
						|
        "table_id",
 | 
						|
        orig_table_schema->get_table_id(),
 | 
						|
        "table locality",
 | 
						|
        orig_table_schema->get_locality_str(),
 | 
						|
        "table previous locality",
 | 
						|
        orig_table_schema->get_previous_locality_str());
 | 
						|
  } else {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("fail to start transaction", K(ret));
 | 
						|
    } else {
 | 
						|
      ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
      bool update_index_table = false;
 | 
						|
      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(new_table_schema.set_previous_locality(ObString::make_string("")))) {
 | 
						|
        LOG_WARN("fail to set previous locality", K(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.alter_table_options(
 | 
						|
                     schema_guard, new_table_schema, *orig_table_schema, update_index_table, trans))) {
 | 
						|
        LOG_WARN("fail to update table attributes", K(ret), K(new_table_schema));
 | 
						|
      } else {  // complete the RS JOB
 | 
						|
        ObRsJobInfo job_info;
 | 
						|
        if (OB_SUCC(RS_JOB_FIND(job_info,
 | 
						|
                trans,
 | 
						|
                "job_type",
 | 
						|
                "ALTER_TABLE_LOCALITY",
 | 
						|
                "job_status",
 | 
						|
                "INPROGRESS",
 | 
						|
                "table_id",
 | 
						|
                arg.table_id_))) {
 | 
						|
          // good, find job
 | 
						|
        } else if (OB_SUCC(RS_JOB_FIND(job_info,
 | 
						|
                       trans,
 | 
						|
                       "job_type",
 | 
						|
                       "ROLLBACK_ALTER_TABLE_LOCALITY",
 | 
						|
                       "job_status",
 | 
						|
                       "INPROGRESS",
 | 
						|
                       "table_id",
 | 
						|
                       arg.table_id_))) {
 | 
						|
          // good, find job
 | 
						|
        } else {
 | 
						|
          LOG_WARN("failed to find job", K(ret), "table_id", arg.table_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));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }  // no more to do
 | 
						|
      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(tenant_id))) {
 | 
						|
          LOG_WARN("fail to publish schema", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_alter_table_locality_type(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTableSchema& orig_table_schema, const share::schema::ObTableSchema& new_table_schema,
 | 
						|
    AlterLocalityType& alter_locality_type)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
  if (!new_table_schema.get_locality_str().empty()) {
 | 
						|
    if (orig_table_schema.get_previous_locality_str().empty()) {
 | 
						|
      alter_locality_type = TO_NEW_LOCALITY;
 | 
						|
    } else if (orig_table_schema.get_previous_locality_str() == new_table_schema.get_locality_str()) {
 | 
						|
      alter_locality_type = ROLLBACK_LOCALITY;
 | 
						|
    } else {
 | 
						|
      alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    bool locality_in_modification = true;
 | 
						|
    bool is_completed_match = false;
 | 
						|
    if (orig_table_schema.get_locality_str().empty()) {
 | 
						|
      alter_locality_type = LOCALITY_NOT_CHANGED;
 | 
						|
    } else if (!orig_table_schema.get_previous_locality_str().empty()) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("set locality DEFAULT when locality is changing is not allowed", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "set locality DEFAULT when locality is changing");
 | 
						|
    } else if (OB_FAIL(new_table_schema.check_in_locality_modification(schema_guard, locality_in_modification))) {
 | 
						|
      LOG_WARN("fail to check locality in modification", K(ret));
 | 
						|
    } else if (locality_in_modification) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("set locality DEFAULT when locality is changing is not allowed", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "set locality DEFAULT when locality is changing");
 | 
						|
    } else if (OB_FAIL(ObLocalityUtil::check_locality_completed_match(
 | 
						|
                   schema_guard, orig_table_schema, new_table_schema, is_completed_match))) {
 | 
						|
      LOG_WARN("fail to check table locality completed match", K(ret), K(orig_table_schema), K(new_table_schema));
 | 
						|
    } else if (!is_completed_match) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("table locality not completely matching is not allowed", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "table locality not completely matching");
 | 
						|
    } else {
 | 
						|
      alter_locality_type = LOCALITY_NOT_CHANGED;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// The locality of the table of the tablegroup created after adding 2.0 will be reset to inherited semantics,
 | 
						|
// so there is no need to check the table below it
 | 
						|
int ObDDLService::check_alter_tablegroup_locality_type(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTablegroupSchema& orig_tablegroup, const share::schema::ObTablegroupSchema& new_tablegroup,
 | 
						|
    AlterLocalityType& alter_locality_type)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
  if (!new_tablegroup.get_locality_str().empty()) {
 | 
						|
    if (orig_tablegroup.get_previous_locality_str().empty()) {
 | 
						|
      alter_locality_type = TO_NEW_LOCALITY;
 | 
						|
    } else if (orig_tablegroup.get_previous_locality_str() == new_tablegroup.get_locality_str()) {
 | 
						|
      alter_locality_type = ROLLBACK_LOCALITY;
 | 
						|
    } else {
 | 
						|
      alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    // set tablegroup locality default:
 | 
						|
    // 1) Requires that tablegroup and tenant are not in the process of locality change
 | 
						|
    // 2) ensure locality of tablegroup and locality of tenant complete matched
 | 
						|
    const ObTenantSchema* tenant_schema = NULL;
 | 
						|
    const uint64_t tenant_id = new_tablegroup.get_tenant_id();
 | 
						|
    bool is_completed_match = true;
 | 
						|
    if (orig_tablegroup.get_locality_str().empty()) {
 | 
						|
      alter_locality_type = LOCALITY_NOT_CHANGED;
 | 
						|
    } else if (!orig_tablegroup.get_previous_locality_str().empty()) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("set locality DEFAULT when locality is changing is not allowed", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "set locality DEFAULT when locality is changing");
 | 
						|
    } 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("get invalid tenant schema", K(ret), K(tenant_id));
 | 
						|
    } else if (!tenant_schema->get_previous_locality_str().empty()) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("set locality DEFAULT when locality is changing is not allowed", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "set locality DEFAULT when locality is changing");
 | 
						|
    } else if (OB_FAIL(ObLocalityUtil::check_locality_completed_match(
 | 
						|
                   schema_guard, orig_tablegroup, *tenant_schema, is_completed_match))) {
 | 
						|
      LOG_WARN(
 | 
						|
          "fail to check tablegroup tenant locality completed match", K(ret), K(orig_tablegroup), KPC(tenant_schema));
 | 
						|
    } else if (!is_completed_match) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("tablegroup locality not completely matching with tenant's is not allowed", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "tablegroup locality not completely matching with tenant's");
 | 
						|
    } else {
 | 
						|
      alter_locality_type = LOCALITY_NOT_CHANGED;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
    common::ObArray<const share::schema::ObSimpleTablegroupSchema*> tablegroup_schemas;
 | 
						|
    if (is_restore) {
 | 
						|
      // skip
 | 
						|
    } else if (OB_FAIL(schema_guard.get_tablegroup_schemas_in_tenant(tenant_id, tablegroup_schemas))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schemas in tenant", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; TO_NEW_LOCALITY == alter_locality_type && OB_SUCC(ret) && i < tablegroup_schemas.count();
 | 
						|
           ++i) {
 | 
						|
        bool tablegroup_tenant_locality_match = false;
 | 
						|
        const ObSimpleTablegroupSchema* tablegroup_schema = tablegroup_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == tablegroup_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("unexpected error, tablegroup schema is null", K(ret), KP(tablegroup_schema));
 | 
						|
        } else if (!is_new_tablegroup_id(tablegroup_schema->get_tablegroup_id())) {
 | 
						|
          // skip
 | 
						|
        } else if (tablegroup_schema->get_locality_str().empty()) {
 | 
						|
          // tablegroup locality is empty, derived from tenant, ignore it
 | 
						|
        } else {
 | 
						|
          // tablegroup locality is not empty, has its own locality
 | 
						|
          if (!tablegroup_schema->get_previous_locality_str().empty()) {
 | 
						|
            // a tablegroup locality modification is being executed,
 | 
						|
            alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
            LOG_WARN("cannot invalid a tenant locality modification", K(*tablegroup_schema));
 | 
						|
          } else if (OB_FAIL(check_tablegroup_tenant_locality_match(
 | 
						|
                         schema_guard, *tablegroup_schema, orig_tenant_schema, tablegroup_tenant_locality_match))) {
 | 
						|
            LOG_WARN("fail to check tablegroup tenant locality match", K(ret));
 | 
						|
          } else if (!tablegroup_tenant_locality_match) {
 | 
						|
            alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
            LOG_WARN("table and tenant locality not match", K(*tablegroup_schema));
 | 
						|
          } else {
 | 
						|
          }  // tablegroup tenant locality match, good
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    common::ObArray<const share::schema::ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
    if (OB_FAIL(ret) || TO_NEW_LOCALITY != alter_locality_type) {
 | 
						|
      // do nothing
 | 
						|
    } else if (is_restore) {
 | 
						|
      // skip
 | 
						|
    } else if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(tenant_id, table_schemas))) {
 | 
						|
      LOG_WARN("fail to get table schemas of tenant", K(ret), K(tenant_id));
 | 
						|
    } else {
 | 
						|
      // If the current tenant locality has not changed, check that all tables under the tenant whose locality is not
 | 
						|
      // empty must satisfy that no locality has occurred
 | 
						|
      for (int64_t i = 0; TO_NEW_LOCALITY == alter_locality_type && OB_SUCC(ret) && i < table_schemas.count(); ++i) {
 | 
						|
        bool table_tenant_locality_match = false;
 | 
						|
        const ObSimpleTableSchemaV2*& table_schema = table_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == table_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("unexpected error, table schema is null", K(ret), KP(table_schema));
 | 
						|
        } else if (is_new_tablegroup_id(table_schema->get_tablegroup_id())) {
 | 
						|
          // skip
 | 
						|
        } else if (!table_schema->has_self_partition()) {
 | 
						|
          // table without partition, ignore it
 | 
						|
        } else if (table_schema->get_locality_str().empty()) {
 | 
						|
          // table locality is empty, derived from tenant, ignore it
 | 
						|
        } else {
 | 
						|
          if (!table_schema->get_previous_locality_str().empty()) {
 | 
						|
            // a table locality modification is being executed,
 | 
						|
            // cannot invoke a tenant locality modification
 | 
						|
            alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
            LOG_WARN("cannot invalid a tenant locality modification", K(*table_schema));
 | 
						|
          } else if (OB_FAIL(check_table_tenant_locality_match(
 | 
						|
                         schema_guard, *table_schema, orig_tenant_schema, table_tenant_locality_match))) {
 | 
						|
            LOG_WARN("fail to check table tenant loality match", K(ret), K(new_tenant_schema), K(*table_schema));
 | 
						|
          } else if (!table_tenant_locality_match) {
 | 
						|
            alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
            LOG_WARN("table and tenant locality not match", K(*table_schema));
 | 
						|
          } else {
 | 
						|
          }  // no more to do
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } 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<common::ObZone> 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<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
    AlterLocalityType alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
    bool tenant_pools_in_shrinking = false;
 | 
						|
    common::ObArray<share::ObResourcePoolName> 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<share::ObResourcePoolName> 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_rollback_modify_tenant_locality(const obrpc::ObModifyTenantArg& arg,
 | 
						|
    share::schema::ObTenantSchema& new_schema, const share::schema::ObTenantSchema& orig_schema,
 | 
						|
    const common::ObIArray<common::ObZone>& zones_in_pool,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& zone_region_list, AlterLocalityOp& alter_locality_op)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(arg);
 | 
						|
  alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
  ObArray<AlterPaxosLocalityTask> alter_paxos_tasks;
 | 
						|
  ObArray<share::ObZoneReplicaNumSet> pre_zone_locality;
 | 
						|
  common::ObArray<share::ObZoneReplicaAttrSet> 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<share::schema::ObZoneRegion>& zone_region_list,
 | 
						|
    common::ObArray<common::ObZone>& zone_list) const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  zone_list.reset();
 | 
						|
  UNUSED(zone_region_list);
 | 
						|
  common::ObArray<common::ObZone> 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<common::ObZone>& 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<common::ObZone>& zones_in_pool,
 | 
						|
    const common::ObIArray<share::schema::ObZoneRegion>& 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<AlterPaxosLocalityTask> alter_paxos_tasks;
 | 
						|
  ObArray<share::ObZoneReplicaNumSet> pre_zone_locality;
 | 
						|
  common::ObArray<share::ObZoneReplicaAttrSet> 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<share::ObResourcePoolName>& resource_pool_names,
 | 
						|
    common::ObIArray<common::ObZone>& zones_in_pool)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> 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<common::ObZone>& zones_in_pool)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<share::ObResourcePoolName> 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<share::ObResourcePoolName>& resource_pool_names, common::ObIArray<common::ObZone>& zones_in_pool,
 | 
						|
    common::ObIArray<share::schema::ObZoneRegion>& 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::REWRITE_MERGE_VERSION: {
 | 
						|
          int64_t frozen_version = 0;
 | 
						|
          int64_t try_frozen_version = 0;
 | 
						|
          if (NULL == zone_mgr_) {
 | 
						|
            ret = OB_NOT_INIT;
 | 
						|
            LOG_WARN("The zone_mgr_ is NULL, ", K(ret));
 | 
						|
          } else if (OB_FAIL(zone_mgr_->get_try_frozen_version(frozen_version, try_frozen_version))) {
 | 
						|
            LOG_WARN("Fail to get try frozen version, ", K(ret));
 | 
						|
          } else if (alter_tenant_schema.get_rewrite_merge_version() <= frozen_version) {
 | 
						|
            ret = OB_INVALID_ARGUMENT;
 | 
						|
            LOG_WARN("The rewrite merge version must larger than frozen version, ",
 | 
						|
                K(frozen_version),
 | 
						|
                K(alter_tenant_schema.get_rewrite_merge_version()),
 | 
						|
                K(ret));
 | 
						|
          } else {
 | 
						|
            new_tenant_schema.set_rewrite_merge_version(alter_tenant_schema.get_rewrite_merge_version());
 | 
						|
          }
 | 
						|
          break;
 | 
						|
        }
 | 
						|
        case ObModifyTenantArg::STORAGE_FORMAT_VERSION: {
 | 
						|
          const int64_t storage_format_version = alter_tenant_schema.get_storage_format_version();
 | 
						|
          if (storage_format_version < 0) {
 | 
						|
            ret = OB_INVALID_ARGUMENT;
 | 
						|
            LOG_WARN("invalid storage_format_version", K(ret), K(storage_format_version));
 | 
						|
          } else {
 | 
						|
            new_tenant_schema.set_storage_format_version(storage_format_version);
 | 
						|
          }
 | 
						|
          break;
 | 
						|
        }
 | 
						|
        case ObModifyTenantArg::STORAGE_FORMAT_WORK_VERSION: {
 | 
						|
          int64_t frozen_version = 0;
 | 
						|
          int64_t try_frozen_version = 0;
 | 
						|
          const int64_t storage_format_work_version = alter_tenant_schema.get_storage_format_work_version();
 | 
						|
          if (NULL == zone_mgr_) {
 | 
						|
            ret = OB_NOT_INIT;
 | 
						|
            LOG_WARN("The zone_mgr_ is NULL, ", K(ret));
 | 
						|
          } else if (OB_FAIL(zone_mgr_->get_try_frozen_version(frozen_version, try_frozen_version))) {
 | 
						|
            LOG_WARN("Fail to get try frozen version, ", K(ret));
 | 
						|
          } else if (storage_format_work_version <= try_frozen_version) {
 | 
						|
            ret = OB_INVALID_ARGUMENT;
 | 
						|
            LOG_WARN("The storage fromat work version must larger than try frozen version, ",
 | 
						|
                K(ret),
 | 
						|
                K(try_frozen_version),
 | 
						|
                K(storage_format_work_version));
 | 
						|
          } else {
 | 
						|
            new_tenant_schema.set_storage_format_work_version(storage_format_work_version);
 | 
						|
          }
 | 
						|
          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,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const share::schema::ObTenantSchema& new_tenant_schema,
 | 
						|
    const common::ObIArray<common::ObString>& new_pool_list, bool& grant,
 | 
						|
    common::ObIArray<ObResourcePoolName>& diff_pools)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::ObWorker::CompatMode compat_mode = share::ObWorker::CompatMode::INVALID;
 | 
						|
  common::ObArray<ObResourcePoolName> new_pool_name_list;
 | 
						|
  common::ObArray<ObResourcePoolName> 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, 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, 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<share::ObResourcePoolName>& 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<share::ObResourcePoolName>& 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<share::ObResourcePoolName>& 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;
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_normal_tenant_revoke_pools_permitted(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const common::ObIArray<share::ObResourcePoolName>& new_pool_name_list,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, bool& is_permitted)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  is_permitted = true;
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  common::ObArray<const share::schema::ObPartitionSchema*> partition_schemas;
 | 
						|
  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 if (OB_FAIL(ObPartMgrUtils::get_partition_entity_schemas_in_tenant(
 | 
						|
                 schema_guard, tenant_schema.get_tenant_id(), partition_schemas))) {
 | 
						|
    LOG_WARN("fail to get table schemas in tenant", K(ret), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
  } 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 */
 | 
						|
      }
 | 
						|
    }
 | 
						|
    for (int64_t j = 0; j < partition_schemas.count() && OB_SUCC(ret) && is_permitted; ++j) {
 | 
						|
      zone_list.reset();
 | 
						|
      const ObPartitionSchema* simple_schema = partition_schemas.at(j);
 | 
						|
      if (OB_UNLIKELY(NULL == simple_schema)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("unexpected error, table schema null", K(ret), KP(simple_schema));
 | 
						|
      } else if (!simple_schema->has_self_partition()) {
 | 
						|
        // table without partition, ignore
 | 
						|
      } else if (simple_schema->get_locality_str().empty()) {
 | 
						|
        // empty, derived from tenant, go on next
 | 
						|
      } else if (!simple_schema->get_previous_locality_str().empty()) {
 | 
						|
        is_permitted = false;
 | 
						|
        LOG_USER_ERROR(OB_OP_NOT_ALLOW, "revoking resource pools when table/tg in locality modification");
 | 
						|
      } else if (OB_FAIL(simple_schema->get_zone_list(schema_guard, zone_list))) {
 | 
						|
        LOG_WARN("fail to get zone list", KR(ret), "tid", simple_schema->get_table_id());
 | 
						|
      } 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),
 | 
						|
                K(zone),
 | 
						|
                "tg/table_id",
 | 
						|
                simple_schema->get_table_id());
 | 
						|
          } else if (OB_FAIL(unit_mgr_->check_schema_zone_unit_enough(zone,
 | 
						|
                         total_unit_num,
 | 
						|
                         full_unit_num,
 | 
						|
                         logonly_unit_num,
 | 
						|
                         *simple_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 table/tg locality on");
 | 
						|
          } else {
 | 
						|
          }  //  go on to check next
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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<ObResourcePoolName>& long_pool_name_list,
 | 
						|
    const common::ObIArray<ObResourcePoolName>& short_pool_name_list, common::ObIArray<ObResourcePoolName>& 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;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::modify_gts_tenant(const ObModifyTenantArg& arg, share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTenantSchema& ori_tenant_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::schema::ObTenantSchema new_tenant_schema = ori_tenant_schema;
 | 
						|
  new_tenant_schema.reset_sysvars();
 | 
						|
  for (int32_t i = ObModifyTenantArg::REPLICA_NUM; OB_SUCC(ret) && i < ObModifyTenantArg::MAX_OPTION; ++i) {
 | 
						|
    if (arg.alter_option_bitset_.has_member(i)) {
 | 
						|
      switch (i) {
 | 
						|
        case ObModifyTenantArg::RESOURCE_POOL_LIST:
 | 
						|
          break;
 | 
						|
        default:
 | 
						|
          ret = OB_NOT_SUPPORTED;
 | 
						|
          LOG_WARN("modify tenant option other than resource pool list not supported", K(ret));
 | 
						|
          LOG_USER_ERROR(OB_NOT_SUPPORTED, "modify tenant option other than resource pool list");
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret) && arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    bool grant = true;
 | 
						|
    ObArray<ObResourcePoolName> diff_pools;
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("fail to start trans", K(ret));
 | 
						|
    } else if (OB_FAIL(modify_and_cal_resource_pool_diff(
 | 
						|
                   trans, schema_guard, new_tenant_schema, arg.pool_list_, grant, diff_pools))) {
 | 
						|
      LOG_WARN("fail to modify and calc resource pool diff", K(ret));
 | 
						|
    }
 | 
						|
 | 
						|
    if (trans.is_started()) {
 | 
						|
      int temp_ret = OB_SUCCESS;
 | 
						|
      if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
 | 
						|
        LOG_WARN("fail to end trans", K(ret), "is_commit", OB_SUCC(ret));
 | 
						|
        ret = (OB_SUCC(ret) ? temp_ret : ret);
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret) && arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST) &&
 | 
						|
        diff_pools.count() > 0) {
 | 
						|
      const uint64_t tenant_id = ori_tenant_schema.get_tenant_id();
 | 
						|
      if (OB_FAIL(unit_mgr_->commit_change_pool_owner(grant, diff_pools, tenant_id))) {
 | 
						|
        LOG_WARN("fail to commit change pool owner", K(ret), K(tenant_id), K(grant), K(diff_pools));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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 (OB_UNLIKELY(NULL == root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_balancer is null", K(ret), KP(root_balancer_));
 | 
						|
  } 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_GTS_TENANT_ID == orig_tenant_schema->get_tenant_id()) {
 | 
						|
    if (arg.sys_var_list_.count() > 0) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_WARN("cannot modify GTS tenant system variable", K(ret));
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify GTS tenant system variable is");
 | 
						|
    } else if (OB_FAIL(modify_gts_tenant(arg, schema_guard, *orig_tenant_schema))) {
 | 
						|
      LOG_WARN("fail to modify gts tenant", K(ret));
 | 
						|
    }
 | 
						|
  } 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;
 | 
						|
    // the schema info in the tenant
 | 
						|
    trans.set_end_tenant_id(tenant_id);
 | 
						|
    if (is_standby || is_restore) {
 | 
						|
      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) {
 | 
						|
        if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
          LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
        } 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<ObResourcePoolName> diff_pools;
 | 
						|
    AlterLocalityOp alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
    ObTenantSchema new_tenant_schema = *orig_tenant_schema;
 | 
						|
    new_tenant_schema.reset_sysvars();
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
 | 
						|
    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));
 | 
						|
    }
 | 
						|
    // tranction 1: Primary_zone verification of database/tablegroup/table, the transaction fails without rollback
 | 
						|
    if (is_standby || is_restore) {
 | 
						|
    } else if (OB_SUCC(ret)) {
 | 
						|
      ObDDLSQLTransaction trans(schema_service_);
 | 
						|
      // Modify the primary zone information under the tenant, it may be an empty DDL,
 | 
						|
      // it need to put the tenant_id in and write it under the tenant
 | 
						|
      trans.set_end_tenant_id(tenant_id);
 | 
						|
      const int64_t pre_last_schema_version = ddl_operator.get_last_operation_schema_version();
 | 
						|
      int64_t total_sub_pz_count = 0;
 | 
						|
      int64_t db_sub_pz_count = 0;
 | 
						|
      int64_t tg_sub_pz_count = 0;
 | 
						|
      int64_t tb_sub_pz_count = 0;
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
      } else if (OB_FAIL(try_modify_databases_attributes_in_tenant(
 | 
						|
                     arg, ddl_operator, trans, schema_guard, new_tenant_schema, db_sub_pz_count))) {
 | 
						|
        LOG_WARN(
 | 
						|
            "fail to try modify database attributes of tenant", K(ret), "tenant_id", new_tenant_schema.get_tenant_id());
 | 
						|
      } else if (OB_FAIL(try_modify_tablegroups_attributes_in_tenant(
 | 
						|
                     arg, ddl_operator, trans, schema_guard, new_tenant_schema, tg_sub_pz_count))) {
 | 
						|
        LOG_WARN("fail to try modify tablegroup attributes of tenant",
 | 
						|
            K(ret),
 | 
						|
            "tenant_id",
 | 
						|
            new_tenant_schema.get_tenant_id());
 | 
						|
      } else if (OB_FAIL(try_modify_tables_attributes_in_tenant(
 | 
						|
                     arg, ddl_operator, trans, schema_guard, new_tenant_schema, tb_sub_pz_count))) {
 | 
						|
        LOG_WARN(
 | 
						|
            "fail to try modify table attributes of tenant", K(ret), "tenant_id", new_tenant_schema.get_tenant_id());
 | 
						|
      } else {
 | 
						|
        total_sub_pz_count = db_sub_pz_count + tg_sub_pz_count + tb_sub_pz_count;
 | 
						|
        if (total_sub_pz_count <= 0) {
 | 
						|
          // no need to update
 | 
						|
        } else if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(
 | 
						|
                       trans, schema_guard, false /*sub*/, total_sub_pz_count, tenant_id))) {
 | 
						|
          LOG_WARN("fail to try modify tenant primary zone entity count", 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)) {
 | 
						|
        const int64_t cur_last_schema_version = ddl_operator.get_last_operation_schema_version();
 | 
						|
        if (cur_last_schema_version == pre_last_schema_version) {
 | 
						|
          // schema not changed, no need to publish schema version
 | 
						|
        } else if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
          LOG_WARN("publish schema failed, ", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    // transaction 2: modify tenant option
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      ObDDLSQLTransaction trans(schema_service_);
 | 
						|
      trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
      } else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST) &&
 | 
						|
                 OB_FAIL(modify_and_cal_resource_pool_diff(
 | 
						|
                     trans, 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) {
 | 
						|
        const ObSysVariableSchema* orig_sys_variable = NULL;
 | 
						|
        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 if (OB_FAIL(check_tenant_primary_zone_gts_condition(
 | 
						|
                       schema_guard, new_tenant_schema, *orig_sys_variable, *orig_sys_variable))) {
 | 
						|
          LOG_WARN("fail to check tenant primary zone gts condition", K(ret));
 | 
						|
        } else if (OB_FAIL(try_check_tenant_turn_gts_on_condition(
 | 
						|
                       schema_guard, *orig_tenant_schema, new_tenant_schema, *orig_sys_variable))) {
 | 
						|
          LOG_WARN("fail to check tenant turn gts on condition", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
 | 
						|
      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_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));
 | 
						|
      }
 | 
						|
 | 
						|
      if (OB_SUCC(ret) && ROLLBACK_ALTER_LOCALITY == alter_locality_op) {
 | 
						|
        task_mgr_->clear_task(tenant_id, obrpc::ObAdminClearBalanceTaskArg::ALL);
 | 
						|
      }
 | 
						|
      // always notify locality modification whether the above process succeed
 | 
						|
      // and this shall be invoked alter trans.end(), don't change the sequence.
 | 
						|
      if (!new_tenant_schema.get_previous_locality_str().empty()) {
 | 
						|
        root_balancer_->notify_locality_modification();
 | 
						|
        root_balancer_->wakeup();
 | 
						|
      }
 | 
						|
 | 
						|
      // 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(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 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_);
 | 
						|
    trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
    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 (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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
    } 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<obrpc::ObSysVarIdValue>& 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.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::record_table_locality_event_history(const AlterLocalityOp& alter_locality_op,
 | 
						|
    const obrpc::ObAlterTableArg& alter_table_arg, const share::schema::ObTableSchema& table_schema,
 | 
						|
    common::ObMySQLTransaction& trans)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (ALTER_LOCALITY == alter_locality_op) {
 | 
						|
    int64_t job_id = RS_JOB_CREATE(ALTER_TABLE_LOCALITY,
 | 
						|
        trans,
 | 
						|
        "table_name",
 | 
						|
        table_schema.get_table_name(),
 | 
						|
        "table_id",
 | 
						|
        table_schema.get_table_id(),
 | 
						|
        "sql_text",
 | 
						|
        ObHexEscapeSqlStr(alter_table_arg.ddl_stmt_str_),
 | 
						|
        "extra_info",
 | 
						|
        table_schema.get_previous_locality_str());
 | 
						|
    if (job_id < 1) {
 | 
						|
      ret = OB_SQL_OPT_ERROR;
 | 
						|
      LOG_WARN("insert into all_rootservice_job failed", K(ret));
 | 
						|
    }
 | 
						|
  } else if (ROLLBACK_ALTER_LOCALITY == alter_locality_op) {
 | 
						|
    ObRsJobInfo job_info;
 | 
						|
    if (OB_SUCC(RS_JOB_FIND(job_info,
 | 
						|
            trans,
 | 
						|
            "job_type",
 | 
						|
            "ALTER_TABLE_LOCALITY",
 | 
						|
            "job_status",
 | 
						|
            "INPROGRESS",
 | 
						|
            "table_id",
 | 
						|
            table_schema.get_table_id()))) {
 | 
						|
      // good find job
 | 
						|
    } else if (OB_SUCC(RS_JOB_FIND(job_info,
 | 
						|
                   trans,
 | 
						|
                   "job_type",
 | 
						|
                   "ROLLBACK_ALTER_TABLE_LOCALITY",
 | 
						|
                   "job_status",
 | 
						|
                   "INPROGRESS",
 | 
						|
                   "table_id",
 | 
						|
                   table_schema.get_table_id()))) {
 | 
						|
      // good find job
 | 
						|
    } else {
 | 
						|
      LOG_WARN("failed to find job need rollback", K(ret), K(table_schema.get_table_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), K(job_info));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      LOG_WARN("failed to find job", K(ret));
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      int64_t job_id = RS_JOB_CREATE(ROLLBACK_ALTER_TABLE_LOCALITY,
 | 
						|
          trans,
 | 
						|
          "table_name",
 | 
						|
          table_schema.get_table_name(),
 | 
						|
          "table_id",
 | 
						|
          table_schema.get_table_id(),
 | 
						|
          "sql_text",
 | 
						|
          ObHexEscapeSqlStr(alter_table_arg.ddl_stmt_str_));
 | 
						|
      if (job_id < 1) {
 | 
						|
        ret = OB_SQL_OPT_ERROR;
 | 
						|
        LOG_WARN("insert into all_rootservice_job failed", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (NOP_LOCALITY_OP == alter_locality_op) {  // success only
 | 
						|
    int64_t job_id = RS_JOB_CREATE(ALTER_TABLE_LOCALITY,
 | 
						|
        trans,
 | 
						|
        "table_name",
 | 
						|
        table_schema.get_table_name(),
 | 
						|
        "table_id",
 | 
						|
        table_schema.get_table_id(),
 | 
						|
        "sql_text",
 | 
						|
        ObHexEscapeSqlStr(alter_table_arg.ddl_stmt_str_),
 | 
						|
        "extra_info",
 | 
						|
        table_schema.get_previous_locality_str());
 | 
						|
    if (job_id < 1) {
 | 
						|
      ret = OB_SQL_OPT_ERROR;
 | 
						|
      LOG_WARN("insert into all_rootservice_job failed", K(ret));
 | 
						|
    } else if (OB_FAIL(RS_JOB_COMPLETE(job_id, 0, trans))) {
 | 
						|
      LOG_WARN("update rs_job failed", K(ret), K(job_id));
 | 
						|
    }
 | 
						|
 | 
						|
  } else {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("invalid alter locality op", K(ret), K(alter_locality_op));
 | 
						|
  }
 | 
						|
  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;
 | 
						|
  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))) {
 | 
						|
    LOG_WARN("fail to check sequence exist", K(ret));
 | 
						|
  } else if (sequence_exist) {
 | 
						|
    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,
 | 
						|
    common::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;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::record_tablegroup_locality_event_history(const AlterLocalityOp& alter_locality_op,
 | 
						|
    const obrpc::ObAlterTablegroupArg& alter_tablegroup_arg, const share::schema::ObTablegroupSchema& tablegroup_schema,
 | 
						|
    common::ObMySQLTransaction& trans)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (ALTER_LOCALITY == alter_locality_op) {
 | 
						|
    int64_t job_id = RS_JOB_CREATE(ALTER_TABLEGROUP_LOCALITY,
 | 
						|
        trans,
 | 
						|
        "tenant_id",
 | 
						|
        tablegroup_schema.get_tenant_id(),
 | 
						|
        "tablegroup_name",
 | 
						|
        tablegroup_schema.get_tablegroup_name(),
 | 
						|
        "tablegroup_id",
 | 
						|
        tablegroup_schema.get_tablegroup_id(),
 | 
						|
        "sql_text",
 | 
						|
        ObHexEscapeSqlStr(alter_tablegroup_arg.ddl_stmt_str_),
 | 
						|
        "extra_info",
 | 
						|
        tablegroup_schema.get_previous_locality_str());
 | 
						|
    if (job_id < 1) {
 | 
						|
      ret = OB_SQL_OPT_ERROR;
 | 
						|
      LOG_WARN("failed to create rs job", K(tablegroup_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_TABLEGROUP_LOCALITY",
 | 
						|
            "job_status",
 | 
						|
            "INPROGRESS",
 | 
						|
            "tenant_id",
 | 
						|
            tablegroup_schema.get_tenant_id(),
 | 
						|
            "tablegroup_id",
 | 
						|
            tablegroup_schema.get_tablegroup_id()))) {
 | 
						|
      // good find job
 | 
						|
    } else if (OB_SUCC(RS_JOB_FIND(job_info,
 | 
						|
                   trans,
 | 
						|
                   "job_type",
 | 
						|
                   "ROLLBACK_ALTER_TABLEGROUP_LOCALITY",
 | 
						|
                   "job_status",
 | 
						|
                   "INPROGRESS",
 | 
						|
                   "tenant_id",
 | 
						|
                   tablegroup_schema.get_tenant_id(),
 | 
						|
                   "tablegroup_id",
 | 
						|
                   tablegroup_schema.get_tablegroup_id()))) {
 | 
						|
      // good find job
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret) && job_info.job_id_ > 0) {
 | 
						|
      if (OB_FAIL(RS_JOB_COMPLETE(job_info.job_id_, -1, trans))) {
 | 
						|
        LOG_WARN("failed to complete rs job", K(ret), K(tablegroup_schema.get_tenant_id()));
 | 
						|
      }  // The change task is rolled back, this change failed
 | 
						|
    }
 | 
						|
    int64_t job_id = RS_JOB_CREATE(ROLLBACK_ALTER_TABLEGROUP_LOCALITY,
 | 
						|
        trans,
 | 
						|
        "tablegroup_name",
 | 
						|
        tablegroup_schema.get_tablegroup_name(),
 | 
						|
        "tenant_id",
 | 
						|
        tablegroup_schema.get_tenant_id(),
 | 
						|
        "tablegroup_id",
 | 
						|
        tablegroup_schema.get_tablegroup_id(),
 | 
						|
        "sql_text",
 | 
						|
        ObHexEscapeSqlStr(alter_tablegroup_arg.ddl_stmt_str_),
 | 
						|
        "extra_info",
 | 
						|
        tablegroup_schema.get_locality_str());
 | 
						|
    if (job_id < 1) {
 | 
						|
      ret = OB_SQL_OPT_ERROR;
 | 
						|
      LOG_WARN("failed to create rs job", K(tablegroup_schema.get_tenant_id()));
 | 
						|
    }
 | 
						|
  } else if (NOP_LOCALITY_OP == alter_locality_op) {
 | 
						|
    int64_t job_id = RS_JOB_CREATE(ALTER_TABLEGROUP_LOCALITY,
 | 
						|
        trans,
 | 
						|
        "tenant_id",
 | 
						|
        tablegroup_schema.get_tenant_id(),
 | 
						|
        "tablegroup_name",
 | 
						|
        tablegroup_schema.get_tablegroup_name(),
 | 
						|
        "tablegroup_id",
 | 
						|
        tablegroup_schema.get_tablegroup_id(),
 | 
						|
        "sql_text",
 | 
						|
        ObHexEscapeSqlStr(alter_tablegroup_arg.ddl_stmt_str_),
 | 
						|
        "extra_info",
 | 
						|
        tablegroup_schema.get_previous_locality_str());
 | 
						|
    if (job_id < 1) {
 | 
						|
      ret = OB_SQL_OPT_ERROR;
 | 
						|
      LOG_WARN("failed to create rs job", K(tablegroup_schema.get_tenant_id()));
 | 
						|
    } else if (OB_FAIL(RS_JOB_COMPLETE(job_id, 0, trans))) {
 | 
						|
      LOG_WARN("failed to complete rs job", K(ret), K(tablegroup_schema.get_tenant_id()));
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("invalid alter locality op", K(ret), K(alter_locality_op));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// When creating a tenant, in order to create tenant partitions, first get the units and get whether it is a small
 | 
						|
// tenant Kicked out logonly replica; implementation is relatively rough
 | 
						|
int ObDDLService::get_tenant_units_and_scalescope(const common::ObIArray<obrpc::ObSysVarIdValue>& sys_var_list,
 | 
						|
    const common::ObIArray<common::ObString>& pool_list, const share::schema::ObTenantSchema& tenant_schema,
 | 
						|
    ObReplicaCreator::ZoneUnitArray& all_zone_unit, ObReplicaCreator::ZoneUnitPtrArray& all_zone_unit_ptr,
 | 
						|
    bool& small_tenant)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(sys_var_list);
 | 
						|
  ObArray<ObUnitInfo> tmp;
 | 
						|
  small_tenant = true;
 | 
						|
  all_zone_unit.reset();
 | 
						|
  all_zone_unit_ptr.reset();
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else {
 | 
						|
  }  // no more to do
 | 
						|
  FOREACH_CNT_X(p, pool_list, OB_SUCC(ret))
 | 
						|
  {
 | 
						|
    if (OB_ISNULL(p)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("p is null", K(ret));
 | 
						|
    } else {
 | 
						|
      tmp.reuse();
 | 
						|
      ObResourcePoolName name = *p;
 | 
						|
      uint64_t pool_id = 0;
 | 
						|
      if (OB_FAIL(unit_mgr_->get_pool_id(name, pool_id))) {
 | 
						|
        LOG_WARN("get resource pool name failed", K(ret), K(name));
 | 
						|
      } else if (OB_FAIL(unit_mgr_->get_active_unit_infos_of_pool(pool_id, tmp))) {
 | 
						|
        LOG_WARN("get resource unit failed", K(ret), K(name), K(pool_id));
 | 
						|
      } else {
 | 
						|
        ObArray<ObZone> zone_units;
 | 
						|
        for (int64_t i = 1; small_tenant && i < tmp.count(); ++i) {
 | 
						|
          if (REPLICA_TYPE_LOGONLY == tmp.at(i).unit_.replica_type_) {
 | 
						|
            // Units in a pool must be of the same type
 | 
						|
            break;
 | 
						|
          } else if (tmp.at(i).unit_.zone_ == tmp.at(0).unit_.zone_) {
 | 
						|
            small_tenant = false;
 | 
						|
          }
 | 
						|
        }
 | 
						|
        FOREACH_X(u, tmp, OB_SUCC(ret))
 | 
						|
        {
 | 
						|
          if (OB_ISNULL(u)) {
 | 
						|
            ret = OB_ERR_UNEXPECTED;
 | 
						|
            LOG_WARN("u is null", K(ret));
 | 
						|
          } else if (has_exist_in_array(zone_list, u->unit_.zone_)) {
 | 
						|
            if (u->unit_.replica_type_ == REPLICA_TYPE_LOGONLY) {
 | 
						|
            } else {
 | 
						|
              bool zone_alloc = false;
 | 
						|
              FOREACH_CNT_X(zone_unit, all_zone_unit, OB_SUCC(ret) && !zone_alloc)
 | 
						|
              {
 | 
						|
                if (OB_UNLIKELY(NULL == zone_unit) || OB_UNLIKELY(zone_unit->count() <= 0)) {
 | 
						|
                  ret = OB_ERR_UNEXPECTED;
 | 
						|
                  LOG_WARN("unit is null or zone unit count is zero", K(ret));
 | 
						|
                } else if (zone_unit->at(0).unit_.zone_ == u->unit_.zone_) {
 | 
						|
                  zone_alloc = true;
 | 
						|
                  if (OB_FAIL(zone_unit->push_back(*u))) {
 | 
						|
                    LOG_WARN("fail to push back", K(ret));
 | 
						|
                  }
 | 
						|
                } else {
 | 
						|
                }  // go on find
 | 
						|
              }
 | 
						|
              if (OB_FAIL(ret)) {
 | 
						|
              } else if (!zone_alloc) {
 | 
						|
                bool is_alive = false;
 | 
						|
                if (OB_FAIL(server_mgr_->check_server_alive(u->unit_.server_, is_alive))) {
 | 
						|
                  LOG_WARN("check_server_alive failed", "server", u->unit_.server_, K(ret));
 | 
						|
                } else if (is_alive) {
 | 
						|
                  ObReplicaCreator::UnitArray new_unit_array;
 | 
						|
                  if (OB_FAIL(new_unit_array.push_back(*u))) {
 | 
						|
                    LOG_WARN("push array item failed", K(ret));
 | 
						|
                  } else if (OB_FAIL(all_zone_unit.push_back(new_unit_array))) {
 | 
						|
                    LOG_WARN("fail to push back", K(ret));
 | 
						|
                  } else {
 | 
						|
                  }  // no more to do
 | 
						|
                } else {
 | 
						|
                }  // not alive, do nothing
 | 
						|
              } else {
 | 
						|
              }  // do nothing
 | 
						|
            }
 | 
						|
          } else {
 | 
						|
          }  // do not in tenant zone list, ignore
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (!small_tenant) {
 | 
						|
      // bypass
 | 
						|
    } else {
 | 
						|
      const ObIArray<ObZoneScore>& primary_zone_array = tenant_schema.get_primary_zone_array();
 | 
						|
      if (primary_zone_array.count() <= 0) {
 | 
						|
        small_tenant = false;  // empty or random
 | 
						|
      } else {
 | 
						|
        int64_t high_primary_zone_counter = 0;
 | 
						|
        const ObZoneScore& sample_zone = primary_zone_array.at(0);
 | 
						|
        for (int64_t i = 0; i < primary_zone_array.count(); ++i) {
 | 
						|
          if (sample_zone.score_ == primary_zone_array.at(i).score_) {
 | 
						|
            high_primary_zone_counter++;
 | 
						|
          } else {
 | 
						|
            break;
 | 
						|
          }
 | 
						|
        }
 | 
						|
        if (high_primary_zone_counter > 1) {
 | 
						|
          small_tenant = false;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    ObReplicaCreator::UnitPtrArray unit_ptr_array;
 | 
						|
    all_zone_unit_ptr.reserve(all_zone_unit.count());
 | 
						|
    FOREACH_CNT_X(zu, all_zone_unit, OB_SUCC(ret))
 | 
						|
    {
 | 
						|
      unit_ptr_array.reuse();
 | 
						|
      unit_ptr_array.reserve(zu->count());
 | 
						|
      FOREACH_CNT_X(up, *zu, OB_SUCC(ret))
 | 
						|
      {
 | 
						|
        if (OB_FAIL(unit_ptr_array.push_back(up))) {
 | 
						|
          LOG_WARN("unit_ptr_array push back failed", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(all_zone_unit_ptr.push_back(unit_ptr_array))) {
 | 
						|
        LOG_WARN("all_zone_unit_ptr push back failed", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    common::ObArray<share::ObZoneReplicaAttrSet> zone_locality;
 | 
						|
    if (!small_tenant) {
 | 
						|
      // bypass
 | 
						|
    } else if (OB_FAIL(tenant_schema.get_zone_replica_attr_array(zone_locality))) {
 | 
						|
      LOG_WARN("fail to get zone replica attr array", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; small_tenant && i < zone_locality.count(); ++i) {
 | 
						|
        const share::ObZoneReplicaAttrSet& this_locality = zone_locality.at(i);
 | 
						|
        if (this_locality.zone_set_.count() > 1) {
 | 
						|
          small_tenant = false;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// @partitions_already_created: it is used for the standalone cluster repeatedly creates tenants.
 | 
						|
// If there is a persistent member_list, there is no need to create partitions.
 | 
						|
int ObDDLService::create_tenant_partitions(const ObCreateTenantArg& arg,
 | 
						|
    const share::schema::ObTenantSchema& tenant_schema, common::ObAddrArray& leader_addrs, const int64_t schema_version,
 | 
						|
    const int64_t frozen_version, ObPartitionCreator& creator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  int64_t start = ObTimeUtility::current_time();
 | 
						|
  const common::ObIArray<obrpc::ObSysVarIdValue>& sys_var_list = arg.sys_var_list_;
 | 
						|
  const common::ObIArray<common::ObString>& pool_list = arg.pool_list_;
 | 
						|
  const bool is_restore = arg.is_restore_;
 | 
						|
  ObArray<ObUnitInfo> tmp;
 | 
						|
  ObReplicaCreator::ZoneUnitArray zone_units;
 | 
						|
  ObReplicaCreator::ZoneUnitPtrArray zone_units_ptr;
 | 
						|
  bool small_tenant = false;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  leader_addrs.reset();
 | 
						|
  bool is_standby = false;
 | 
						|
  bool partitions_already_created = creator.is_tenant_partition_already_exist();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } 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", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(OB_SYS_TENANT_ID, schema_guard))) {
 | 
						|
    LOG_WARN("fail to get schema guard with version in inner table", K(ret));
 | 
						|
  } else if (!partitions_already_created &&
 | 
						|
             OB_FAIL(get_tenant_units_and_scalescope(
 | 
						|
                 sys_var_list, pool_list, tenant_schema, zone_units, zone_units_ptr, small_tenant))) {
 | 
						|
    LOG_WARN("failed to get tenant units", K(ret), K(pool_list));
 | 
						|
  } else if (OB_FAIL(tenant_schema.get_paxos_replica_num(schema_guard, paxos_replica_num))) {
 | 
						|
    LOG_WARN("fail to get paxos replica num", K(ret), K(tenant_schema));
 | 
						|
  } else if (OB_UNLIKELY(paxos_replica_num <= 0 && !is_standby)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("paxos replica num", K(ret), K(paxos_replica_num), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
  } else {
 | 
						|
    LOG_INFO("start to create tenant partitions", K(zone_units), K(tenant_schema));
 | 
						|
    ObRandomZoneSelector random_selector;
 | 
						|
    common::ObZone first_primary_zone;
 | 
						|
    const int64_t partition_id = 0;
 | 
						|
    ObPartitionAddr partition_addr;
 | 
						|
    ObArray<share::ObResourcePoolName> pools;
 | 
						|
    uint64_t tg_id = tenant_schema.get_tenant_id();
 | 
						|
    ObArray<TenantUnitRepCnt*> ten_unit_arr;
 | 
						|
    if (!small_tenant) {
 | 
						|
      tg_id = combine_id(tenant_schema.get_tenant_id(), OB_SYS_TABLEGROUP_ID);
 | 
						|
    }
 | 
						|
    if (OB_FAIL(random_selector.init(*zone_mgr_))) {
 | 
						|
      LOG_WARN("fail to init random zone selector", K(ret));
 | 
						|
    } else if (OB_FAIL(random_selector.update_score(tg_id, partition_id))) {
 | 
						|
      LOG_WARN("update random zone selector score failed", K(ret), K(tg_id), K(partition_id));
 | 
						|
    } else if (OB_FAIL(get_pools(pool_list, pools))) {
 | 
						|
      LOG_WARN("fail to get pools", K(ret), K(pool_list));
 | 
						|
    } else if (is_standby && is_restore) {
 | 
						|
      ret = OB_NOT_SUPPORTED;
 | 
						|
      LOG_WARN("restore in standby cluster is not supported", K(ret));
 | 
						|
    } else if (OB_FAIL(alloc_tenant_partitions(  // alloc cluster private table partitions
 | 
						|
                   unit_mgr_,
 | 
						|
                   zone_mgr_,
 | 
						|
                   pools,
 | 
						|
                   tenant_schema,
 | 
						|
                   zone_units_ptr,
 | 
						|
                   ten_unit_arr,
 | 
						|
                   partition_addr))) {
 | 
						|
      LOG_WARN("fail to alloc tenant partitions", K(ret));
 | 
						|
    } else if (OB_FAIL(tenant_schema.get_first_primary_zone(random_selector, partition_addr, first_primary_zone))) {
 | 
						|
      LOG_WARN("fail to get first primary zone", K(ret));
 | 
						|
    } else {
 | 
						|
      ObReplicaAddr* advised_leader = NULL;
 | 
						|
      int64_t max_score = INT64_MIN;
 | 
						|
      int64_t max_memstore_percent = 0;
 | 
						|
      FOREACH_CNT_X(r, partition_addr, OB_SUCC(ret) && !partitions_already_created)
 | 
						|
      {
 | 
						|
        int64_t score = 0;
 | 
						|
        bool is_active = false;
 | 
						|
        bool is_stopped = false;
 | 
						|
        bool in_service = false;
 | 
						|
        bool is_zone_active = false;
 | 
						|
        if (OB_FAIL(server_mgr_->check_server_active(r->addr_, is_active)) ||
 | 
						|
            OB_FAIL(server_mgr_->check_server_stopped(r->addr_, is_stopped)) ||
 | 
						|
            OB_FAIL(zone_mgr_->check_zone_active(r->zone_, is_zone_active)) ||
 | 
						|
            OB_FAIL(server_mgr_->check_in_service(r->addr_, in_service))) {
 | 
						|
          LOG_WARN("fail to check server stopped", "server", r->addr_, K(ret));
 | 
						|
        } else if (!is_active || is_stopped || !in_service || !is_zone_active) {
 | 
						|
          // bypass
 | 
						|
        } else if (common::REPLICA_TYPE_FULL != r->replica_type_) {
 | 
						|
          // continue
 | 
						|
        } else if (r->zone_ == first_primary_zone && 100 == r->get_memstore_percent()) {
 | 
						|
          advised_leader = r;
 | 
						|
          break;
 | 
						|
        } else if (OB_FAIL(random_selector.get_zone_score(r->zone_, score))) {
 | 
						|
          LOG_WARN("get zone score failed", K(ret), "zone", r->zone_);
 | 
						|
        } else if (r->get_memstore_percent() > max_memstore_percent) {
 | 
						|
          advised_leader = r;
 | 
						|
          max_memstore_percent = r->get_memstore_percent();
 | 
						|
          max_score = score;
 | 
						|
        } else if (r->get_memstore_percent() == max_memstore_percent) {
 | 
						|
          if (score > max_score) {
 | 
						|
            advised_leader = r;
 | 
						|
            max_memstore_percent = r->get_memstore_percent();
 | 
						|
            max_score = score;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_UNLIKELY(NULL == advised_leader) && !is_standby) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("no avaliable leader found", K(ret));
 | 
						|
      } else {
 | 
						|
        if (OB_ISNULL(advised_leader)) {
 | 
						|
        } else {
 | 
						|
          advised_leader->initial_leader_ = true;
 | 
						|
          bool found = false;
 | 
						|
          for (int64_t i = 0; !found && i < leader_addrs.count(); i++) {
 | 
						|
            if (leader_addrs.at(i) == advised_leader->addr_) {
 | 
						|
              found = true;
 | 
						|
            }
 | 
						|
          }
 | 
						|
          if (!found && OB_FAIL(leader_addrs.push_back(advised_leader->addr_))) {
 | 
						|
            LOG_WARN("fail to push back leader addr", K(ret), KP(advised_leader));
 | 
						|
          }
 | 
						|
        }
 | 
						|
        ObTablePartitionAddr table_addr;
 | 
						|
        if (OB_FAIL(table_addr.push_back(partition_addr))) {
 | 
						|
          LOG_WARN("fail to push back", K(ret));
 | 
						|
        } else if (OB_FAIL(do_create_tenant_partitions(
 | 
						|
                       arg, schema_guard, tenant_schema, schema_version, table_addr, frozen_version, creator))) {
 | 
						|
          LOG_WARN("fail to do create tenant partitions", K(ret));
 | 
						|
        } else {
 | 
						|
        }  // no more to do
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  LOG_INFO("create tenant partitions",
 | 
						|
      K(ret),
 | 
						|
      "tenant_id",
 | 
						|
      tenant_schema.get_tenant_id(),
 | 
						|
      "cost",
 | 
						|
      ObTimeUtility::current_time() - start);
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::alloc_tenant_partitions(ObUnitManager* unit_mgr, ObZoneManager* zone_mgr,
 | 
						|
    const ObIArray<share::ObResourcePoolName>& pools, const share::schema::ObTenantSchema& tenant_schema,
 | 
						|
    ObReplicaCreator::ZoneUnitPtrArray& zone_units_ptr, ObIArray<TenantUnitRepCnt*>& ten_unit_arr,
 | 
						|
    ObPartitionAddr& partition_addr)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  int64_t start = ObTimeUtility::current_time();
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  ObArray<share::ObZoneReplicaAttrSet> zone_locality;
 | 
						|
  if (OB_ISNULL(unit_mgr) || OB_ISNULL(zone_mgr)) {
 | 
						|
    LOG_WARN("invalid argument", K(unit_mgr), K(zone_mgr));
 | 
						|
  } else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", KR(ret), K(zone_list));
 | 
						|
  } else if (OB_FAIL(tenant_schema.get_zone_replica_attr_array(zone_locality))) {
 | 
						|
    LOG_WARN("fail to get zone replica attr array", KR(ret));
 | 
						|
  } else if (OB_FAIL(ObLocalityTaskHelp::alloc_logonly_replica(*unit_mgr, pools, zone_locality, partition_addr))) {
 | 
						|
    LOG_WARN("fail to alloc logonly replica", KR(ret), K(zone_locality));
 | 
						|
  } else if (OB_FAIL(ObLocalityTaskHelp::filter_logonly_task(pools, *unit_mgr, zone_locality))) {
 | 
						|
    LOG_WARN("fail to filter logonly task", KR(ret), K(tenant_schema));
 | 
						|
  } else {
 | 
						|
    common::ObPartitionKey pkey;  // indeed, this key is dummy
 | 
						|
    const uint64_t tenant_id = tenant_schema.get_tenant_id();
 | 
						|
    const uint64_t table_id = combine_id(tenant_id, OB_ALL_DUMMY_TID);
 | 
						|
    const int64_t partition_id = 0;
 | 
						|
    const int64_t partition_cnt = 0;
 | 
						|
    ObAliveZoneUnitsProvider zone_units_provider;
 | 
						|
    ObCreateTableReplicaByLocality addr_allocator(*zone_mgr,
 | 
						|
        zone_units_provider,
 | 
						|
        zone_locality,
 | 
						|
        OB_CREATE_TABLE_MODE_STRICT,
 | 
						|
        zone_list,
 | 
						|
        tenant_schema.get_tenant_id());
 | 
						|
    if (OB_FAIL(zone_units_provider.init(zone_units_ptr))) {
 | 
						|
      LOG_WARN("fail to init zone units", K(ret));
 | 
						|
    } else if (OB_FAIL(addr_allocator.init())) {
 | 
						|
      LOG_WARN("fail init replica addr allocator", K(ret));
 | 
						|
    } else if (OB_FAIL(addr_allocator.prepare_for_next_partition(partition_addr))) {
 | 
						|
      LOG_WARN("fail to prepare variables for next partition", K(ret));
 | 
						|
    } else if (OB_FAIL(pkey.init(table_id, partition_id, partition_cnt))) {
 | 
						|
      LOG_WARN("fail to init pkey", K(ret));
 | 
						|
    } else {
 | 
						|
      ObReplicaAddr replica_addr;
 | 
						|
      while (OB_SUCC(ret)) {
 | 
						|
        const bool non_partition =
 | 
						|
            false;  // The partitions of system tenants all belong to the same pg, no need to tile
 | 
						|
        if (OB_FAIL(addr_allocator.get_next_replica(pkey, ten_unit_arr, non_partition, replica_addr))) {
 | 
						|
          if (OB_ITER_END == ret) {
 | 
						|
            ret = OB_SUCCESS;
 | 
						|
            break;
 | 
						|
          } else {
 | 
						|
            LOG_WARN("fail to get next replica", KR(ret));
 | 
						|
          }
 | 
						|
        } else if (OB_FAIL(partition_addr.push_back(replica_addr))) {
 | 
						|
          LOG_WARN("fail to add replica addr", KR(ret));
 | 
						|
        }  // no more to do
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(addr_allocator.fill_all_rest_server_with_replicas(partition_addr))) {
 | 
						|
        LOG_WARN("fail to fill all rest server with replicas", KR(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  LOG_INFO("alloc tenant partitions",
 | 
						|
      K(ret),
 | 
						|
      "tenant_id",
 | 
						|
      tenant_schema.get_tenant_id(),
 | 
						|
      "cost",
 | 
						|
      ObTimeUtility::current_time() - start);
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::do_create_tenant_partitions(const ObCreateTenantArg& arg,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const share::schema::ObTenantSchema& tenant_schema,
 | 
						|
    const int64_t schema_version, const ObITablePartitionAddr& table_addr, const int64_t frozen_version,
 | 
						|
    ObPartitionCreator& creator)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(frozen_version);
 | 
						|
  const bool is_restore = arg.is_restore_;
 | 
						|
  const bool is_replay_schema = arg.is_replay_schema_;
 | 
						|
  int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  bool is_standby = false;
 | 
						|
  const int64_t tenant_id = tenant_schema.get_tenant_id();
 | 
						|
  int64_t start = ObTimeUtility::current_time();
 | 
						|
  share::ObSimpleFrozenStatus frozen_status;
 | 
						|
  if (OB_FAIL(get_is_standby_cluster(is_standby))) {
 | 
						|
    LOG_WARN("failed to get is_standby", K(ret));
 | 
						|
  } else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id, frozen_status))) {
 | 
						|
    LOG_WARN("fail to get freeze info", K(ret), K(frozen_status));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_FAIL(tenant_schema.get_paxos_replica_num(schema_guard, paxos_replica_num))) {
 | 
						|
    LOG_WARN("fail to get paxos replica num", KR(ret));
 | 
						|
  } else if (OB_UNLIKELY(paxos_replica_num <= 0)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("paxos replica num error", K(ret), K(paxos_replica_num));
 | 
						|
  }
 | 
						|
 | 
						|
  // For physical restore, init restore_partition_map
 | 
						|
  common::hash::ObHashMap<uint64_t, ObReplicaRestoreStatus> restore_partition_map;
 | 
						|
  int64_t tenant_space_tables_cnt = ARRAYSIZEOF(tenant_space_tables);
 | 
						|
  if (FAILEDx(restore_partition_map.create(hash::cal_next_prime(tenant_space_tables_cnt), "ResPartMap"))) {
 | 
						|
    LOG_WARN("failed to create restore partition map", KR(ret));
 | 
						|
  } else {
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < arg.restore_pkeys_.count(); i++) {
 | 
						|
      uint64_t pure_id = extract_pure_id(arg.restore_pkeys_.at(i).get_table_id());
 | 
						|
      if (OB_FAIL(restore_partition_map.set_refactored(pure_id, REPLICA_RESTORE_DATA, 0 /*not overwrite*/))) {
 | 
						|
        LOG_WARN("fail to set restore data key", KR(ret), K(pure_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < arg.restore_log_pkeys_.count(); i++) {
 | 
						|
      uint64_t pure_id = extract_pure_id(arg.restore_log_pkeys_.at(i).get_table_id());
 | 
						|
      if (OB_FAIL(restore_partition_map.set_refactored(pure_id, REPLICA_RESTORE_ARCHIVE_DATA, 0 /*not overwrite*/))) {
 | 
						|
        LOG_WARN("fail to set restore log key", KR(ret), K(pure_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  for (int64_t i = 0; OB_SUCC(ret) && i < ARRAYSIZEOF(tenant_space_tables); ++i) {
 | 
						|
    const uint64_t tid = combine_id(OB_SYS_TENANT_ID, tenant_space_tables[i]);
 | 
						|
    const ObTableSchema* table = NULL;
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schema(tid, table))) {
 | 
						|
      LOG_WARN("get table schema failed", K(ret), K(tid));
 | 
						|
    } else if (NULL == table) {
 | 
						|
      // do-nothing, tenant_space_table can be dropped, ignore it!
 | 
						|
    } else if (!table->has_self_partition()) {
 | 
						|
      ;
 | 
						|
    } else {
 | 
						|
      ObTableSchema copy;
 | 
						|
      if (OB_FAIL(copy.assign(*table))) {
 | 
						|
        LOG_WARN("fail to assign schema", K(ret));
 | 
						|
      } else {
 | 
						|
        ObArray<ObTableSchema> schemas;
 | 
						|
        copy.set_table_id(combine_id(tenant_schema.get_tenant_id(), tid));
 | 
						|
        if (copy.has_self_partition()) {
 | 
						|
          // reset partition option
 | 
						|
          copy.reset_partition_schema();
 | 
						|
          copy.get_part_option().set_max_used_part_id(0);
 | 
						|
          copy.get_part_option().set_partition_cnt_within_partition_table(0);
 | 
						|
        }
 | 
						|
        ObArray<int64_t> part_ids;
 | 
						|
        ObPartIdsGenerator gen(copy);
 | 
						|
        ObCreateTableMode create_mode = OB_CREATE_TABLE_MODE_LOOSE;
 | 
						|
        int64_t non_paxos_replica_num = 0;
 | 
						|
        int64_t restore = REPLICA_NOT_RESTORE;
 | 
						|
        int64_t table_id = copy.get_table_id();
 | 
						|
        if (OB_FAIL(ret)) {
 | 
						|
        } else if (is_restore) {  // physical restore
 | 
						|
          create_mode = OB_CREATE_TABLE_MODE_PHYSICAL_RESTORE;
 | 
						|
          if (ObSysTableChecker::is_backup_private_tenant_table(table_id)) {
 | 
						|
            restore = REPLICA_NOT_RESTORE;
 | 
						|
          } else {
 | 
						|
            ObReplicaRestoreStatus status;
 | 
						|
            int hash_ret = restore_partition_map.get_refactored(extract_pure_id(table_id), status);
 | 
						|
            if (OB_SUCCESS == hash_ret) {
 | 
						|
              restore = status;
 | 
						|
            } else if (OB_HASH_NOT_EXIST == hash_ret) {
 | 
						|
              // create sys table in tenant space which were not backuped in lower version.
 | 
						|
              restore = REPLICA_NOT_RESTORE;
 | 
						|
            } else {
 | 
						|
              ret = hash_ret;
 | 
						|
              LOG_WARN("fail to get key", K(ret), K(table_id));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        } else if (is_standby) {  // standby cluster
 | 
						|
          if (!ObMultiClusterUtil::is_cluster_private_table(table_id)) {
 | 
						|
            restore = REPLICA_RESTORE_STANDBY;
 | 
						|
          }
 | 
						|
        }
 | 
						|
        if (FAILEDx(gen.gen(part_ids))) {
 | 
						|
          LOG_WARN("generate part ids failed", K(ret));
 | 
						|
        } else if (OB_FAIL(schemas.push_back(copy))) {
 | 
						|
          LOG_WARN("fail to push back schemas", K(ret));
 | 
						|
        } else if (OB_FAIL(prepare_create_partitions(creator,
 | 
						|
                       copy.get_table_id(),
 | 
						|
                       schema_version,
 | 
						|
                       copy.get_all_part_num(),
 | 
						|
                       copy.get_partition_cnt(),
 | 
						|
                       paxos_replica_num,
 | 
						|
                       non_paxos_replica_num,
 | 
						|
                       part_ids,
 | 
						|
                       table_addr,
 | 
						|
                       schemas,
 | 
						|
                       false,
 | 
						|
                       is_standby,
 | 
						|
                       create_mode,
 | 
						|
                       restore,
 | 
						|
                       frozen_status))) {
 | 
						|
          LOG_WARN("fail to create partitions", K(ret));
 | 
						|
        } else {
 | 
						|
        }  // no more to do
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (OB_FAIL(creator.execute())) {
 | 
						|
      LOG_WARN("execute create partition failed", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  LOG_INFO("do create tenant partitions",
 | 
						|
      K(ret),
 | 
						|
      "tenant_id",
 | 
						|
      tenant_schema.get_tenant_id(),
 | 
						|
      "cost",
 | 
						|
      ObTimeUtility::current_time() - start);
 | 
						|
  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.
 | 
						|
 */
 | 
						|
int ObDDLService::drop_tenant(const ObDropTenantArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
  const bool if_exist = arg.if_exist_;
 | 
						|
  const bool drop_force = !arg.delay_to_drop_;
 | 
						|
  const ObTenantSchema* tenant_schema = NULL;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  ObArray<ObResourcePoolName> pool_names;
 | 
						|
  ObArray<share::ObResourcePool*> pools;
 | 
						|
  ObRootService* rootservice = GCTX.root_service_;
 | 
						|
  ret = E(EventTable::EN_DROP_TENANT_FAILED) OB_SUCCESS;
 | 
						|
  bool is_standby = false;
 | 
						|
  uint64_t tenant_id = common::OB_INVALID_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 (OB_FAIL(schema_guard.get_tenant_info(arg.tenant_name_, tenant_schema))) {
 | 
						|
    LOG_WARN("fail to gt tenant info", KR(ret));
 | 
						|
  } 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 (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(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("start transaction failed", KR(ret));
 | 
						|
  } 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_);
 | 
						|
    tenant_id = tenant_schema->get_tenant_id();
 | 
						|
    // 1.drop tenant force
 | 
						|
    if (drop_force) {
 | 
						|
      if (OB_FAIL(drop_resource_pool_pre(tenant_id, pool_names, is_standby, trans))) {
 | 
						|
        LOG_WARN("fail to drop resource pool pre", KR(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.drop_tenant(tenant_id, trans, &arg.ddl_stmt_str_))) {
 | 
						|
        LOG_WARN("ddl_operator drop_tenant failed", K(tenant_id), KR(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.drop_restore_point(tenant_id, trans))) {
 | 
						|
        LOG_WARN("fail to drop restore point", K(ret), K(tenant_id));
 | 
						|
      } else if (tenant_schema->is_in_recyclebin()) {
 | 
						|
        // try recycle record from __all_recyclebin
 | 
						|
        ObArray<ObRecycleObject> 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(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));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  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(), 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::drop_resource_pool_pre(const uint64_t tenant_id, ObIArray<ObResourcePoolName>& 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, 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));
 | 
						|
    } else if (OB_FAIL(ObTenantUtils::remove_ineffective_task(trans, tenant_id))) {
 | 
						|
      LOG_WARN("fail to drop ineffective task", KR(ret), K(tenant_id));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::drop_resource_pool_final(
 | 
						|
    const uint64_t tenant_id, const bool is_standby, ObIArray<ObResourcePoolName>& 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(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_);
 | 
						|
    trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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<ObResourcePoolName> 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_);
 | 
						|
    trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
    const uint64_t tenant_id = tenant_schema->get_tenant_id();
 | 
						|
    bool is_standby = false;
 | 
						|
    if (OB_FAIL(get_is_standby_cluster(is_standby))) {
 | 
						|
      LOG_WARN("fail to get is standby", K(ret));
 | 
						|
    } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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;
 | 
						|
  trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
  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(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("start transaction failed", K(ret));
 | 
						|
  } else {
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    ObTenantSchema new_tenant_schema = *tenant_schema;
 | 
						|
    new_tenant_schema.reset_sysvars();
 | 
						|
    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));
 | 
						|
  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();
 | 
						|
  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_info))) {
 | 
						|
    LOG_WARN("get tenant info failed", 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_SUCC(sys_variable_schema->get_sysvar_schema(var_name, old_schema))) {
 | 
						|
    if (arg.if_not_exist_) {
 | 
						|
      // do nothing
 | 
						|
    } else {
 | 
						|
      ret = OB_ERR_PARAM_DUPLICATE;
 | 
						|
      LOG_WARN("system variable duplicated", K(var_name));
 | 
						|
    }
 | 
						|
  } else if (OB_ERR_SYS_VARIABLE_UNKNOWN != ret) {
 | 
						|
    LOG_WARN("get tenant system variable failed", K(arg));
 | 
						|
  } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("start transaction failed", K(ret));
 | 
						|
  } else {
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    int64_t schema_version = OB_INVALID_VERSION;
 | 
						|
    ObSysVariableSchema new_sys_variable_schema = *sys_variable_schema;
 | 
						|
    new_sys_variable_schema.reset_sysvars();
 | 
						|
    if (OB_UNLIKELY(!arg.is_valid())) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("add sysvar argument is invalid", K(arg));
 | 
						|
    } else if (OB_UNLIKELY(!new_sys_variable_schema.is_valid())) {
 | 
						|
      ret = new_sys_variable_schema.get_err_ret();
 | 
						|
      LOG_WARN("new tenant info is invalid", K(ret));
 | 
						|
    } else if (OB_FAIL(new_sys_variable_schema.add_sysvar_schema(arg.sysvar_))) {
 | 
						|
      LOG_WARN("add sysvar schema to new tenant info failed", K(ret));
 | 
						|
    } else if (OB_FAIL(try_check_tenant_turn_gts_on_condition(
 | 
						|
                   schema_guard, *tenant_info, *tenant_info, new_sys_variable_schema))) {
 | 
						|
      LOG_WARN("fail to check tenant turn gts on condition", K(ret));
 | 
						|
    } 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, schema_version, trans, operation_type, &arg.ddl_stmt_str_))) {
 | 
						|
        LOG_WARN("alter tenant info 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::try_check_tenant_turn_gts_on_condition(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTenantSchema& old_tenant_schema, const share::schema::ObTenantSchema& new_tenant_schema,
 | 
						|
    const share::schema::ObSysVariableSchema& new_sys_variable)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = old_tenant_schema.get_tenant_id();
 | 
						|
  const ObSysVariableSchema* old_sys_variable = NULL;
 | 
						|
  GTSOperation gts_operation = GTS_OPERATION_INVALID;
 | 
						|
  if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2000) {
 | 
						|
    // Do not check before the upgrade is completed
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, old_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(old_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_tenant_gts_operation(*old_sys_variable, new_sys_variable, gts_operation))) {
 | 
						|
    LOG_WARN("fail to check tenant gts operation", K(ret));
 | 
						|
  } else if (GTS_OPERATION_INVALID == gts_operation) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("unexpected gts operation", K(ret));
 | 
						|
  } else if (GTS_TURN_ON != gts_operation) {
 | 
						|
    // The gts switch is not adjusted, or gts is turned off, bypass
 | 
						|
  } else if (OB_FAIL(check_tenant_primary_zone_gts_condition(
 | 
						|
                 schema_guard, new_tenant_schema, new_sys_variable, *old_sys_variable))) {
 | 
						|
    LOG_WARN("fail to check tenant primary zone gts condition", K(ret));
 | 
						|
  } else {
 | 
						|
    common::ObArray<const ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
    common::ObArray<const ObTablegroupSchema*> tablegroup_schemas;
 | 
						|
    common::ObArray<const ObDatabaseSchema*> database_schemas;
 | 
						|
    const uint64_t tenant_id = new_tenant_schema.get_tenant_id();
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(tenant_id, table_schemas))) {
 | 
						|
      LOG_WARN("fail to get table schemas in tenant", K(ret), K(tenant_id));
 | 
						|
    } else if (OB_FAIL(schema_guard.get_tablegroup_schemas_in_tenant(tenant_id, tablegroup_schemas))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schemas in tenant", K(ret), K(tenant_id));
 | 
						|
    } else if (OB_FAIL(schema_guard.get_database_schemas_in_tenant(tenant_id, database_schemas))) {
 | 
						|
      LOG_WARN("fail to get database schemas in tenant", K(ret), K(tenant_id));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); ++i) {
 | 
						|
        const ObSimpleTableSchemaV2* table_schema = table_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == table_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("table schema ptr is null", K(ret));
 | 
						|
        } else if (!table_schema->has_partition()) {
 | 
						|
          // bypass
 | 
						|
        } else if (OB_FAIL(check_table_primary_zone_gts_condition(
 | 
						|
                       old_tenant_schema, new_tenant_schema, new_sys_variable, *table_schema, schema_guard))) {
 | 
						|
          LOG_WARN("fail to check table primary zone gts condition", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < tablegroup_schemas.count(); ++i) {
 | 
						|
        const ObTablegroupSchema* tablegroup_schema = tablegroup_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == tablegroup_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("table schema ptr is null", K(ret));
 | 
						|
        } else if (!is_new_tablegroup_id(tablegroup_schema->get_tablegroup_id())) {
 | 
						|
          // bypass
 | 
						|
        } else if (OB_FAIL(check_new_tablegroup_primary_zone_gts_condition(
 | 
						|
                       old_tenant_schema, new_tenant_schema, new_sys_variable, *tablegroup_schema, schema_guard))) {
 | 
						|
          LOG_WARN("fail to check table primary zone gts condition", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < database_schemas.count(); ++i) {
 | 
						|
        const ObDatabaseSchema* database_schema = database_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == database_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("table schema ptr is null", K(ret));
 | 
						|
        } else if (OB_FAIL(check_database_primary_zone_gts_condition(
 | 
						|
                       old_tenant_schema, new_tenant_schema, new_sys_variable, *database_schema, schema_guard))) {
 | 
						|
          LOG_WARN("fail to check table primary zone gts condition", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::do_modify_system_variable(
 | 
						|
    uint64_t tenant_id, const ObSysVarSchema& modify_var, ObSysVarSchema& new_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  constexpr int64_t BUF_SIZE = 64;
 | 
						|
  char value_buf[BUF_SIZE];
 | 
						|
  ObString new_value(modify_var.get_value());
 | 
						|
 | 
						|
  if (0 == MEMCMP(modify_var.get_name().ptr(), PARALLEL_MAX_SERVERS, modify_var.get_name().length())) {
 | 
						|
    // If the value of parallel_max_servers exceeds the maximum allowed by unit, reduce the configuration
 | 
						|
    LOG_INFO("parallel_max_servers update", K(modify_var));
 | 
						|
    common::ObArray<share::ObUnitInfo> units;
 | 
						|
    ObRootService* rootservice = GCTX.root_service_;
 | 
						|
    if (OB_ISNULL(rootservice)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("rootservice null", K(ret), KP(rootservice));
 | 
						|
    } else if (OB_FAIL(rootservice->get_unit_mgr().get_active_unit_infos_by_tenant(tenant_id, units))) {
 | 
						|
      LOG_WARN("fail get units", K(modify_var), K(ret));
 | 
						|
    } else {
 | 
						|
      int64_t user_max_servers =
 | 
						|
          ObCharset::strntoll(modify_var.get_value().ptr(), modify_var.get_value().length(), 10, &ret);
 | 
						|
      int64_t max_servers = 0;
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
        LOG_WARN("fail convert value from str to int", K(modify_var), K(ret));
 | 
						|
      } else if (user_max_servers < 0) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("unexpected parallel_max_server value", K(user_max_servers), K(modify_var), K(ret));
 | 
						|
      } else if (OB_FAIL(
 | 
						|
                     sql::ObPxAdmissionUtil::check_parallel_max_servers_value(units, user_max_servers, max_servers))) {
 | 
						|
        LOG_WARN("fail check parallel max servers", K(modify_var), K(user_max_servers), K(ret));
 | 
						|
      } else if (max_servers >= user_max_servers) {
 | 
						|
        // nop, don't change any value
 | 
						|
      } else if (OB_FAIL(databuff_printf(value_buf, BUF_SIZE, "%ld", max_servers))) {
 | 
						|
        LOG_WARN("fail do print buf", K(max_servers), K(BUF_SIZE), K(ret));
 | 
						|
      } else {
 | 
						|
        LOG_INFO("downgrade parallel max servers", K(user_max_servers), K(max_servers));
 | 
						|
        LOG_USER_WARN(OB_ERR_DOWNGRADE_PARALLEL_MAX_SERVERS, user_max_servers, max_servers);
 | 
						|
        new_value.assign(value_buf, STRLEN(value_buf));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    // new_value deep copy to new_schema
 | 
						|
    if (OB_FAIL(new_schema.set_value(new_value))) {
 | 
						|
      LOG_WARN("fail update system variable", K(modify_var), K(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(do_modify_system_variable(tenant_id, modify_var, new_schema))) {
 | 
						|
              LOG_WARN("fail modify system variable", K(modify_var), 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) {
 | 
						|
      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_);
 | 
						|
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed", K(ret));
 | 
						|
      } else if (OB_FAIL(try_check_tenant_turn_gts_on_condition(
 | 
						|
                     schema_guard, *tenant_schema, *tenant_schema, new_sys_variable_schema))) {
 | 
						|
        LOG_WARN("fail to check tenant turn gts on condition", K(ret));
 | 
						|
      } 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));
 | 
						|
      }
 | 
						|
      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;
 | 
						|
  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_ISNULL(ora_user_trans) && OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } else {
 | 
						|
      if (database_schema.get_primary_zone().empty()) {
 | 
						|
        // bypass, since the primary zone is empty, derive from upper level,
 | 
						|
        // no need to statistic primary zone count
 | 
						|
      } else {
 | 
						|
        if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans,
 | 
						|
                schema_guard,
 | 
						|
                true /*inc*/,
 | 
						|
                1,
 | 
						|
                database_schema.get_tenant_id()))) {
 | 
						|
          LOG_WARN("fail to try inc tenant primary zone entity count", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
 | 
						|
      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<ObCollationType>(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(check_create_database_replica_options(database_schema, schema_guard))) {
 | 
						|
          LOG_WARN("fail to check database replica options", K(ret), K(database_schema));
 | 
						|
        } 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: {
 | 
						|
          ret = new_database_schema.set_primary_zone(alter_database_schema.get_primary_zone());
 | 
						|
          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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
    } 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));
 | 
						|
      } else if (OB_FAIL(check_alter_database_replica_options(
 | 
						|
                     arg, new_database_schema, *origin_database_schema, schema_guard))) {
 | 
						|
        LOG_WARN("fail to check alter database replica options", K(ret));
 | 
						|
      } else if (origin_database_schema->get_primary_zone().empty() && new_database_schema.get_primary_zone().empty()) {
 | 
						|
        // bypass
 | 
						|
      } else if (!origin_database_schema->get_primary_zone().empty() &&
 | 
						|
                 !new_database_schema.get_primary_zone().empty()) {
 | 
						|
        // bypass
 | 
						|
      } else if (!origin_database_schema->get_primary_zone().empty() &&
 | 
						|
                 new_database_schema.get_primary_zone().empty()) {
 | 
						|
        if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(trans, schema_guard, false /*sub*/, 1, tenant_id))) {
 | 
						|
          LOG_WARN("fail to try modify primary zone entity count", K(ret));
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(trans, schema_guard, true /*add*/, 1, tenant_id))) {
 | 
						|
          LOG_WARN("fail to try modify primary zone entity count", 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, UInt64& affected_row, 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_;
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  uint64_t table_count = 0;
 | 
						|
  ObArray<uint64_t> 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();
 | 
						|
    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_ISNULL(ora_user_trans) && OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } else {
 | 
						|
      // drop mv force
 | 
						|
      const ObTableSchema* schema = NULL;
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < table_count; i++) {
 | 
						|
        if (OB_FAIL(schema_guard.get_table_schema(table_ids.at(i), schema))) {
 | 
						|
          LOG_WARN("fail to get table schema", K(ret), "table_id", table_ids.at(i));
 | 
						|
        } 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 (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 {
 | 
						|
        int64_t pz_count = 0;
 | 
						|
        if (db_schema->get_primary_zone().empty()) {
 | 
						|
          // derive from upper level
 | 
						|
        } else {
 | 
						|
          ++pz_count;
 | 
						|
        }
 | 
						|
        for (int64_t i = 0; OB_SUCC(ret) && i < table_ids.count(); ++i) {
 | 
						|
          const ObSimpleTableSchemaV2* schema = nullptr;
 | 
						|
          if (OB_FAIL(schema_guard.get_table_schema(table_ids.at(i), schema))) {
 | 
						|
            LOG_WARN("fail to get table schema", K(ret), "table_id", table_ids.at(i));
 | 
						|
          } else if (nullptr == schema) {
 | 
						|
            // bypass
 | 
						|
          } else if (schema->get_primary_zone().empty()) {
 | 
						|
            // bypass
 | 
						|
          } else {
 | 
						|
            pz_count++;
 | 
						|
          }
 | 
						|
        }
 | 
						|
        if (OB_FAIL(ret)) {
 | 
						|
          // FAIL
 | 
						|
        } else if (OB_FAIL(
 | 
						|
                       try_modify_tenant_primary_zone_entity_count(OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans,
 | 
						|
                           schema_guard,
 | 
						|
                           false /*sub*/,
 | 
						|
                           pz_count,
 | 
						|
                           db_schema->get_tenant_id()))) {
 | 
						|
          LOG_WARN("fail to sub tenant primary zone entity count", K(ret));
 | 
						|
        } 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 (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_partitions_for_create(const share::schema::ObTablegroupSchema& tablegroup_schema,
 | 
						|
    const common::ObIArray<ObPartitionAddr>& tablegroup_addr, const common::ObIArray<int64_t>& part_ids,
 | 
						|
    const share::ObSimpleFrozenStatus& frozen_status, const obrpc::ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::schema::ObSchemaGetterGuard schema_guard;
 | 
						|
  ObPartitionCreator creator(*rpc_proxy_, *pt_operator_, server_mgr_);
 | 
						|
  creator.set_create_mode(create_mode);
 | 
						|
  int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  int64_t non_paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
  const uint64_t tenant_id = tablegroup_schema.get_tenant_id();
 | 
						|
  const uint64_t tablegroup_id = tablegroup_schema.get_tablegroup_id();
 | 
						|
  if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || !frozen_status.is_valid())) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(tenant_id), K(frozen_status));
 | 
						|
  } 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 with version", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(calc_schema_replica_num(
 | 
						|
                 schema_guard, tablegroup_schema, paxos_replica_num, non_paxos_replica_num, create_mode))) {
 | 
						|
    LOG_WARN("fail to calc schema replica num",
 | 
						|
        K(ret),
 | 
						|
        "tg_id",
 | 
						|
        tablegroup_schema.get_tablegroup_id(),
 | 
						|
        K(paxos_replica_num),
 | 
						|
        K(non_paxos_replica_num));
 | 
						|
  } else if (tablegroup_addr.count() != part_ids.count()) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN(
 | 
						|
        "array count unexpected", K(ret), "tg_addr_count", tablegroup_addr.count(), "part_ids_count", part_ids.count());
 | 
						|
  } else {
 | 
						|
    obrpc::ObCreatePartitionArg arg;
 | 
						|
    share::ObSplitPartition split_info;
 | 
						|
    if (tablegroup_schema.is_in_splitting()) {
 | 
						|
      if (OB_FAIL(ObPartitionSplitHelper::build_split_info(tablegroup_schema, split_info))) {
 | 
						|
        LOG_WARN("failed to build split info", K(ret), K(split_info), K(tablegroup_schema));
 | 
						|
      } else if (!split_info.is_valid()) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("split info is invalid", K(ret), K(split_info), K(tablegroup_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < tablegroup_addr.count(); ++i) {
 | 
						|
      arg.reset();
 | 
						|
      arg.schema_version_ = tablegroup_schema.get_schema_version();
 | 
						|
      arg.last_replay_log_id_ = 0 /* tablegroup is 0 */;
 | 
						|
      const int64_t now = ObTimeUtility::current_time();
 | 
						|
      const ObPartitionAddr& part_addr = tablegroup_addr.at(i);
 | 
						|
      const int64_t partition_id = part_ids.at(i);
 | 
						|
      if (OB_FAIL(fill_partition_member_list(part_addr, now, paxos_replica_num, arg))) {
 | 
						|
        LOG_WARN("fail to fill create partition arg memberlist", KR(ret), K(arg), K(paxos_replica_num));
 | 
						|
      } else {
 | 
						|
        FOREACH_CNT_X(a, part_addr, OB_SUCC(ret))
 | 
						|
        {
 | 
						|
          ObPartitionReplica flag_replica;
 | 
						|
          if (OB_UNLIKELY(nullptr == a || nullptr == rpc_proxy_)) {
 | 
						|
            ret = OB_ERR_UNEXPECTED;
 | 
						|
            LOG_WARN("addr or rpc_proxy is null", K(ret));
 | 
						|
          } else {
 | 
						|
            int64_t restore = REPLICA_NOT_RESTORE;
 | 
						|
            if (OB_CREATE_TABLE_MODE_PHYSICAL_RESTORE == create_mode) {
 | 
						|
              ret = OB_NOT_SUPPORTED;
 | 
						|
              LOG_WARN("create mode is invalid", K(ret), K(create_mode));
 | 
						|
            } else if (OB_CREATE_TABLE_MODE_RESTORE == create_mode &&
 | 
						|
                       ObReplicaTypeCheck::is_replica_with_ssstore(a->replica_type_)) {
 | 
						|
              // logical restore
 | 
						|
              restore = REPLICA_LOGICAL_RESTORE_DATA;
 | 
						|
            } else {
 | 
						|
              restore = REPLICA_NOT_RESTORE;
 | 
						|
            }
 | 
						|
            if (OB_FAIL(fill_create_partition_arg(tablegroup_id,
 | 
						|
                    tablegroup_schema.get_partition_cnt(),
 | 
						|
                    paxos_replica_num,
 | 
						|
                    non_paxos_replica_num,
 | 
						|
                    partition_id,
 | 
						|
                    *a,
 | 
						|
                    now,
 | 
						|
                    false /*is_bootstrap*/,
 | 
						|
                    false /*is_standby*/,
 | 
						|
                    restore,
 | 
						|
                    frozen_status,
 | 
						|
                    arg))) {
 | 
						|
              LOG_WARN("fail to fill ObCreatePartitionArg", K(ret), K(tablegroup_id));
 | 
						|
            } else if (OB_FAIL(fill_flag_replica(tablegroup_id,
 | 
						|
                           tablegroup_schema.get_partition_cnt(),
 | 
						|
                           partition_id,
 | 
						|
                           arg,
 | 
						|
                           *a,
 | 
						|
                           flag_replica))) {
 | 
						|
              LOG_WARN("fail to fill flag replica", K(ret), K(tablegroup_id));
 | 
						|
            } else if (OB_FAIL(creator.add_flag_replica(flag_replica))) {
 | 
						|
              LOG_WARN("fail to add flag replica to partition creator", K(ret), K(flag_replica));
 | 
						|
            } else if (OB_FAIL(arg.split_info_.assign(split_info))) {
 | 
						|
              LOG_WARN("failed to assign split info", K(ret), K(split_info), K(arg));
 | 
						|
            } else if (OB_FAIL(creator.add_create_partition_arg(a->addr_, arg))) {
 | 
						|
              LOG_WARN("fail to add create partition argument", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(creator.execute())) {
 | 
						|
        LOG_WARN("fail execute create partition", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
int ObDDLService::create_tablegroup_partitions_for_physical_restore(const obrpc::ObRestorePartitionsArg& arg,
 | 
						|
    const common::hash::ObHashSet<int64_t>& base_part_id_set, const share::ObSimpleFrozenStatus frozen_status,
 | 
						|
    const int64_t last_schema_version, ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  RS_TRACE(create_tablegroup_partitions_begin);
 | 
						|
  const uint64_t tablegroup_id = arg.schema_id_;
 | 
						|
  const ObCreateTableMode create_mode = ObCreateTableMode::OB_CREATE_TABLE_MODE_LOOSE;
 | 
						|
  const ObTablegroupSchema* tablegroup_schema = NULL;
 | 
						|
  common::ObArray<ObPartitionAddr> tablegroup_addr;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_ISNULL(root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_balancer_ is null", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_tablegroup_schema(tablegroup_id, tablegroup_schema))) {
 | 
						|
    LOG_WARN("fail to get tablegroup", K(ret), K(tablegroup_id));
 | 
						|
  } else if (OB_ISNULL(tablegroup_schema)) {
 | 
						|
    ret = OB_TABLEGROUP_NOT_EXIST;
 | 
						|
    LOG_WARN("tablegroup not exist", K(ret), K(tablegroup_id));
 | 
						|
  } else if (OB_FAIL(root_balancer_->alloc_tablegroup_partitions_for_create(
 | 
						|
                 *tablegroup_schema, create_mode, tablegroup_addr))) {
 | 
						|
    LOG_WARN("fail to alloc tablegroup partitions for create", K(ret));
 | 
						|
  } else if (OB_FAIL(create_partitions_for_physical_restore(schema_guard,
 | 
						|
                 *tablegroup_schema,
 | 
						|
                 arg,
 | 
						|
                 base_part_id_set,
 | 
						|
                 frozen_status,
 | 
						|
                 last_schema_version,
 | 
						|
                 tablegroup_addr))) {
 | 
						|
    LOG_WARN("fail to create restore partitions", K(ret), K(tablegroup_id));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_partitions_for_physical_restore(ObSchemaGetterGuard& schema_guard,
 | 
						|
    const ObPartitionSchema& partition_schema, const obrpc::ObRestorePartitionsArg& restore_arg,
 | 
						|
    const common::hash::ObHashSet<int64_t>& base_part_id_set, const share::ObSimpleFrozenStatus frozen_status,
 | 
						|
    const int64_t last_schema_version, const ObIArray<ObPartitionAddr>& addrs)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t schema_id = restore_arg.schema_id_;
 | 
						|
  const ObCreateTableMode create_mode = ObCreateTableMode::OB_CREATE_TABLE_MODE_LOOSE;
 | 
						|
  bool is_standby = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (!restore_arg.is_valid() || addrs.count() <= 0 || last_schema_version <= OB_CORE_SCHEMA_VERSION) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("schema verison is invalid", K(ret), K(last_schema_version), K(restore_arg), "addr_cnt", addrs.count());
 | 
						|
  } else if (!partition_schema.has_self_partition()) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("table/tablegroup should has self partition", K(ret), K(schema_id));
 | 
						|
  } else {
 | 
						|
    ObPartIdsGenerator gen(partition_schema);
 | 
						|
    ObArray<int64_t> part_ids;
 | 
						|
    int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
    int64_t non_paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
    ObArray<ObTableSchema> base_table_schemas;
 | 
						|
    ObArray<ObTableSchema> inc_table_schemas;
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_FAIL(gen.gen(part_ids))) {
 | 
						|
      LOG_WARN("fail to generate part ids", K(ret));
 | 
						|
    } else if (addrs.count() != part_ids.count()) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("array count unexpected", K(ret), "addr_count", addrs.count(), "part_ids_count", part_ids.count());
 | 
						|
    } else if (OB_FAIL(calc_schema_replica_num(
 | 
						|
                   schema_guard, partition_schema, paxos_replica_num, non_paxos_replica_num, create_mode))) {
 | 
						|
      LOG_WARN("fail to calc schema replica num", K(ret), K(schema_id), K(paxos_replica_num), K(non_paxos_replica_num));
 | 
						|
    } else if (OB_FAIL(construct_table_schemas_for_physical_restore(
 | 
						|
                   schema_guard, restore_arg, base_part_id_set, part_ids, base_table_schemas, inc_table_schemas))) {
 | 
						|
      LOG_WARN("fail to construct table schemas", K(ret), K(restore_arg));
 | 
						|
    } else {
 | 
						|
      ObPartitionCreator creator(*rpc_proxy_, *pt_operator_, server_mgr_);
 | 
						|
      creator.set_create_mode(create_mode);
 | 
						|
      obrpc::ObCreatePartitionArg arg;
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < addrs.count(); ++i) {
 | 
						|
        arg.reset();
 | 
						|
        arg.schema_version_ = last_schema_version;
 | 
						|
        arg.last_replay_log_id_ = 0;
 | 
						|
        const int64_t now = ObTimeUtility::current_time();
 | 
						|
        const ObPartitionAddr& part_addr = addrs.at(i);
 | 
						|
        const int64_t partition_id = part_ids.at(i);
 | 
						|
        int64_t restore = REPLICA_NOT_RESTORE;
 | 
						|
        int hash_ret = base_part_id_set.exist_refactored(partition_id);
 | 
						|
        if (OB_HASH_EXIST == hash_ret) {
 | 
						|
          restore = REPLICA_RESTORE_DATA;
 | 
						|
        } else if (OB_HASH_NOT_EXIST == hash_ret) {
 | 
						|
          restore = REPLICA_RESTORE_ARCHIVE_DATA;
 | 
						|
        } else {
 | 
						|
          ret = OB_SUCC(ret) ? OB_ERR_UNEXPECTED : hash_ret;
 | 
						|
          LOG_WARN("fail to check partition_id exist", K(ret), K(partition_id));
 | 
						|
        }
 | 
						|
        if (FAILEDx(fill_partition_member_list(part_addr, now, paxos_replica_num, arg))) {
 | 
						|
          LOG_WARN("fail to fill create partition arg memberlist", KR(ret), K(arg), K(paxos_replica_num));
 | 
						|
        } else if (REPLICA_RESTORE_DATA == restore && OB_FAIL(arg.table_schemas_.assign(base_table_schemas))) {
 | 
						|
          LOG_WARN("fail to assign table schemas", K(ret));
 | 
						|
        } else if (REPLICA_RESTORE_ARCHIVE_DATA == restore && OB_FAIL(arg.table_schemas_.assign(inc_table_schemas))) {
 | 
						|
          LOG_WARN("fail to assign table schemas", K(ret));
 | 
						|
        } else {
 | 
						|
          arg.member_list_.reset();
 | 
						|
          FOREACH_CNT_X(a, part_addr, OB_SUCC(ret))
 | 
						|
          {
 | 
						|
            ObPartitionReplica flag_replica;
 | 
						|
            if (OB_UNLIKELY(nullptr == a || nullptr == rpc_proxy_)) {
 | 
						|
              ret = OB_ERR_UNEXPECTED;
 | 
						|
              LOG_WARN("addr or rpc_proxy is null", K(ret));
 | 
						|
            } else if (OB_FAIL(fill_create_partition_arg(schema_id,
 | 
						|
                           partition_schema.get_partition_cnt(),
 | 
						|
                           paxos_replica_num,
 | 
						|
                           non_paxos_replica_num,
 | 
						|
                           partition_id,
 | 
						|
                           *a,
 | 
						|
                           now,
 | 
						|
                           false /*is_bootstrap*/,
 | 
						|
                           false /*is_standby*/,
 | 
						|
                           restore,
 | 
						|
                           frozen_status,
 | 
						|
                           arg))) {
 | 
						|
              LOG_WARN("fail to fill ObCreatePartitionArg", K(ret), K(schema_id));
 | 
						|
            } else if (OB_FAIL(fill_flag_replica(
 | 
						|
                           schema_id, partition_schema.get_partition_cnt(), partition_id, arg, *a, flag_replica))) {
 | 
						|
              LOG_WARN("fail to fill flag replica", K(ret), K(schema_id));
 | 
						|
            } else if (OB_FAIL(creator.add_flag_replica(flag_replica))) {
 | 
						|
              LOG_WARN("fail to add flag replica to partition creator", K(ret), K(flag_replica));
 | 
						|
            } else if (OB_FAIL(creator.add_create_partition_arg(a->addr_, arg))) {
 | 
						|
              LOG_WARN("fail to add create partition argument", K(ret));
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        if (OB_FAIL(creator.execute())) {
 | 
						|
          LOG_WARN("fail execute create partition", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::construct_table_schemas_for_physical_restore(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const obrpc::ObRestorePartitionsArg& restore_arg, const common::hash::ObHashSet<int64_t>& base_part_id_set,
 | 
						|
    const common::ObIArray<int64_t>& part_ids, common::ObIArray<share::schema::ObTableSchema>& base_table_schemas,
 | 
						|
    common::ObIArray<share::schema::ObTableSchema>& inc_table_schemas)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t schema_id = restore_arg.schema_id_;
 | 
						|
  const uint64_t tenant_id = extract_tenant_id(schema_id);
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (is_new_tablegroup_id(schema_id)) {
 | 
						|
    // skip
 | 
						|
  } else {
 | 
						|
    // The partition created by incremental and baseline needs to bring the main table schema
 | 
						|
    ObTableSchema tmp_schema;
 | 
						|
    const ObTableSchema* table_schema = NULL;
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schema(schema_id, table_schema))) {
 | 
						|
      LOG_WARN("fail to get table", K(ret), K(schema_id));
 | 
						|
    } else if (OB_ISNULL(table_schema)) {
 | 
						|
      ret = OB_TABLE_NOT_EXIST;
 | 
						|
      LOG_WARN("table not exist", K(ret), K(restore_arg));
 | 
						|
    } else if (OB_FAIL(tmp_schema.assign(*table_schema))) {
 | 
						|
      LOG_WARN("fail to assign schema", K(ret));
 | 
						|
    } else {
 | 
						|
      tmp_schema.reset_partition_schema();
 | 
						|
      if (OB_FAIL(base_table_schemas.push_back(tmp_schema))) {
 | 
						|
        LOG_WARN("fail to push back schemas", K(ret), K(tmp_schema));
 | 
						|
      } else if (OB_FAIL(inc_table_schemas.push_back(tmp_schema))) {
 | 
						|
        LOG_WARN("fail to push back schemas", K(ret), K(tmp_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    bool inc_part_exist = false;
 | 
						|
    for (int64_t i = 0; !inc_part_exist && OB_SUCC(ret) && i < part_ids.count(); i++) {
 | 
						|
      const int64_t partition_id = part_ids.at(i);
 | 
						|
      int hash_ret = base_part_id_set.exist_refactored(partition_id);
 | 
						|
      if (OB_HASH_NOT_EXIST == hash_ret) {
 | 
						|
        inc_part_exist = true;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // Only need to process partitions that have baselines and incrementally dynamically created partitions,
 | 
						|
    // reducing the consumption of guards for specified versions
 | 
						|
    if (OB_SUCC(ret) && inc_part_exist && base_part_id_set.size() > 0) {
 | 
						|
      ObSchemaGetterGuard base_guard;
 | 
						|
      const ObTableSchema* base_table_schema = NULL;
 | 
						|
      ObMultiVersionSchemaService::RefreshSchemaMode mode = ObMultiVersionSchemaService::FORCE_FALLBACK;
 | 
						|
      ObArray<ObAuxTableMetaInfo> index_infos;
 | 
						|
      if (OB_FAIL(schema_service_->get_tenant_schema_guard(
 | 
						|
              tenant_id, base_guard, restore_arg.schema_version_, OB_INVALID_VERSION, mode))) {
 | 
						|
        LOG_WARN("fail to get schema guard", K(ret), K(restore_arg));
 | 
						|
      } else if (OB_FAIL(base_guard.get_table_schema(schema_id, base_table_schema))) {
 | 
						|
        LOG_WARN("fail to get table schema", K(ret), K(schema_id));
 | 
						|
      } else if (OB_ISNULL(base_table_schema)) {
 | 
						|
        ret = OB_TABLE_NOT_EXIST;
 | 
						|
        LOG_WARN("table should exist", K(ret), K(restore_arg));
 | 
						|
      } else if (OB_FAIL(table_schema->get_simple_index_infos(index_infos))) {
 | 
						|
        LOG_WARN("fail to get simple index infos", K(ret), K(schema_id));
 | 
						|
      } else {
 | 
						|
        for (int64_t i = 0; OB_SUCC(ret) && i < index_infos.count(); i++) {
 | 
						|
          const uint64_t index_tid = index_infos.at(i).table_id_;
 | 
						|
          const ObTableSchema* index_table = NULL;
 | 
						|
          if (OB_FAIL(schema_guard.get_table_schema(index_tid, index_table))) {
 | 
						|
            LOG_WARN("fail to get index schema", K(ret), K(index_tid));
 | 
						|
          } else if (OB_ISNULL(index_table)) {
 | 
						|
            ret = OB_TABLE_NOT_EXIST;
 | 
						|
            LOG_WARN("table not exist", K(ret), K(index_tid));
 | 
						|
          } else if (!index_table->has_self_partition() && !index_table->is_dropped_schema() &&
 | 
						|
                     INDEX_STATUS_AVAILABLE == index_table->get_index_status()) {
 | 
						|
            // Non-delayed delete local index with final index status effective
 | 
						|
            const ObTableSchema* tmp_index = NULL;
 | 
						|
            if (OB_FAIL(base_guard.get_table_schema(index_tid, tmp_index))) {
 | 
						|
              LOG_WARN("fail to get table schema", K(ret), K(restore_arg), K(index_tid));
 | 
						|
            } else if (OB_ISNULL(tmp_index) || INDEX_STATUS_AVAILABLE != tmp_index->get_index_status()) {
 | 
						|
              // skip
 | 
						|
            } else if (OB_FAIL(tmp_schema.assign(*index_table))) {
 | 
						|
              LOG_WARN("fail to assign schema", K(ret), K(index_tid));
 | 
						|
            } else {
 | 
						|
              tmp_schema.reset_partition_schema();
 | 
						|
              if (OB_FAIL(inc_table_schemas.push_back(tmp_schema))) {
 | 
						|
                LOG_WARN("fail to push back schemas", K(ret), K(tmp_schema));
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_tablegroup_partitions(
 | 
						|
    const obrpc::ObCreateTableMode create_mode, const share::schema::ObTablegroupSchema& tablegroup_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  RS_TRACE(create_tablegroup_partitions_begin);
 | 
						|
  share::ObSimpleFrozenStatus frozen_status;
 | 
						|
  common::ObArray<ObPartitionAddr> tablegroup_addr;
 | 
						|
  ObPartIdsGenerator gen(tablegroup_schema);
 | 
						|
  ObArray<int64_t> part_ids;
 | 
						|
  int64_t tenant_id = tablegroup_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (create_mode <= obrpc::OB_CREATE_TABLE_MODE_INVALID || create_mode >= obrpc::OB_CREATE_TABLE_MODE_MAX) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(create_mode));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == freeze_info_manager_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("zone_mgr_ ptr is null", K(ret));
 | 
						|
  } else if (OB_FAIL(freeze_info_manager_->get_frozen_status_for_create_partition(tenant_id, frozen_status))) {
 | 
						|
    LOG_WARN("fail to get frozen status", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == root_balancer_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root balancer ptr is null", K(ret));
 | 
						|
  } else if (OB_FAIL(root_balancer_->alloc_tablegroup_partitions_for_create(
 | 
						|
                 tablegroup_schema, create_mode, tablegroup_addr))) {
 | 
						|
    LOG_WARN("fail to alloc tablegroup partitions for create", K(ret));
 | 
						|
  } else if (OB_FAIL(gen.gen(part_ids))) {
 | 
						|
    LOG_WARN("fail to generate part ids", K(ret));
 | 
						|
  } else if (OB_FAIL(create_tablegroup_partitions_for_create(
 | 
						|
                 tablegroup_schema, tablegroup_addr, part_ids, frozen_status, create_mode))) {
 | 
						|
    LOG_WARN("fail to create tablegroup partitions for create", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_tablegroup(const bool if_not_exist, share::schema::ObTablegroupSchema& tablegroup_schema,
 | 
						|
    const ObString* ddl_stmt_str, const obrpc::ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  RS_TRACE(create_tablegroup_begin);
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  bool need_process_failed = false;
 | 
						|
  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(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 {
 | 
						|
      if (OB_FAIL(normalize_tablegroup_option(tablegroup_schema))) {
 | 
						|
        LOG_WARN("fail to normalize tablegroup option", K(ret), K(tablegroup_schema));
 | 
						|
      } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
      } else if (tablegroup_schema.get_primary_zone().empty()) {
 | 
						|
        // bypass, since the primary zone is empty, drive from upper level,
 | 
						|
        // no need to statistic primary zone count
 | 
						|
      } else {
 | 
						|
        if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(
 | 
						|
                trans, schema_guard, true /*inc*/, 1, tablegroup_schema.get_tenant_id()))) {
 | 
						|
          LOG_WARN("fail to try inc tenant primary zone entity count", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      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));
 | 
						|
        } else if (!tablegroup_schema.get_binding()) {
 | 
						|
          // Non-binding tablegroup, no need to create pg partition
 | 
						|
        } else if (OB_CREATE_TABLE_MODE_RESTORE == create_mode) {
 | 
						|
          // Backup and restore, first build schema, then partition
 | 
						|
        } else if (OB_FAIL(create_tablegroup_partitions(create_mode, tablegroup_schema))) {
 | 
						|
          need_process_failed = true;
 | 
						|
          LOG_WARN("fail to create tablegroup partitions", K(ret));
 | 
						|
        } else {
 | 
						|
          need_process_failed = true;
 | 
						|
        }
 | 
						|
        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) && need_process_failed) {
 | 
						|
          // push schema version
 | 
						|
          int tmp_ret = OB_SUCCESS;
 | 
						|
          if (OB_SUCCESS != (tmp_ret = process_create_partition_failed(tenant_id))) {
 | 
						|
            LOG_ERROR("create tablegroup failed, may has garbage partition", K(ret), KR(tmp_ret), K(tenant_id));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  // publish schema
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    int64_t orig_schema_version = OB_INVALID_VERSION;
 | 
						|
    int64_t new_schema_version = OB_INVALID_VERSION;
 | 
						|
    RS_TRACE(publish_tablegroup_schema_begin);
 | 
						|
    if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(tenant_id, orig_schema_version))) {
 | 
						|
      LOG_WARN("fail to get tenant refreshed schema version", K(ret), K(tenant_id));
 | 
						|
    } else if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
      LOG_WARN("fail to public schema", K(ret), K(tenant_id));
 | 
						|
    } else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(tenant_id, new_schema_version))) {
 | 
						|
      LOG_WARN("fail to get tenant refresh schema version", K(ret), K(tenant_id));
 | 
						|
    } else if (new_schema_version != orig_schema_version && tablegroup_schema.get_binding() &&
 | 
						|
               OB_CREATE_TABLE_MODE_RESTORE != create_mode) {
 | 
						|
      RS_TRACE(publish_tablegroup_schema_end);
 | 
						|
      if (OB_FAIL(check_need_wait_leader_by_tablegroup_schema(tablegroup_schema))) {
 | 
						|
        LOG_WARN("fail to check need wait leader by schema", KR(ret), K(tablegroup_schema));
 | 
						|
      }
 | 
						|
      RS_TRACE(wait_tablegroup_leader_end);
 | 
						|
    }
 | 
						|
  }
 | 
						|
  FORCE_PRINT_TRACE(THE_RS_TRACE, "[create tablegroup]");
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::normalize_tablegroup_option(share::schema::ObTablegroupSchema& tablegroup_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  uint64_t tenant_id = tablegroup_schema.get_tenant_id();
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObTenantSchema* tenant_info = NULL;
 | 
						|
  ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
  ObArray<ObZoneReplicaAttrSet> zone_locality;
 | 
						|
  const bool just_check_zone_list = 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_tenant_info(tenant_id, tenant_info))) {
 | 
						|
    LOG_WARN("fail to get tenant info", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_ISNULL(tenant_info)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("get invalid tenant_info", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(try_set_and_check_locality_with_tenant(
 | 
						|
                 just_check_zone_list, schema_guard, *tenant_info, tablegroup_schema))) {
 | 
						|
    LOG_WARN("fail to set and check locality with tenant", K(ret));
 | 
						|
  } else if (OB_FAIL(check_create_tablegroup_replica_options(tablegroup_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail to check and set primary_zone", K(ret));
 | 
						|
  } else if (!tablegroup_schema.get_locality_str().empty()) {
 | 
						|
    common::ObArray<share::ObResourcePoolName> 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 (OB_FAIL(unit_mgr_->get_pool_names_of_tenant(tablegroup_schema.get_tenant_id(), pool_names))) {
 | 
						|
      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, tablegroup_schema))) {
 | 
						|
      LOG_WARN("pool unit num is not enough for locality", K(ret));
 | 
						|
    } else {
 | 
						|
    }  // no more to do
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// alter tablegroup set primary_zone/locality
 | 
						|
// 1. Check the validity of tablegroup_option first, the ones that are exactly the same as tenant will be reset
 | 
						|
// 2. Modify the attributes of the tablegroup (because the primary_zone and locality of the table added to the new
 | 
						|
// tablegroup
 | 
						|
//  are reset to the inheritance relationship, there is no need to change the attributes of the table)
 | 
						|
// 3. For locality changes, similar to tenant locality changes, all tables under the tablegroup will participate
 | 
						|
//  in locality changes
 | 
						|
int ObDDLService::set_new_tablegroup_options(ObMySQLTransaction& trans, const ObAlterTablegroupArg& arg,
 | 
						|
    ObSchemaGetterGuard& schema_guard, const ObTablegroupSchema& orig_tablegroup, ObTablegroupSchema& new_tablegroup)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  new_tablegroup = orig_tablegroup;
 | 
						|
  LOG_INFO("start to modify tablegroup", K(ret), K(arg), K(orig_tablegroup), K(new_tablegroup));
 | 
						|
  bool modify_primary_zone = true;
 | 
						|
  bool modify_locality = true;
 | 
						|
  const uint64_t tablegroup_id = orig_tablegroup.get_tablegroup_id();
 | 
						|
  AlterLocalityOp alter_locality_op = ALTER_LOCALITY_OP_INVALID;
 | 
						|
  const ObString& locality = arg.alter_tablegroup_schema_.get_locality();
 | 
						|
  if (!is_new_tablegroup_id(tablegroup_id)) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("cannot modify tablegroup created before 2.0", K(ret), K(tablegroup_id));
 | 
						|
  } else if (!arg.alter_option_bitset_.has_member(ObAlterTablegroupArg::LOCALITY)) {
 | 
						|
    modify_locality = false;
 | 
						|
  } else if (OB_FAIL(new_tablegroup.set_locality(locality))) {
 | 
						|
    LOG_WARN("fail to set locality", K(ret), K(arg));
 | 
						|
  } else if (OB_FAIL(check_tablegroup_alter_locality(
 | 
						|
                 arg, schema_guard, orig_tablegroup, new_tablegroup, alter_locality_op))) {
 | 
						|
    LOG_WARN("fail to check tablegroup alter locality", K(ret));
 | 
						|
  }
 | 
						|
  const ObString& primary_zone = arg.alter_tablegroup_schema_.get_primary_zone();
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (!arg.alter_option_bitset_.has_member(ObAlterTablegroupArg::PRIMARY_ZONE)) {
 | 
						|
    modify_primary_zone = false;
 | 
						|
  } else if (OB_FAIL(new_tablegroup.set_primary_zone(primary_zone))) {
 | 
						|
    LOG_WARN("fail to set primary zone", K(ret), K(arg));
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_FAIL(check_alter_tablegroup_replica_options(arg, schema_guard, orig_tablegroup, new_tablegroup))) {
 | 
						|
    LOG_WARN("fail to check alter tablegroup replica options", K(ret), K(tablegroup_id));
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret) && (modify_primary_zone || modify_locality)) {
 | 
						|
    // 2. change primary_zone/locality of tablegroup
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    if (OB_FAIL(ddl_operator.alter_tablegroup(new_tablegroup, trans, &(arg.ddl_stmt_str_)))) {
 | 
						|
      LOG_WARN("fail to alter tablegroup", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret) && modify_locality && alter_locality_op == ROLLBACK_ALTER_LOCALITY) {
 | 
						|
    uint64_t tenant_id = new_tablegroup.get_tenant_id();
 | 
						|
    task_mgr_->clear_task(tenant_id, obrpc::ObAdminClearBalanceTaskArg::ALL);
 | 
						|
  }
 | 
						|
  // always notify locality modification whether the above process succeed
 | 
						|
  // and this shall be invoked alter trans.end(), don't change the sequence.
 | 
						|
  if (!new_tablegroup.get_previous_locality_str().empty()) {
 | 
						|
    root_balancer_->notify_locality_modification();
 | 
						|
    root_balancer_->wakeup();
 | 
						|
  }
 | 
						|
  if (OB_SUCC(ret) && arg.alter_option_bitset_.has_member(obrpc::ObAlterTablegroupArg::LOCALITY)) {
 | 
						|
    int tmp_ret = OB_SUCCESS;
 | 
						|
    if (OB_SUCCESS !=
 | 
						|
        (tmp_ret = record_tablegroup_locality_event_history(alter_locality_op, arg, new_tablegroup, trans))) {
 | 
						|
      LOG_WARN("fail to record tablegroup locality event history", K(tmp_ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  LOG_DEBUG("set new tablegroup options", K(orig_tablegroup), K(new_tablegroup));
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_tablegroup_alter_locality(const obrpc::ObAlterTablegroupArg& arg,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const share::schema::ObTablegroupSchema& orig_tablegroup,
 | 
						|
    share::schema::ObTablegroupSchema& new_tablegroup, AlterLocalityOp& alter_locality_op)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const uint64_t tablegroup_id = new_tablegroup.get_tablegroup_id();
 | 
						|
  const uint64_t tenant_id = new_tablegroup.get_tenant_id();
 | 
						|
  LOG_INFO("start to modify tablegroup locality", K(ret), K(arg), K(orig_tablegroup), K(new_tablegroup));
 | 
						|
  if (!is_new_tablegroup_id(tablegroup_id)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("tablegroup_id is invalid", K(ret), K(tablegroup_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("get invalid tenant schema", K(ret), K(tenant_id));
 | 
						|
  } else {
 | 
						|
    common::ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
    common::ObArray<ObZone> zone_list;
 | 
						|
    AlterLocalityType alter_locality_type = ALTER_LOCALITY_INVALID;
 | 
						|
    bool tenant_pools_in_shrinking = false;
 | 
						|
    bool extend_empty_locality = false;
 | 
						|
    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_tablegroup.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 tablegroup locality when tenant pool is shrinking");
 | 
						|
      LOG_WARN("alter tablegroup locality not allowed", K(ret), K(orig_tablegroup));
 | 
						|
    } else if (OB_FAIL(tenant_schema->get_zone_list(zone_list))) {
 | 
						|
      LOG_WARN("fail to get zone list", K(ret));
 | 
						|
    } else 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(parse_and_set_new_locality_options(
 | 
						|
                   schema_guard, tenant_id, new_tablegroup, zone_list, zone_region_list, extend_empty_locality))) {
 | 
						|
      LOG_WARN("fail ot parse and set new table locality options", K(ret));
 | 
						|
    } else if (OB_FAIL(check_alter_tablegroup_locality_type(
 | 
						|
                   schema_guard, orig_tablegroup, new_tablegroup, alter_locality_type))) {
 | 
						|
      LOG_WARN("fail to check alter tenant locality type", K(ret));
 | 
						|
    } else if (ALTER_LOCALITY_INVALID == alter_locality_type) {
 | 
						|
      ret = OB_OP_NOT_ALLOW;
 | 
						|
      LOG_USER_ERROR(OB_OP_NOT_ALLOW, "alter tablegroup locality when previous operations is in progress");
 | 
						|
      LOG_WARN("alter tablegroup locality not allowed", K(ret), K(orig_tablegroup));
 | 
						|
    } else if (ROLLBACK_LOCALITY == alter_locality_type) {
 | 
						|
      if (OB_FAIL(try_rollback_alter_tablegroup_locality(arg, new_tablegroup, orig_tablegroup, alter_locality_op))) {
 | 
						|
        LOG_WARN("fail to try rollback alter table locality", K(ret), K(new_tablegroup), K(orig_tablegroup));
 | 
						|
      } else {
 | 
						|
      }  // no more to do
 | 
						|
    } else if (TO_NEW_LOCALITY == alter_locality_type) {
 | 
						|
      if (OB_FAIL(try_alter_tablegroup_locality(
 | 
						|
              schema_guard, *tenant_schema, orig_tablegroup, new_tablegroup, zone_region_list, alter_locality_op))) {
 | 
						|
        LOG_WARN("fail to try alter table locality", K(ret));
 | 
						|
      } else {
 | 
						|
      }  // no more to do
 | 
						|
    } else if (LOCALITY_NOT_CHANGED == alter_locality_type) {
 | 
						|
      alter_locality_op = NOP_LOCALITY_OP;
 | 
						|
      LOG_INFO("set locality to default", K(ret), K(tablegroup_id));
 | 
						|
    } else {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("invalid alter locality type", K(ret), K(alter_locality_type));
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      common::ObArray<share::ObResourcePoolName> pool_names;
 | 
						|
      if (new_tablegroup.get_locality_str().empty()) {
 | 
						|
        // derived from tenant locality, no need to check
 | 
						|
      } 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_pool_names_of_tenant(new_tablegroup.get_tenant_id(), pool_names))) {
 | 
						|
        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_tablegroup))) {
 | 
						|
        LOG_WARN("pool unit num is not enough for locality", K(ret));
 | 
						|
      } else {
 | 
						|
      }  // no more to do
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
    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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } else if (OB_FAIL(schema_guard.get_tablegroup_schema(tablegroup_id, tablegroup_schema))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schema", K(ret), 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 (tablegroup_schema->get_primary_zone().empty()) {
 | 
						|
        // bypass, since the primary zone is empty, derive from upper level,
 | 
						|
        // no need to statistic primary zone count
 | 
						|
      } else {
 | 
						|
        if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(
 | 
						|
                trans, schema_guard, false /*dec*/, 1, tablegroup_schema->get_tenant_id()))) {
 | 
						|
          LOG_WARN("fail to try dec tenant primary zone entity count", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      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, including primary_zone;locality;partition
 | 
						|
// 2. Modify the attributes of the tablegroup: including primary_zone; locality; 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();
 | 
						|
  bool need_process_failed = true;
 | 
						|
  ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_);
 | 
						|
  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(tablegroup_id, tablegroup_schema))) {
 | 
						|
    LOG_WARN("fail to get tablegroup schema", K(ret), K(tablegroup_id));
 | 
						|
  } else if (OB_FAIL(helper.update_max_part_id_if_needed(*this, schema_guard, *tablegroup_schema, arg))) {
 | 
						|
    LOG_WARN("fail to update max part id", K(ret));
 | 
						|
  } 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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } else if (OB_FAIL(new_schema_guard.get_tablegroup_schema(tablegroup_id, orig_tablegroup))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schema", K(ret), 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(set_new_tablegroup_options(trans, arg, new_schema_guard, *orig_tablegroup, new_tablegroup))) {
 | 
						|
      LOG_WARN("fail to modify tablegroup options", K(ret));
 | 
						|
    } else if (OB_FAIL(helper.modify_partition_option(trans, new_schema_guard, new_tablegroup, arg))) {
 | 
						|
      need_process_failed = true;
 | 
						|
      LOG_WARN("fail to modify partition option", K(ret), K(new_tablegroup), K(arg));
 | 
						|
    } else {
 | 
						|
      need_process_failed = true;
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (orig_tablegroup->get_primary_zone().empty() && new_tablegroup.get_primary_zone().empty()) {
 | 
						|
        // bypass
 | 
						|
      } else if (!orig_tablegroup->get_primary_zone().empty() && !new_tablegroup.get_primary_zone().empty()) {
 | 
						|
        // bypass
 | 
						|
      } else if (!orig_tablegroup->get_primary_zone().empty() && new_tablegroup.get_primary_zone().empty()) {
 | 
						|
        if (OB_FAIL(
 | 
						|
                try_modify_tenant_primary_zone_entity_count(trans, new_schema_guard, false /*sub*/, 1, tenant_id))) {
 | 
						|
          LOG_WARN("fail to try modify primary zone entity count", K(ret));
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        if (OB_FAIL(try_modify_tenant_primary_zone_entity_count(trans, new_schema_guard, true /*add*/, 1, tenant_id))) {
 | 
						|
          LOG_WARN("fail to try modify primary zone entity count", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // 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) && need_process_failed) {
 | 
						|
      // push schema version
 | 
						|
      int tmp_ret = OB_SUCCESS;
 | 
						|
      if (OB_SUCCESS != (tmp_ret = process_create_partition_failed(tenant_id))) {
 | 
						|
        LOG_ERROR("alter tablegroup failed, may has garbage partition", K(ret), K(tmp_ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  int tmp_ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
 | 
						|
    LOG_WARN("fail to publish schema", K(ret));
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      ret = tmp_ret;
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    int64_t start_usec = ObTimeUtility::current_time();
 | 
						|
    int64_t end_usec = 0;
 | 
						|
    int64_t cost_usec = 0;
 | 
						|
    if (arg.alter_option_bitset_.has_member(ObAlterTablegroupArg::ADD_PARTITION) ||
 | 
						|
        arg.alter_option_bitset_.has_member(ObAlterTablegroupArg::SPLIT_PARTITION) ||
 | 
						|
        arg.alter_option_bitset_.has_member(ObAlterTablegroupArg::PARTITIONED_TABLE) ||
 | 
						|
        arg.alter_option_bitset_.has_member(ObAlterTablegroupArg::REORGANIZE_PARTITION)) {
 | 
						|
      ObSchemaGetterGuard new_schema_guard;
 | 
						|
      const ObTablegroupSchema* new_tablegroup_schema = NULL;
 | 
						|
      if (OB_SUCCESS != (tmp_ret = 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_SUCCESS !=
 | 
						|
                 (tmp_ret = new_schema_guard.get_tablegroup_schema(tablegroup_id, new_tablegroup_schema))) {
 | 
						|
        LOG_WARN("fail to get table schema", K(tmp_ret), K(tablegroup_id));
 | 
						|
      } else if (OB_ISNULL(new_tablegroup_schema)) {
 | 
						|
        tmp_ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("NULL ptr", K(tablegroup_id), K(tmp_ret));
 | 
						|
      } else if (new_tablegroup_schema->has_self_partition()) {
 | 
						|
        // it is distinguished according to whether the tablegroup is a binding
 | 
						|
        if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_tablegroup_schema(*new_tablegroup_schema))) {
 | 
						|
          LOG_WARN(
 | 
						|
              "fail to check need wait leader by schema", KR(tmp_ret), "new_tablegroup_schema", *new_tablegroup_schema);
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        ObArray<const ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
        if (OB_SUCCESS !=
 | 
						|
            (tmp_ret = new_schema_guard.get_table_schemas_in_tablegroup(tenant_id, tablegroup_id, table_schemas))) {
 | 
						|
          LOG_WARN("fail to get table schemas in tablegroup", K(ret), K(tmp_ret), K(tenant_id), K(tablegroup_id));
 | 
						|
        } else {
 | 
						|
          for (int64_t i = 0; OB_SUCCESS != tmp_ret && i < table_schemas.count(); ++i) {
 | 
						|
            const ObSimpleTableSchemaV2* new_table_schema = table_schemas.at(i);
 | 
						|
            if (OB_ISNULL(new_table_schema)) {
 | 
						|
              tmp_ret = OB_ERR_UNEXPECTED;
 | 
						|
              LOG_WARN("table schema is null", K(ret), K(tmp_ret), K(i));
 | 
						|
            } else if (!new_table_schema->has_partition()) {
 | 
						|
              // do nothing, index etc.
 | 
						|
            } else if (!new_table_schema->has_self_partition()) {
 | 
						|
              tmp_ret = OB_ERR_UNEXPECTED;
 | 
						|
              LOG_WARN("table schema should has partition", K(ret), K(tmp_ret), "new_table_schema", *new_table_schema);
 | 
						|
              // The above inspection requirements are more stringent, and there is redundancy with check_..._by_schema,
 | 
						|
              // but should be retained
 | 
						|
            } else if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_tablegroup_schema(*new_tablegroup_schema))) {
 | 
						|
              LOG_WARN("fail to check need wait leader by schema",
 | 
						|
                  KR(tmp_ret),
 | 
						|
                  "new_tablegroup_schema",
 | 
						|
                  *new_tablegroup_schema);
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
      end_usec = ObTimeUtility::current_time();
 | 
						|
      cost_usec = end_usec - start_usec;
 | 
						|
      start_usec = end_usec;
 | 
						|
      LOG_INFO("waiter.wait cost: ", K(cost_usec));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  LOG_INFO("finish alter tablegroup", K(tenant_id), KT(tablegroup_id), K(ret));
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_split_partition_can_execute() const
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  bool is_doing_backup = false;
 | 
						|
  rootserver::ObRootService* root_service = GCTX.root_service_;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("inner stat error", K(ret));
 | 
						|
  } else if (OB_ISNULL(root_service)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("root_service is null", K(ret));
 | 
						|
  } else if (CLUSTER_VERSION_2250 <= GET_MIN_CLUSTER_VERSION() &&
 | 
						|
             OB_FAIL(share::ObBackupInfoMgr::get_instance().check_if_doing_backup(is_doing_backup))) {
 | 
						|
    LOG_WARN("failed to check_if_doing_backup", K(ret));
 | 
						|
  } else if (is_doing_backup) {
 | 
						|
    ret = OB_BACKUP_IN_PROGRESS;
 | 
						|
    LOG_WARN("cannot split partition during backup", 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<uint64_t> 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<int64_t>(REFRESH_SCHEMA_INTERVAL_US),
 | 
						|
              K(ret));
 | 
						|
        } else {
 | 
						|
          LOG_WARN("refresh schema failed",
 | 
						|
              K(refresh_count),
 | 
						|
              "refresh_schema_interval",
 | 
						|
              static_cast<int64_t>(REFRESH_SCHEMA_INTERVAL_US),
 | 
						|
              K(ret));
 | 
						|
        }
 | 
						|
        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;
 | 
						|
  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", K(ret));
 | 
						|
  } else if (OB_ISNULL(schema_service_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema_service is null", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_service_->get_refresh_schema_info(local_schema_info))) {
 | 
						|
    LOG_WARN("fail to get schema info", K(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", K(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", K(ret), K(local_schema_info));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      schema_info.set_schema_version(schema_version);
 | 
						|
      schema_info.set_split_schema_version(GCTX.split_schema_version_);
 | 
						|
    }
 | 
						|
 | 
						|
    LOG_INFO("try to notify refresh schema", K(schema_version), K(local_schema_info), K(schema_info));
 | 
						|
    const int64_t force_refresh_schema_timeout = 10 * 1000 * 1000L;  // 10s
 | 
						|
    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;
 | 
						|
          }
 | 
						|
        }
 | 
						|
        timeout = !found ? rpc_timeout : std::max(rpc_timeout, force_refresh_schema_timeout);
 | 
						|
        timeout = std::min(THIS_WORKER.get_timeout_remain(), timeout);
 | 
						|
        arg.force_refresh_ = found;
 | 
						|
        // overwrite ret
 | 
						|
        if (OB_FAIL(rpc_proxy_->to(*s).timeout(timeout).switch_schema(arg))) {
 | 
						|
          LOG_WARN("notify switch schema failed",
 | 
						|
              K(ret),
 | 
						|
              K(timeout),
 | 
						|
              K(schema_version),
 | 
						|
              K(schema_info),
 | 
						|
              K(arg),
 | 
						|
              "server",
 | 
						|
              *s);
 | 
						|
          // RPCs that only require synchronous schema refresh must be executed successfully
 | 
						|
          if (!found) {
 | 
						|
            ret = OB_SUCCESS;
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          LOG_INFO("notify switch schema success", K(ret), K(schema_version), K(schema_info), K(arg), "server", *s);
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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_tenant_schema(const ObIArray<ObString>& 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");
 | 
						|
  } 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", K(OB_MAX_TENANT_NAME_LENGTH), K(ret));
 | 
						|
  } else if (OB_FAIL(check_create_tenant_locality(pool_list, tenant_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail to check create tenant locality", K(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", K(tenant_schema), K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_create_tenant_locality(const ObIArray<ObString>& pool_list, ObTenantSchema& tenant_schema,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObResourcePoolName> pools;
 | 
						|
  ObArray<ObZone> pool_zones;
 | 
						|
  ObArray<ObZone> temp_zones;
 | 
						|
  ObArray<share::schema::ObZoneRegion> 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 <typename SCHEMA>
 | 
						|
int ObDDLService::check_pools_unit_num_enough_for_schema_locality(const common::ObIArray<ObResourcePoolName>& pools,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const SCHEMA& schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  common::ObArray<share::ObZoneReplicaNumSet> 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::create_tenant_env(share::schema::ObSchemaGetterGuard& schema_guard, const ObCreateTenantArg& arg,
 | 
						|
    const common::ObRegion& tenant_primary_region, ObTenantSchema& tenant_schema, const int64_t frozen_version,
 | 
						|
    const ObString* ddl_stmt_str /*NULL*/)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  ObRootService* rootservice = GCTX.root_service_;
 | 
						|
  bool is_standby = false;
 | 
						|
  const bool is_bind = 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 cluster", K(ret));
 | 
						|
  } else if (OB_ISNULL(rootservice) || OB_ISNULL(schema_service_) || OB_ISNULL(rpc_proxy_) || OB_ISNULL(pt_operator_) ||
 | 
						|
             OB_ISNULL(server_mgr_) || OB_ISNULL(sql_proxy_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("pointer is empty",
 | 
						|
        KR(ret),
 | 
						|
        KP(rootservice),
 | 
						|
        KP_(schema_service),
 | 
						|
        KP_(rpc_proxy),
 | 
						|
        KP_(pt_operator),
 | 
						|
        KP_(server_mgr),
 | 
						|
        KP_(sql_proxy));
 | 
						|
  } else if (GCONF._enable_ha_gts_full_service &&
 | 
						|
             OB_FAIL(rootservice->get_rs_gts_manager().erase_tenant_gts(tenant_schema.get_tenant_id()))) {
 | 
						|
    LOG_WARN("fail to try erase tenant gts for tenant", K(ret), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
  } else if (OB_FAIL(
 | 
						|
                 GCONF._enable_ha_gts_full_service && rootservice->get_rs_gts_manager().alloc_gts_instance_for_tenant(
 | 
						|
                                                          tenant_schema.get_tenant_id(), tenant_primary_region))) {
 | 
						|
    LOG_WARN("fail to alloc gts instance for tenant", K(ret), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
  } else {
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    LOG_INFO("start create tenant env", K(tenant_schema));
 | 
						|
    ObPartitionCreator creator(*rpc_proxy_, *pt_operator_, server_mgr_, is_bind, is_standby, sql_proxy_);
 | 
						|
    const uint64_t tenant_id = tenant_schema.get_tenant_id();
 | 
						|
    ObArray<ObResourcePoolName> pools;
 | 
						|
    ObSysVariableSchema sys_variable;
 | 
						|
    share::ObWorker::CompatMode compat_mode = share::ObWorker::CompatMode::INVALID;
 | 
						|
    bool is_follower_cluster = false;
 | 
						|
    CreateTenantStatus create_tenant_status = CREATE_TENANT_FAILED;
 | 
						|
    ObAddrArray leader_addrs;
 | 
						|
    bool partitions_already_created = false;
 | 
						|
    bool need_process_failed = false;
 | 
						|
    int64_t paxos_replica_num = OB_INVALID_COUNT;
 | 
						|
    // For standalone cluster and physical recovery, only transaction one needs to be executed
 | 
						|
    // Transaction 1: Write system tenant __all_tenant&create partition
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      LOG_INFO("[CREATE_TENANT] start create tenant trans one", K(ret), K(tenant_id));
 | 
						|
      // When creating tenant transaction one, you need to write it on the system tenant
 | 
						|
      // and use the schema version generated by the system tenant
 | 
						|
      trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
      const ObSchemaOperationType operation_type = OB_DDL_MAX_OP;
 | 
						|
      if (arg.is_restore_) {
 | 
						|
        tenant_schema.set_status(TENANT_STATUS_RESTORE);
 | 
						|
      }
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
      } else if (OB_FAIL(init_tenant_storage_format_version(tenant_schema))) {
 | 
						|
        LOG_WARN("init_tenant_storage_format_version failed", K(ret));
 | 
						|
      } else if (OB_FAIL(init_tenant_sys_params(arg, tenant_schema, sys_variable))) {
 | 
						|
        LOG_WARN("fail to init tenant sys params", K(ret), K(tenant_schema), K(arg));
 | 
						|
      } else if (FALSE_IT(
 | 
						|
                     compat_mode = (common::ObCompatibilityMode::ORACLE_MODE == tenant_schema.get_compatibility_mode()
 | 
						|
                                        ? share::ObWorker::CompatMode::ORACLE
 | 
						|
                                        : share::ObWorker::CompatMode::MYSQL))) {
 | 
						|
        // will never be here
 | 
						|
      } else if (OB_FAIL(get_pools(arg.pool_list_, pools))) {
 | 
						|
        LOG_WARN("get_pools failed", K(arg), K(ret));
 | 
						|
      } else if (OB_FAIL(unit_mgr_->distrubte_for_unit_intersect(tenant_schema.get_tenant_id(), pools))) {
 | 
						|
        LOG_WARN("fail to distribute for unit intersect", K(ret));
 | 
						|
      }
 | 
						|
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(tenant_schema.get_paxos_replica_num(schema_guard, paxos_replica_num))) {
 | 
						|
        LOG_WARN("fail to get paxos replica num", KR(ret));
 | 
						|
      } else if (OB_UNLIKELY(paxos_replica_num <= 0)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("paxos replica num error", KR(ret), K(paxos_replica_num));
 | 
						|
      } else if (is_standby) {
 | 
						|
        if (OB_FAIL(creator.set_tenant_exist_partition_member_list(tenant_id, paxos_replica_num))) {
 | 
						|
          LOG_WARN("failed to set exist partition member list",
 | 
						|
              KR(ret),
 | 
						|
              K(paxos_replica_num),
 | 
						|
              K(tenant_id),
 | 
						|
              K(tenant_schema));
 | 
						|
        } else {
 | 
						|
          partitions_already_created = creator.is_tenant_partition_already_exist();
 | 
						|
        }
 | 
						|
      }
 | 
						|
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (partitions_already_created) {
 | 
						|
        // The standalone cluster repeatedly creates tenants. If there is already a persistent member list,
 | 
						|
        // it means that the majority partition has been created successfully.
 | 
						|
        // No need to send rpc to obs to modify the unit persistence, just modify the internal table.
 | 
						|
        if (OB_FAIL(unit_mgr_->grant_pools_for_standby(trans, pools, tenant_id))) {
 | 
						|
          LOG_WARN("failed to grant pool to standby", K(ret), K(tenant_id), K(pools));
 | 
						|
        }
 | 
						|
      } else {
 | 
						|
        // The standalone cluster creates tenants, allowing insufficient servers in certain zones
 | 
						|
        // Allow some units to be persisted successfully when granting;
 | 
						|
        if (OB_FAIL(unit_mgr_->grant_pools(trans,
 | 
						|
                compat_mode,
 | 
						|
                pools,
 | 
						|
                tenant_id,
 | 
						|
                false /*is_bootstrap*/,
 | 
						|
                is_standby, /*if not grant*/
 | 
						|
                is_standby /*skip_offline_server*/))) {
 | 
						|
          LOG_WARN("grant_pools_to_tenant failed", K(pools), K(tenant_id), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(ddl_operator.create_tenant(tenant_schema, OB_DDL_ADD_TENANT_START, trans, ddl_stmt_str))) {
 | 
						|
        LOG_WARN("create tenant failed", K(tenant_schema), KR(ret));
 | 
						|
      } else if (OB_FAIL(check_tenant_primary_zone_gts_condition(
 | 
						|
                     schema_guard, tenant_schema, sys_variable, sys_variable))) {
 | 
						|
        // both old and new use sys variable schema
 | 
						|
        LOG_WARN("fail to check tenant primary zone gts condition", KR(ret), K(tenant_schema));
 | 
						|
      } else if (OB_FAIL(create_tenant_partitions(
 | 
						|
                     arg, tenant_schema, leader_addrs, tenant_schema.get_schema_version(), frozen_version, creator))) {
 | 
						|
        need_process_failed = true;
 | 
						|
        LOG_WARN("create tenant system table partitions failed", K(ret), K(tenant_schema), K(frozen_version));
 | 
						|
      } else {
 | 
						|
        need_process_failed = true;
 | 
						|
      }
 | 
						|
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        ret = E(EventTable::EN_CREATE_TENANT_TRANS_ONE_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 (OB_FAIL(ret) && need_process_failed) {
 | 
						|
        // push schema version
 | 
						|
        int tmp_ret = OB_SUCCESS;
 | 
						|
        if (OB_SUCCESS != (tmp_ret = process_create_partition_failed(OB_SYS_TENANT_ID))) {
 | 
						|
          LOG_ERROR("create tenant failed, may has garbage parition", K(ret), K(tmp_ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
 | 
						|
      if (OB_SUCC(ret) && is_standby) {
 | 
						|
        // After the transaction is committed, the persistent member_list information can be cleared
 | 
						|
        // to prevent the failure of the transaction commit after the master election is successful
 | 
						|
        // clear all info of member list.
 | 
						|
        //
 | 
						|
        int tmp_ret = OB_SUCCESS;
 | 
						|
        // do not care whether clear success
 | 
						|
        const int64_t max_schema_version = ddl_operator.get_last_operation_schema_version();
 | 
						|
        const bool is_inner_table = true;
 | 
						|
        if (OB_SUCCESS != (tmp_ret = clear_partition_member_list(max_schema_version, tenant_id, is_inner_table))) {
 | 
						|
          LOG_WARN("failed to clear persist member list",
 | 
						|
              KR(ret),
 | 
						|
              KR(tmp_ret),
 | 
						|
              K(tenant_id),
 | 
						|
              K(max_schema_version),
 | 
						|
              K(is_inner_table));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      // 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)) {
 | 
						|
        const bool grant = true;
 | 
						|
        if (OB_FAIL(unit_mgr_->commit_change_pool_owner(grant, pools, tenant_id))) {
 | 
						|
          LOG_WARN("commit change pool owner failed", K(grant), K(pools), K(tenant_id), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        create_tenant_status = CREATE_TENANT_TRANS_ONE_DONE;
 | 
						|
        if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID, leader_addrs))) {
 | 
						|
          LOG_WARN("publish schema failed", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      LOG_INFO("[CREATE_TENANT] end create tenant trans one", K(ret), K(tenant_id));
 | 
						|
    }
 | 
						|
 | 
						|
    DEBUG_SYNC(BEFORE_CREATE_TENANT_TRANS_TWO);
 | 
						|
 | 
						|
    if (OB_SUCC(ret) && !is_follower_cluster && !arg.is_restore_) {
 | 
						|
      LOG_INFO("[CREATE_TENANT] start wait partition election", K(ret), K(tenant_id));
 | 
						|
      // it only need to ensure that the schema that the transaction 2 depends on has a master
 | 
						|
      const uint64_t table_ids[] = {OB_ALL_DDL_OPERATION_TID,
 | 
						|
          OB_ALL_SYS_STAT_TID,
 | 
						|
          OB_ALL_SYS_VARIABLE_TID,
 | 
						|
          OB_ALL_SYS_VARIABLE_HISTORY_TID,
 | 
						|
          OB_ALL_TABLEGROUP_TID,
 | 
						|
          OB_ALL_TABLEGROUP_HISTORY_TID,
 | 
						|
          OB_ALL_DATABASE_TID,
 | 
						|
          OB_ALL_DATABASE_HISTORY_TID,
 | 
						|
          OB_ALL_DATABASE_PRIVILEGE_TID,
 | 
						|
          OB_ALL_DATABASE_PRIVILEGE_HISTORY_TID,
 | 
						|
          OB_ALL_USER_TID,
 | 
						|
          OB_ALL_USER_HISTORY_TID,
 | 
						|
          OB_ALL_WEAK_READ_SERVICE_TID,
 | 
						|
          OB_ALL_COLUMN_STATISTIC_TID,
 | 
						|
          OB_ALL_TABLE_STAT_TID,
 | 
						|
          OB_ALL_COLUMN_STAT_TID,
 | 
						|
          OB_ALL_HISTOGRAM_STAT_TID};
 | 
						|
      int tmp_ret = OB_SUCCESS;
 | 
						|
      for (int64_t i = 0; i < ARRAYSIZEOF(table_ids); ++i) {
 | 
						|
        // Different tables are mutually exclusive and do not affect each other
 | 
						|
        const uint64_t table_id = combine_id(tenant_id, table_ids[i]);
 | 
						|
        const int64_t partition_id = 0;  // Tenant-level system tables are non-partitioned tables
 | 
						|
        if (OB_SUCCESS != (tmp_ret = check_need_wait_leader_by_id(table_id, partition_id))) {
 | 
						|
          LOG_WARN("fail to check need wait leader by id", KR(tmp_ret), K(table_id), K(partition_id));
 | 
						|
        }
 | 
						|
        if (OB_SUCC(ret)) {
 | 
						|
          ret = tmp_ret;
 | 
						|
        }
 | 
						|
      }
 | 
						|
      LOG_INFO("[CREATE_TENANT] end wait partition election", K(ret), K(tenant_id));
 | 
						|
    }
 | 
						|
 | 
						|
    // Transaction 2: Write the user tenant schema table, the standalone cluster does not need to perform transaction
 | 
						|
    // two
 | 
						|
    if (OB_SUCC(ret) && !is_follower_cluster && !arg.is_restore_) {
 | 
						|
      LOG_INFO("[CREATE_TENANT] start create tenant trans two", K(ret), K(tenant_id));
 | 
						|
      // Transaction 2 needs to be executed on user tenants
 | 
						|
      trans.set_end_tenant_id(tenant_schema.get_tenant_id());
 | 
						|
      // The upgrade process prohibits the creation of tenants, so there is no compatibility here,
 | 
						|
      // and directly write system variables in the second transaction
 | 
						|
      const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR;
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
      } else {
 | 
						|
        int64_t new_schema_version = OB_INVALID_VERSION;
 | 
						|
        int64_t refreshed_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
 | 
						|
        if (OB_ISNULL(schema_service_impl)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("schema_service_impl is null", K(ret));
 | 
						|
        } else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(
 | 
						|
                       OB_SYS_TENANT_ID, refreshed_schema_version))) {
 | 
						|
          LOG_WARN("fail to get tenant schema version", K(ret));
 | 
						|
        } else if (OB_FAIL(schema_service_impl->gen_new_schema_version(
 | 
						|
                       tenant_id, refreshed_schema_version, new_schema_version))) {
 | 
						|
          LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
 | 
						|
        } else if (OB_FAIL(ddl_operator.finish_schema_split_v2(trans, tenant_id))) {
 | 
						|
          LOG_WARN("fail to log ddl operation", K(ret), K(tenant_id));
 | 
						|
        } else if (OB_FAIL(
 | 
						|
                       ddl_operator.replace_sys_variable(sys_variable, new_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, trans))) {
 | 
						|
          LOG_WARN("ddl_operator init tenant env failed", "tenant_id", tenant_schema.get_tenant_id(), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        ret = E(EventTable::EN_CREATE_TENANT_TRANS_TWO_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 (OB_SUCC(ret)) {
 | 
						|
        create_tenant_status = CREATE_TENANT_TRANS_TWO_DONE;
 | 
						|
        if (OB_SUCCESS != (temp_ret = publish_schema(tenant_id))) {
 | 
						|
          LOG_WARN("publish schema failed", K(temp_ret), K(tenant_id));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      LOG_INFO("[CREATE_TENANT] end create tenant trans two", K(ret), K(temp_ret), K(tenant_id));
 | 
						|
    }
 | 
						|
 | 
						|
    // If transaction 2 fails, try to drop tenant
 | 
						|
    if (OB_FAIL(ret) && CREATE_TENANT_TRANS_ONE_DONE == create_tenant_status) {
 | 
						|
      int temp_ret = OB_SUCCESS;
 | 
						|
      obrpc::ObDropTenantArg arg;
 | 
						|
      arg.tenant_name_ = tenant_schema.get_tenant_name();
 | 
						|
      arg.if_exist_ = true;
 | 
						|
      arg.delay_to_drop_ = false;
 | 
						|
      ObSqlString sql;
 | 
						|
      if (OB_SUCCESS !=
 | 
						|
          (temp_ret = sql.append_fmt("DROP TENANT IF EXISTS %s FORCE", tenant_schema.get_tenant_name()))) {
 | 
						|
        LOG_WARN("fail to generate sql", K(temp_ret), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
      } else if (FALSE_IT(arg.ddl_stmt_str_ = sql.string())) {
 | 
						|
      } else if (OB_SUCCESS != (temp_ret = drop_tenant(arg))) {
 | 
						|
        LOG_WARN("fail to drop tenant", K(temp_ret), K(arg));
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    DEBUG_SYNC(BEFORE_CREATE_TENANT_TRANS_THREE);
 | 
						|
 | 
						|
    // Transaction 3: Write system tenant
 | 
						|
    // 1. The failure of the transaction does not affect the visibility of the tenant. The failure of the transaction
 | 
						|
    //  is compensated by the RS background inspection. The ordinary tenant DDL is prohibited
 | 
						|
    //  before the transaction is submitted.
 | 
						|
    // 2. liboblog uses transaction three submission as a sign of successful tenant creation
 | 
						|
    if (OB_SUCC(ret) && !is_follower_cluster && !arg.is_restore_) {
 | 
						|
      LOG_INFO("[CREATE_TENANT] start create tenant trans three", K(tenant_id));
 | 
						|
      // Transaction three needs to be executed on the system tenant
 | 
						|
      trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
      // Ensure that the schema_version monotonically increases among tenants' cross-tenant transactions
 | 
						|
      int64_t new_schema_version = OB_INVALID_VERSION;
 | 
						|
      int64_t sys_schema_version = OB_INVALID_VERSION;
 | 
						|
      int64_t refreshed_schema_version = OB_INVALID_VERSION;
 | 
						|
      ObSchemaService* schema_service_impl = schema_service_->get_schema_service();
 | 
						|
      if (OB_ISNULL(schema_service_impl)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("schema_service_impl is null", K(ret));
 | 
						|
      } else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(tenant_id, refreshed_schema_version))) {
 | 
						|
        LOG_WARN("fail to get tenant schema version", K(ret));
 | 
						|
      } else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(OB_SYS_TENANT_ID, sys_schema_version))) {
 | 
						|
        LOG_WARN("fail to get tenant schema version", K(ret));
 | 
						|
      } 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_FAIL(ret)) {
 | 
						|
      } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.create_tenant(tenant_schema, OB_DDL_ADD_TENANT_END, trans))) {
 | 
						|
        LOG_WARN("create tenant failed", K(tenant_schema), K(ret));
 | 
						|
      }
 | 
						|
 | 
						|
      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 (OB_SUCC(ret)) {
 | 
						|
        create_tenant_status = CREATE_TENANT_TRANS_THREE_DONE;
 | 
						|
        if (OB_SUCCESS != (temp_ret = publish_schema(OB_SYS_TENANT_ID))) {
 | 
						|
          LOG_WARN("publish schema failed", K(temp_ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      LOG_INFO("[CREATE_TENANT] end create tenant trans three", K(ret), K(temp_ret), K(tenant_id));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  LOG_INFO("finish create tenant env", K(tenant_schema), K(ret));
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::init_tenant_sys_params(
 | 
						|
    const ObCreateTenantArg& arg, ObTenantSchema& tenant_schema, ObSysVariableSchema& sys_variable_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  uint64_t tenant_id = tenant_schema.get_tenant_id();
 | 
						|
  // 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".
 | 
						|
  ObMalloc alloc(ObModIds::OB_TEMP_VARIABLES);
 | 
						|
  ObPtrGuard<ObSysParam, OB_MAX_SYS_PARAM_NUM> sys_params_guard(alloc);
 | 
						|
  sys_variable_schema.reset();
 | 
						|
  if (OB_FAIL(sys_params_guard.init())) {
 | 
						|
    LOG_WARN("alloc sys parameters failed", K(ret));
 | 
						|
  } else if (OB_INVALID_TENANT_ID == tenant_id) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_ISNULL(schema_service_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema_service must not null");
 | 
						|
  } else if (OB_ISNULL(sql_proxy_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sql_proxy must not null");
 | 
						|
  } 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_);
 | 
						|
    }
 | 
						|
    if (ObCompatibilityMode::OCEANBASE_MODE == tenant_schema.get_compatibility_mode()) {
 | 
						|
      tenant_schema.set_compatibility_mode(ObCompatibilityMode::MYSQL_MODE);
 | 
						|
    }
 | 
						|
    sys_variable_schema.set_tenant_id(tenant_schema.get_tenant_id());
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    ObSysParam* sys_params = sys_params_guard.ptr();
 | 
						|
    if (OB_FAIL(init_system_variables(arg, tenant_schema, sys_variable_schema, sys_params, OB_MAX_SYS_PARAM_NUM))) {
 | 
						|
      RS_LOG(WARN, "fail to init system variables", K(sys_variable_schema), K(arg), K(ret));
 | 
						|
    } else {
 | 
						|
      int64_t var_amount = ObSysVariables::get_amount();
 | 
						|
      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));
 | 
						|
        }
 | 
						|
        if (OB_SUCC(ret) && SYS_VAR_READ_ONLY == i) {
 | 
						|
          bool read_only_value = static_cast<bool>((sysvar_schema.get_value())[0] - '0');
 | 
						|
          tenant_schema.set_read_only(read_only_value);
 | 
						|
        }
 | 
						|
        // compat_mode is only set at the time of creation, and cannot be modified later,
 | 
						|
        // so it is only synchronized when the tenant is created
 | 
						|
        if (OB_SUCC(ret) && SYS_VAR_OB_COMPATIBILITY_MODE == i) {
 | 
						|
          ObCompatibilityMode mode = static_cast<ObCompatibilityMode>((sysvar_schema.get_value())[0] - '0');
 | 
						|
          tenant_schema.set_compatibility_mode(mode);
 | 
						|
        }
 | 
						|
        if (OB_SUCC(ret) && OB_FAIL(sys_variable_schema.add_sysvar_schema(sysvar_schema))) {
 | 
						|
          LOG_WARN("add system variable failed", K(ret));
 | 
						|
        }
 | 
						|
      }  // end for
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::wait_elect_sys_leaders(const ObArray<uint64_t>& table_ids, const ObArray<int64_t>& part_nums)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObLeaderElectionWaiter leader_waiter(*pt_operator_, stopped_);
 | 
						|
  int64_t timeout_us = THIS_WORKER.get_timeout_remain();
 | 
						|
  const int64_t conf_ddl_timeout = GCONF._ob_ddl_timeout;
 | 
						|
  if (INT64_MAX != THIS_WORKER.get_timeout_ts()) {
 | 
						|
    timeout_us = std::min(conf_ddl_timeout, timeout_us);
 | 
						|
  } else {
 | 
						|
    timeout_us = conf_ddl_timeout;
 | 
						|
  }
 | 
						|
 | 
						|
  if (!inited_) {
 | 
						|
    ret = OB_NOT_INIT;
 | 
						|
    LOG_WARN("not init");
 | 
						|
  } else if (OB_FAIL(leader_waiter.wait(table_ids, part_nums, timeout_us))) {
 | 
						|
    LOG_WARN("leader_waiter_ wait failed", K(table_ids), K(part_nums), K(timeout_us), K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// TOOD(): for test create tenant
 | 
						|
int ObDDLService::create_tenant_space(const uint64_t tenant_id)
 | 
						|
{
 | 
						|
  int ret = common::OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else {
 | 
						|
    std::string db;
 | 
						|
    db.append(getenv("USER"));
 | 
						|
    for (std::string::iterator it = db.begin(); it != db.end(); ++it) {
 | 
						|
      if (!isalnum(*it)) {
 | 
						|
        *it = '_';
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    ObSqlString sql;
 | 
						|
    int64_t affect_rows = 0;
 | 
						|
    sql.assign_fmt("drop database if exists %s_%lu", db.c_str(), tenant_id);
 | 
						|
    if (OB_FAIL(sql_proxy_->write(sql.ptr(), affect_rows))) {
 | 
						|
      LOG_WARN("execute create database sql failed", K(ret), K(sql));
 | 
						|
    } else if (OB_FAIL(sql.assign_fmt("create database %s_%lu", db.c_str(), tenant_id))) {
 | 
						|
      LOG_WARN("assign sql failed", K(ret));
 | 
						|
    } else if (OB_FAIL(sql_proxy_->write(sql.ptr(), affect_rows))) {
 | 
						|
      LOG_WARN("execute create database sql failed", K(ret), K(sql));
 | 
						|
    } else {
 | 
						|
      const schema_create_func* creator_ptr_array[] = {core_table_schema_creators, sys_table_schema_creators};
 | 
						|
      ObTableSchema tables[ARRAYSIZEOF(core_table_schema_creators) + ARRAYSIZEOF(sys_table_schema_creators)];
 | 
						|
      int64_t idx = 0;
 | 
						|
      // build system table schema to %tables
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < ARRAYSIZEOF(creator_ptr_array); i++) {
 | 
						|
        for (const schema_create_func* creator_ptr = creator_ptr_array[i]; OB_SUCC(ret) && NULL != *creator_ptr;
 | 
						|
             ++creator_ptr) {
 | 
						|
          if (OB_FAIL((*creator_ptr)(tables[idx++]))) {
 | 
						|
            LOG_WARN("create table schema fialed", K(ret));
 | 
						|
            ret = common::OB_SCHEMA_ERROR;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < ARRAYSIZEOF(tenant_space_tables); i++) {
 | 
						|
        uint64_t tid = tenant_space_tables[i];
 | 
						|
        // found table schema
 | 
						|
        bool exist = false;
 | 
						|
        int64_t tid_idx = 0;
 | 
						|
        for (idx = 0; idx < ARRAYSIZEOF(tables) && !exist; ++idx) {
 | 
						|
          if (tables[idx].get_table_id() == tid) {
 | 
						|
            tid_idx = idx;
 | 
						|
            exist = true;
 | 
						|
          }
 | 
						|
        }
 | 
						|
        UNUSED(tid_idx);
 | 
						|
        if (exist) {
 | 
						|
          HEAP_VAR(char[common::OB_MAX_SQL_LENGTH], csql)
 | 
						|
          {
 | 
						|
            MEMSET(csql, 0, sizeof(csql));
 | 
						|
            if (OB_FAIL(rootserver::ObSchema2DDLSql::convert(tables[tid_idx], csql, sizeof(csql)))) {
 | 
						|
              LOG_WARN("convert table schema to create table sql failed", K(ret));
 | 
						|
            } else if (OB_FAIL(sql_proxy_->write(tenant_id, csql, affect_rows))) {
 | 
						|
              LOG_WARN("execute sql failed", K(ret), "sql", csql);
 | 
						|
            }
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("Can not find table id", K(ret), K(tid));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
//----Functions for managing privileges----
 | 
						|
int ObDDLService::create_sys_user(ObUserInfo& user_info)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  uint64_t user_id = OB_INVALID_ID;
 | 
						|
  // no need to set creator in this case, set it OB_INVALID_ID
 | 
						|
  uint64_t creator_id = OB_INVALID_ID;
 | 
						|
  const uint64_t tenant_id = user_info.get_tenant_id();
 | 
						|
  ObString ddl_stmt_str("create user root");
 | 
						|
  ObString primary_zone;
 | 
						|
  if (OB_INVALID_ID == user_info.get_tenant_id() || OB_INVALID_ID == user_info.get_user_id()) {
 | 
						|
    LOG_WARN(
 | 
						|
        "tenant_id or user_id invalid", "tenant_id", user_info.get_tenant_id(), "user_id", user_info.get_user_id());
 | 
						|
  } else if (OB_FAIL(check_user_exist(user_info))) {
 | 
						|
    LOG_WARN("check_user_exist failed", K(user_info), K(ret));
 | 
						|
  } else if (OB_FAIL(create_user_in_trans(user_info, creator_id, user_id, primary_zone))) {
 | 
						|
    LOG_WARN("create_user_in_trans failed", K(user_info), K(ret), K(creator_id));
 | 
						|
  } else {
 | 
						|
    // publish schema
 | 
						|
    ret = publish_schema(tenant_id);
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
      LOG_WARN("publish schema failed", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::create_user(ObCreateUserArg& arg, ObIArray<int64_t>& 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, arg.is_replay_schema_, arg.primary_zone_))) {
 | 
						|
      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, const bool is_replay_schema, ObString& primary_zone)
 | 
						|
{
 | 
						|
  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, primary_zone))) {
 | 
						|
        LOG_WARN("create_user_in_trans failed", K(user_info), K(ret), K(creator_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (OB_ERR_USER_EXIST == ret) {
 | 
						|
    if (is_replay_schema) {
 | 
						|
      // If it is a replayed schema, and the user exists, you need to overwrite the user's information
 | 
						|
      if (OB_FAIL(replay_alter_user(user_info))) {
 | 
						|
        LOG_WARN("failed to replay alter user", K(ret), K(user_info));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  // 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<int64_t>& 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;
 | 
						|
  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(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("start transaction failed", K(ret));
 | 
						|
  } 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;
 | 
						|
          UInt64 affected_row;
 | 
						|
          if (OB_FAIL(drop_database(arg, affected_row, &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::rename_user(const ObRenameUserArg& arg, ObIArray<int64_t>& 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))) {
 | 
						|
      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)
 | 
						|
{
 | 
						|
  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 {
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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))) {
 | 
						|
          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))) {
 | 
						|
          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))) {
 | 
						|
        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)
 | 
						|
{
 | 
						|
  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 {
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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)
 | 
						|
{
 | 
						|
  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 {
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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 password", 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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  bool commit = false;
 | 
						|
  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 {
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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<ObString>& roles = arg.roles_;
 | 
						|
  const ObIArray<ObString>& remain_roles = arg.remain_roles_;
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  const ObUserInfo* user_info = NULL;
 | 
						|
  ObSysPriv* sys_priv = NULL;
 | 
						|
  uint64_t grantee_id;
 | 
						|
  ObSArray<ObString> users_name;
 | 
						|
  ObSArray<ObString> 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)));
 | 
						|
  }
 | 
						|
  OZ(trans.start(sql_proxy_));
 | 
						|
  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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  /* 1. deal with table privs first if has */
 | 
						|
  ObObjPrivSortKey obj_priv_key(arg.tenant_id_,
 | 
						|
      arg.object_id_,
 | 
						|
      static_cast<uint64_t>(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));
 | 
						|
  /* 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<uint64_t>(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));
 | 
						|
  }
 | 
						|
  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<uint64_t>(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));
 | 
						|
  }
 | 
						|
  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<uint64_t>(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));
 | 
						|
  }
 | 
						|
  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<uint64_t, 8> 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;
 | 
						|
  share::ObWorker::CompatMode compat_mode = share::ObWorker::CompatMode::INVALID;
 | 
						|
  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<ObString>& 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<ObString>& remain_roles = arg.remain_roles_;
 | 
						|
    const bool is_grant_sys = roles.count() == GRANT_SYS_ROLE_NUM;
 | 
						|
    const bool is_grant_role = roles.count() > GRANT_SYS_ROLE_NUM;
 | 
						|
    if (is_grant_sys) {
 | 
						|
      if (OB_FAIL(grant_sys_priv_to_ur(arg, schema_guard))) {
 | 
						|
        LOG_WARN("fail to grant sys priv", K(ret));
 | 
						|
      }
 | 
						|
    } else if (is_grant_role) {
 | 
						|
      // grant roles to user
 | 
						|
      // 1. Get the specified user granted
 | 
						|
      // 2. Get all the permissions owned by each role (level three: user/db/table)
 | 
						|
      // 3. All permissions of compatible role are appended to the specified user
 | 
						|
      ObSArray<ObString> users_name;
 | 
						|
      ObSArray<ObString> hosts_name;
 | 
						|
      ObDDLSQLTransaction trans(schema_service_);
 | 
						|
      const ObUserInfo* user_info = NULL;
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("Start transaction failed", K(ret));
 | 
						|
      } else if (roles.count() < GRANT_ROLE_MIN_ROLE_NUM) {
 | 
						|
        // Roles contain at least usr_name, host_name and one role_name
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("role info is illegal", K(roles.count()));
 | 
						|
      }
 | 
						|
      // Save all user names in array
 | 
						|
      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)));
 | 
						|
      }
 | 
						|
      /* resolve grantee, priv info stored in roles[0], roles[1],
 | 
						|
                          grantee info stored in roles[2], roles[3], ... */
 | 
						|
      ObArray<uint64_t> role_ids;
 | 
						|
      ObArray<ObUserInfo> roles_info;
 | 
						|
      for (int64_t i = GRANT_ROLE_MIN_ROLE_NUM - 1; OB_SUCC(ret) && i < roles.count(); ++i) {
 | 
						|
        // Oracle does not currently support the creation of a role by specifying the hostname
 | 
						|
        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_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 /*user_name*/, host_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 (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 (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_);
 | 
						|
          if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id,
 | 
						|
                  *user_info,
 | 
						|
                  role_ids,
 | 
						|
                  NULL, /*Single specified role info*/
 | 
						|
                  trans,
 | 
						|
                  true /*is_grant*/,
 | 
						|
                  arg.option_))) {
 | 
						|
            LOG_WARN("falied to grant_revoke_role", 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 (FAILEDx(publish_schema(tenant_id))) {
 | 
						|
        LOG_WARN("pubish schema failed", K(ret));
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      const ObSArray<ObString>& users_passwd = arg.users_passwd_;
 | 
						|
      const ObSArray<ObString>& 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))) {
 | 
						|
                LOG_WARN("Set password error", K(ret));
 | 
						|
              }
 | 
						|
            }
 | 
						|
          } 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);
 | 
						|
                ObString primary_zone;
 | 
						|
                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, false /*is_replay_schema*/, primary_zone))) {
 | 
						|
                  LOG_WARN("Create user error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret));
 | 
						|
                } else {
 | 
						|
                  is_user_exist = true;
 | 
						|
                }
 | 
						|
              }
 | 
						|
            } else {
 | 
						|
              // share::ObWorker::CompatMode compat_mode = share::ObWorker::CompatMode::INVALID;
 | 
						|
              tmp_ret = OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST;
 | 
						|
              if (OB_SUCC(ret) && ObWorker::CompatMode::ORACLE == compat_mode) {
 | 
						|
                LOG_USER_ERROR(OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST, user_name.length(), user_name.ptr());
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
          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 (ObWorker::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<uint64_t>(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))) {
 | 
						|
                  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));
 | 
						|
              }
 | 
						|
            }
 | 
						|
          }
 | 
						|
        }
 | 
						|
        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(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<uint64_t> 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(role_id, role_info))) {
 | 
						|
        LOG_WARN("Failed to get role info", K(ret), 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<uint64_t, 8> 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;
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("Start transaction failed", K(ret));
 | 
						|
      } else if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id,
 | 
						|
                     user,
 | 
						|
                     role_ids,
 | 
						|
                     NULL /*Single specified role info*/,
 | 
						|
                     trans,
 | 
						|
                     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))) {
 | 
						|
      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_))) {
 | 
						|
      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)
 | 
						|
{
 | 
						|
  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 {
 | 
						|
    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))) {
 | 
						|
          LOG_WARN("Grant user error", K(ret));
 | 
						|
        }
 | 
						|
        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))) {
 | 
						|
          LOG_WARN("Grant database error", K(ret));
 | 
						|
        }
 | 
						|
        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))) {
 | 
						|
          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)
 | 
						|
{
 | 
						|
  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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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)
 | 
						|
{
 | 
						|
  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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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 profile_id, ObSchemaGetterGuard& schema_guard, ObIArray<uint64_t>& user_ids)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaService* schema_sql_service = NULL;
 | 
						|
  ObSEArray<const ObUserInfo*, 10> user_infos;
 | 
						|
  uint64_t tenant_id = extract_tenant_id(profile_id);
 | 
						|
 | 
						|
  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<uint64_t>& role_id_array,
 | 
						|
    ObIArray<uint64_t>& disable_flag_array)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  uint64_t role_id;
 | 
						|
  uint64_t target_disable_flag = 0;
 | 
						|
  uint64_t org_disable_flag = 0;
 | 
						|
  uint64_t seq = OB_INVALID_ID;
 | 
						|
  uint64_t option = OB_INVALID_ID;
 | 
						|
  ObSEArray<uint64_t, 8> org_role_id_array;
 | 
						|
  ObSEArray<uint64_t, 8> 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<uint64_t, 8> role_id_array;
 | 
						|
  ObSEArray<uint64_t, 8> 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 {
 | 
						|
    const ObUserInfo* role_info = NULL;
 | 
						|
    uint64_t seq = OB_INVALID_ID;
 | 
						|
    /* 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_);
 | 
						|
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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_);
 | 
						|
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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<int64_t>& 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@: 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))) {
 | 
						|
        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)
 | 
						|
{
 | 
						|
  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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } 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))) {
 | 
						|
      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))) {
 | 
						|
      LOG_WARN("Grant table error", K(ret));
 | 
						|
    }
 | 
						|
  } 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)
 | 
						|
{
 | 
						|
  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_);
 | 
						|
    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_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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;
 | 
						|
  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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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::ObString* ddl_stmt_str)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  ObSysPriv* sys_priv = 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 {
 | 
						|
    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_));
 | 
						|
    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,
 | 
						|
             NO_OPTION,
 | 
						|
             sys_priv_array,
 | 
						|
             trans,
 | 
						|
             false /*is_grant*/,
 | 
						|
             ddl_stmt_str,
 | 
						|
             schema_guard),
 | 
						|
          tenant_id,
 | 
						|
          grantee_id,
 | 
						|
          sys_priv_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;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  // 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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = table_key.tenant_id_;
 | 
						|
  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 {
 | 
						|
    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_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = table_key.tenant_id_;
 | 
						|
  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 {
 | 
						|
    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_))) {
 | 
						|
      LOG_WARN("Start transaction failed", K(ret));
 | 
						|
    } 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_;
 | 
						|
  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 with version in inner table", K(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_))) {
 | 
						|
      LOG_WARN("Start transaction failed, ", K(ret));
 | 
						|
    } 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)
 | 
						|
{
 | 
						|
  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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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_);
 | 
						|
          if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
            LOG_WARN("start transaction failed", K(ret));
 | 
						|
          } 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;
 | 
						|
    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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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;
 | 
						|
}
 | 
						|
 | 
						|
//----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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start transaction", K(ret));
 | 
						|
    } 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;
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failestart transaction", K(ret));
 | 
						|
    } 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)
 | 
						|
{
 | 
						|
  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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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;
 | 
						|
    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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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::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_, *sql_proxy_);
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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))) {
 | 
						|
            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))) {
 | 
						|
            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::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)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObDDLSQLTransaction trans(schema_service_);
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init");
 | 
						|
  } else if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
    LOG_WARN("Failed to start trans", K(ret));
 | 
						|
  } else {
 | 
						|
    const uint64_t tenant_id = user_info.get_tenant_id();
 | 
						|
    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, ObString& primary_zone)
 | 
						|
{
 | 
						|
  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_);
 | 
						|
    ObSqlString ddl_stmt_str;
 | 
						|
    ObString ddl_sql;
 | 
						|
    const bool is_role = user_info.is_role();
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("Failed to start trans", K(ret));
 | 
						|
    } 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();
 | 
						|
    }
 | 
						|
  }
 | 
						|
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const ObSysVariableSchema* sys_variable_schema = NULL;
 | 
						|
  bool is_oracle_mode = false;
 | 
						|
  const uint64_t tenant_id = user_info.get_tenant_id();
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
    // do-nothing
 | 
						|
  } 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_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_primary_zone(primary_zone))) {
 | 
						|
      LOG_WARN("fail to set primary zone", K(ret));
 | 
						|
    } else 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<uint64_t> role_ids;
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
      LOG_WARN("failed to grant role to creator", K(ret));
 | 
						|
    } else if (OB_SYS_USER_ID == extract_pure_id(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(creator_id, creator_info))) {
 | 
						|
      LOG_WARN("get_user_info failed", K(ret), 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 /*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----
 | 
						|
 | 
						|
// Some modifications to the included databases when alter tenant, currently only when tenant alter locality
 | 
						|
// Adjust the primary zone of databases
 | 
						|
int ObDDLService::try_modify_databases_attributes_in_tenant(const obrpc::ObModifyTenantArg& arg,
 | 
						|
    ObDDLOperator& ddl_operator, ObMySQLTransaction& trans, share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    share::schema::ObTenantSchema& tenant_schema, int64_t& sub_pz_count)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  sub_pz_count = 0;
 | 
						|
  if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) {
 | 
						|
    common::ObArray<const ObDatabaseSchema*> database_schemas;
 | 
						|
    if (OB_FAIL(schema_guard.get_database_schemas_in_tenant(tenant_schema.get_tenant_id(), database_schemas))) {
 | 
						|
      LOG_WARN("fail to get databases schemas in tenant", K(ret));
 | 
						|
    } else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
      LOG_WARN("fail to get zone list", K(ret), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < database_schemas.count(); ++i) {
 | 
						|
        const ObDatabaseSchema* db_schema = database_schemas.at(i);
 | 
						|
        if (OB_UNLIKELY(NULL == db_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("db schema is null", K(ret), KP(db_schema));
 | 
						|
        } else if (db_schema->get_primary_zone().length() <= 0) {
 | 
						|
          // empty primary zone, no need to trim
 | 
						|
        } else {
 | 
						|
          ObDatabaseSchema new_db_schema = *db_schema;
 | 
						|
          if (OB_FAIL(trim_and_set_primary_zone(new_db_schema, *db_schema, zone_list, schema_guard))) {
 | 
						|
            LOG_WARN("fail to trim and set primary zone", K(ret));
 | 
						|
          } else if (OB_FAIL(ddl_operator.alter_database(new_db_schema, trans, OB_DDL_ALTER_DATABASE))) {
 | 
						|
            LOG_WARN("fail to alter database primary zone", K(ret));
 | 
						|
          } else if (!db_schema->get_primary_zone().empty() && new_db_schema.get_primary_zone().empty()) {
 | 
						|
            ++sub_pz_count;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// Some modifications to the included tablegroups when alter tenant, currently only when tenant alter locality
 | 
						|
// Adjust the primary zone of tablegroups
 | 
						|
int ObDDLService::try_modify_tablegroups_attributes_in_tenant(const obrpc::ObModifyTenantArg& arg,
 | 
						|
    ObDDLOperator& ddl_operator, ObMySQLTransaction& trans, share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    share::schema::ObTenantSchema& tenant_schema, int64_t& sub_pz_count)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  sub_pz_count = 0;
 | 
						|
  if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) {
 | 
						|
    common::ObArray<const ObSimpleTablegroupSchema*> tablegroup_schemas;
 | 
						|
    if (OB_FAIL(schema_guard.get_tablegroup_schemas_in_tenant(tenant_schema.get_tenant_id(), tablegroup_schemas))) {
 | 
						|
      LOG_WARN("fail to get tablegroup schemas in tenant", K(ret));
 | 
						|
    } else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
      LOG_WARN("fail to get zone list", K(ret), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < tablegroup_schemas.count(); ++i) {
 | 
						|
        const ObSimpleTablegroupSchema* simple_schema = tablegroup_schemas.at(i);
 | 
						|
        const ObTablegroupSchema* tg_schema = nullptr;
 | 
						|
        if (OB_UNLIKELY(nullptr == simple_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("table group schema is null", K(ret));
 | 
						|
        } else if (!is_new_tablegroup_id(simple_schema->get_tablegroup_id())) {
 | 
						|
          // not a new tablegroup, has no primary zone attribute
 | 
						|
        } else if (simple_schema->get_primary_zone().length() <= 0) {
 | 
						|
          // empty primary zone, no need to trim
 | 
						|
        } else if (OB_FAIL(schema_guard.get_tablegroup_schema(simple_schema->get_tablegroup_id(), tg_schema))) {
 | 
						|
          LOG_WARN("fail to get tablegroup schema", K(ret));
 | 
						|
        } else if (OB_UNLIKELY(nullptr == tg_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("tg_schema ptrs is null", K(ret));
 | 
						|
        } else {
 | 
						|
          ObTablegroupSchema new_tg_schema = *tg_schema;
 | 
						|
          if (OB_FAIL(trim_and_set_primary_zone(new_tg_schema, *tg_schema, zone_list, schema_guard))) {
 | 
						|
            LOG_WARN("fail to trim and set primary zone", K(ret));
 | 
						|
          } else if (OB_FAIL(ddl_operator.alter_tablegroup(new_tg_schema, trans))) {
 | 
						|
            LOG_WARN("fail to alter tablegroup primary zone", K(ret));
 | 
						|
          } else if (!tg_schema->get_primary_zone().empty() && new_tg_schema.get_primary_zone().empty()) {
 | 
						|
            ++sub_pz_count;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// Some modifications to the included tables when alter tenant, currently only when tenant alter locality
 | 
						|
// Adjust the primary zone of the tables
 | 
						|
int ObDDLService::try_modify_tables_attributes_in_tenant(const obrpc::ObModifyTenantArg& arg,
 | 
						|
    ObDDLOperator& ddl_operator, ObMySQLTransaction& trans, share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    share::schema::ObTenantSchema& tenant_schema, int64_t& sub_pz_count)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  sub_pz_count = 0;
 | 
						|
  if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) {
 | 
						|
    common::ObArray<const ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
    if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(tenant_schema.get_tenant_id(), table_schemas))) {
 | 
						|
      LOG_WARN("fail to get table schemas in tenant", K(ret));
 | 
						|
    } else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
 | 
						|
      LOG_WARN("fail to get zone list", K(ret), "tenant_id", tenant_schema.get_tenant_id());
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); ++i) {
 | 
						|
        const ObSimpleTableSchemaV2* simple_schema = table_schemas.at(i);
 | 
						|
        const ObTableSchema* table_schema = nullptr;
 | 
						|
        if (OB_UNLIKELY(nullptr == simple_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("table group schema is null", K(ret));
 | 
						|
        } else if (simple_schema->get_primary_zone().length() <= 0) {
 | 
						|
          // empty primary zone, no need to trim
 | 
						|
        } else if (OB_FAIL(schema_guard.get_table_schema(simple_schema->get_table_id(), table_schema))) {
 | 
						|
          LOG_WARN("fail to get table schema", K(ret));
 | 
						|
        } else if (OB_UNLIKELY(nullptr == table_schema)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("table_schema ptr is null", K(ret));
 | 
						|
        } else {
 | 
						|
          ObTableSchema new_table_schema;
 | 
						|
          if (OB_FAIL(new_table_schema.assign(*table_schema))) {
 | 
						|
            LOG_WARN("fail to assign schema", K(ret));
 | 
						|
          } else if (OB_FAIL(trim_and_set_primary_zone(new_table_schema, *table_schema, zone_list, schema_guard))) {
 | 
						|
            LOG_WARN("fail to trim and set primary zone", K(ret));
 | 
						|
          } else if (OB_FAIL(ddl_operator.alter_table_options(
 | 
						|
                         schema_guard, new_table_schema, *table_schema, false /*update index*/, trans))) {
 | 
						|
            LOG_WARN("fail to alter table primary zone", K(ret));
 | 
						|
          } else if (!table_schema->get_primary_zone().empty() && new_table_schema.get_primary_zone().empty()) {
 | 
						|
            ++sub_pz_count;
 | 
						|
          }
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
template <typename SCHEMA>
 | 
						|
int ObDDLService::get_schema_primary_regions(const SCHEMA& schema, share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    common::ObIArray<common::ObRegion>& primary_regions)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  primary_regions.reset();
 | 
						|
  common::ObArray<common::ObZone> zone_list;
 | 
						|
  ObSchema tmp_schema;
 | 
						|
  ObPrimaryZone primary_zone_schema(&tmp_schema);
 | 
						|
  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<share::ObZoneReplicaAttrSet> 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<ObZoneScore>& 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;
 | 
						|
}
 | 
						|
 | 
						|
template <typename TABLE_SCHEMA>
 | 
						|
int ObDDLService::check_table_primary_zone_gts_condition(const share::schema::ObTenantSchema& old_tenant_schema,
 | 
						|
    const share::schema::ObTenantSchema& new_tenant_schema, const share::schema::ObSysVariableSchema& new_sys_variable,
 | 
						|
    const TABLE_SCHEMA& table_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = old_tenant_schema.get_tenant_id();
 | 
						|
  const ObSysVariableSchema* old_sys_variable = NULL;
 | 
						|
  GTSOperation gts_operation = GTS_OPERATION_INVALID;
 | 
						|
  common::ObArray<common::ObRegion> tenant_primary_regions;
 | 
						|
  common::ObArray<common::ObRegion> table_primary_regions;
 | 
						|
  if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2000) {
 | 
						|
    // Do not check before the upgrade is completed
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, old_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(old_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_tenant_gts_operation(*old_sys_variable, new_sys_variable, gts_operation))) {
 | 
						|
    LOG_WARN("fail to check tenant gts operation", K(ret));
 | 
						|
  } else if (GTS_OPERATION_INVALID == gts_operation) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("gts operation unexpected", K(ret));
 | 
						|
  } else if (GTS_REMAIN_OFF == gts_operation || GTS_TURN_OFF == gts_operation) {
 | 
						|
    // GTS off, it is not neccessary to check
 | 
						|
  } else 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 (OB_FAIL(get_schema_primary_regions(table_schema, schema_guard, table_primary_regions))) {
 | 
						|
    LOG_WARN("fail to get table primary regions", K(ret));
 | 
						|
  } else if (tenant_primary_regions.count() <= 0 || table_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 when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant primary zone span regions when GTS is on");
 | 
						|
  } else if (table_primary_regions.count() > 1) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("table primary zone span regions when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "table primary zone span regions when GTS is on");
 | 
						|
  } else if (tenant_primary_regions.at(0) != table_primary_regions.at(0)) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("table and tenant primary zone span regions when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "primary zones of table and tenant at different region when GTS is on");
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_database_primary_zone_gts_condition(const share::schema::ObTenantSchema& old_tenant_schema,
 | 
						|
    const share::schema::ObTenantSchema& new_tenant_schema, const share::schema::ObSysVariableSchema& new_sys_variable,
 | 
						|
    const share::schema::ObDatabaseSchema& database_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = old_tenant_schema.get_tenant_id();
 | 
						|
  const ObSysVariableSchema* old_sys_variable = NULL;
 | 
						|
  common::ObArray<common::ObRegion> tenant_primary_regions;
 | 
						|
  common::ObArray<common::ObRegion> database_primary_regions;
 | 
						|
  GTSOperation gts_operation = GTS_OPERATION_INVALID;
 | 
						|
  if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2000) {
 | 
						|
    // Do not check before the upgrade is completed
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, old_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(old_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_tenant_gts_operation(*old_sys_variable, new_sys_variable, gts_operation))) {
 | 
						|
    LOG_WARN("fail to check tenant gts operation", K(ret));
 | 
						|
  } else if (GTS_OPERATION_INVALID == gts_operation) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("gts operation unexpected", K(ret));
 | 
						|
  } else if (GTS_REMAIN_OFF == gts_operation || GTS_TURN_OFF == gts_operation) {
 | 
						|
    // GTS off, it is not neccesary to check
 | 
						|
  } else 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 (OB_FAIL(get_schema_primary_regions(database_schema, schema_guard, database_primary_regions))) {
 | 
						|
    LOG_WARN("fail to get table primary regions", K(ret));
 | 
						|
  } else if (tenant_primary_regions.count() <= 0 || database_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 when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant primary zone span regions when GTS is on");
 | 
						|
  } else if (database_primary_regions.count() > 1) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("database primary zone span regions when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "database primary zone span regions when GTS is on");
 | 
						|
  } else if (tenant_primary_regions.at(0) != database_primary_regions.at(0)) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("database and tenant primary zone at different regions when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "primary zones of database and tenant at different region when GTS is on");
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_new_tablegroup_primary_zone_gts_condition(
 | 
						|
    const share::schema::ObTenantSchema& old_tenant_schema, const share::schema::ObTenantSchema& new_tenant_schema,
 | 
						|
    const share::schema::ObSysVariableSchema& new_sys_variable,
 | 
						|
    const share::schema::ObTablegroupSchema& tablegroup_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const uint64_t tenant_id = old_tenant_schema.get_tenant_id();
 | 
						|
  const ObSysVariableSchema* old_sys_variable = NULL;
 | 
						|
  GTSOperation gts_operation = GTS_OPERATION_INVALID;
 | 
						|
  common::ObArray<common::ObRegion> tenant_primary_regions;
 | 
						|
  common::ObArray<common::ObRegion> tablegroup_primary_regions;
 | 
						|
  if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2000) {
 | 
						|
    // Do not check before the upgrade is completed
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, old_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(old_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_tenant_gts_operation(*old_sys_variable, new_sys_variable, gts_operation))) {
 | 
						|
    LOG_WARN("fail to check tenant gts operation", K(ret));
 | 
						|
  } else if (GTS_OPERATION_INVALID == gts_operation) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("gts operation unexpected", K(ret));
 | 
						|
  } else if (GTS_REMAIN_OFF == gts_operation || GTS_TURN_OFF == gts_operation) {
 | 
						|
    // GTS off, it is not neccesary to check
 | 
						|
  } else 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 (OB_FAIL(get_schema_primary_regions(tablegroup_schema, schema_guard, tablegroup_primary_regions))) {
 | 
						|
    LOG_WARN("fail to get table primary regions", K(ret));
 | 
						|
  } else if (tenant_primary_regions.count() <= 0 || tablegroup_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 when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant primary zone span regions when GTS is on");
 | 
						|
  } else if (tablegroup_primary_regions.count() > 1) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("tablegroup primary zone span regions when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "tablegroup primary zone span regions when GTS is on");
 | 
						|
  } else if (tenant_primary_regions.at(0) != tablegroup_primary_regions.at(0)) {
 | 
						|
    ret = OB_NOT_SUPPORTED;
 | 
						|
    LOG_WARN("tablegroup and tenant primary zone at different region when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "primary zones of tablegroup and tenant at different region when GTS is on");
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// old_tenant_schema is taken from the schema service, with full system variable information
 | 
						|
// new_tenant_schema means to save the modified system variables. By comparing the two,
 | 
						|
// the switch action and switch state of gts can be achieved
 | 
						|
int ObDDLService::check_tenant_gts_operation(const share::schema::ObSysVariableSchema& old_sys_variable,
 | 
						|
    const share::schema::ObSysVariableSchema& new_sys_variable, GTSOperation& gts_operation)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  gts_operation = GTS_OPERATION_INVALID;
 | 
						|
  const common::ObString gts_name(share::OB_SV_TIMESTAMP_SERVICE);
 | 
						|
  const ObSysVarSchema* old_gts_schema = NULL;
 | 
						|
  const ObSysVarSchema* new_gts_schema = NULL;
 | 
						|
  ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
 | 
						|
  ObObj old_gts_obj;
 | 
						|
  ObObj new_gts_obj;
 | 
						|
  int64_t old_gts_value = 0;
 | 
						|
  int64_t new_gts_value = 0;
 | 
						|
  if (OB_FAIL(old_sys_variable.get_sysvar_schema(gts_name, old_gts_schema))) {
 | 
						|
    LOG_WARN("fail to get sysvar schema", K(ret));
 | 
						|
  } else if (OB_FAIL(new_sys_variable.get_sysvar_schema(gts_name, new_gts_schema))) {
 | 
						|
    if (OB_ERR_SYS_VARIABLE_UNKNOWN != ret) {
 | 
						|
      LOG_WARN("fail to get sysvar schema", K(ret));
 | 
						|
    } else {
 | 
						|
      // No adjustment of gts switch this time
 | 
						|
      ret = OB_SUCCESS;  // rewrite
 | 
						|
      if (OB_FAIL(old_gts_schema->get_value(&allocator, NULL, old_gts_obj))) {
 | 
						|
        LOG_WARN("fail to get value", K(ret));
 | 
						|
      } else if (OB_FAIL(old_gts_obj.get_int(old_gts_value))) {
 | 
						|
        LOG_WARN("fail to get int from gts obj", K(ret));
 | 
						|
      } else {
 | 
						|
        if (transaction::is_ts_type_external_consistent(old_gts_value)) {
 | 
						|
          gts_operation = GTS_REMAIN_ON;
 | 
						|
        } else {
 | 
						|
          gts_operation = GTS_REMAIN_OFF;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else if (OB_UNLIKELY(NULL == old_gts_schema || NULL == new_gts_schema)) {
 | 
						|
    ret = OB_SCHEMA_ERROR;
 | 
						|
    LOG_WARN("sysvar schema ob_timestampe_service not exist", K(ret), KP(old_gts_schema), K(new_gts_schema));
 | 
						|
  } else if (OB_FAIL(old_gts_schema->get_value(&allocator, NULL, old_gts_obj))) {
 | 
						|
    LOG_WARN("fail to get value", K(ret));
 | 
						|
  } else if (OB_FAIL(new_gts_schema->get_value(&allocator, NULL, new_gts_obj))) {
 | 
						|
    LOG_WARN("fail to get value", K(ret));
 | 
						|
  } else if (OB_FAIL(old_gts_obj.get_int(old_gts_value))) {
 | 
						|
    LOG_WARN("fail to get int from gts obj", K(ret));
 | 
						|
  } else if (OB_FAIL(new_gts_obj.get_int(new_gts_value))) {
 | 
						|
    LOG_WARN("fail to get int from gts obj", K(ret));
 | 
						|
  } else if (transaction::is_ts_type_external_consistent(old_gts_value) ==
 | 
						|
             transaction::is_ts_type_external_consistent(new_gts_value)) {
 | 
						|
    if (transaction::is_ts_type_external_consistent(old_gts_value)) {
 | 
						|
      gts_operation = GTS_REMAIN_ON;
 | 
						|
    } else {
 | 
						|
      gts_operation = GTS_REMAIN_OFF;
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    if (transaction::is_ts_type_external_consistent(new_gts_value)) {
 | 
						|
      gts_operation = GTS_TURN_ON;
 | 
						|
    } else {
 | 
						|
      gts_operation = GTS_TURN_OFF;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_tenant_primary_zone_gts_condition(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObTenantSchema& new_tenant_schema, const share::schema::ObSysVariableSchema& new_sys_variable,
 | 
						|
    const share::schema::ObSysVariableSchema& old_sys_variable)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  GTSOperation gts_operation = GTS_OPERATION_INVALID;
 | 
						|
  common::ObArray<common::ObRegion> tenant_primary_regions;
 | 
						|
  if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2000) {
 | 
						|
    // Do not check before the upgrade is completed
 | 
						|
  } else if (OB_FAIL(check_tenant_gts_operation(old_sys_variable, new_sys_variable, gts_operation))) {
 | 
						|
    LOG_WARN("fail to check tenant gts operation", K(ret));
 | 
						|
  } else if (GTS_OPERATION_INVALID == gts_operation) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("gts operation unexpected", K(ret));
 | 
						|
  } else if (GTS_REMAIN_OFF == gts_operation || GTS_TURN_OFF == gts_operation) {
 | 
						|
    // GTS off, it is neccesary to check
 | 
						|
  } else 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 when GTS is on not supported", K(ret));
 | 
						|
    LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant primary zone span regions when GTS is on");
 | 
						|
  }
 | 
						|
  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<ObZone> 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_alter_table_replica_options(const obrpc::ObAlterTableArg& arg,
 | 
						|
    share::schema::ObTableSchema& new_table_schema, const share::schema::ObTableSchema& orig_table_schema,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  const share::schema::ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const ObSysVariableSchema* new_sys_variable = NULL;
 | 
						|
  const uint64_t tenant_id = new_table_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_FAIL(new_table_schema.get_zone_list(schema_guard, zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(check_alter_schema_replica_options(
 | 
						|
                 arg.alter_table_schema_.alter_option_bitset_.has_member(ObAlterTableArg::PRIMARY_ZONE),
 | 
						|
                 new_table_schema,
 | 
						|
                 orig_table_schema,
 | 
						|
                 zone_list,
 | 
						|
                 schema_guard))) {
 | 
						|
    LOG_WARN("fail to check replica options", K(ret));
 | 
						|
  } else {
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_FAIL(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 ptr is null", K(ret));
 | 
						|
    // both new and old use tenant schema
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, new_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(new_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (!new_table_schema.has_partition()) {
 | 
						|
    // bypass
 | 
						|
  } else if (OB_FAIL(check_table_primary_zone_gts_condition(
 | 
						|
                 *tenant_schema, *tenant_schema, *new_sys_variable, new_table_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail ot check schema primary zone gts condition", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_alter_database_replica_options(const obrpc::ObAlterDatabaseArg& arg,
 | 
						|
    share::schema::ObDatabaseSchema& new_database_schema, const share::schema::ObDatabaseSchema& orig_database_schema,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  const share::schema::ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const ObSysVariableSchema* new_sys_variable = NULL;
 | 
						|
  const uint64_t tenant_id = new_database_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_FAIL(new_database_schema.get_zone_list(schema_guard, 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(ObAlterDatabaseArg::PRIMARY_ZONE),
 | 
						|
                 new_database_schema,
 | 
						|
                 orig_database_schema,
 | 
						|
                 zone_list,
 | 
						|
                 schema_guard))) {
 | 
						|
    LOG_WARN("fail to check replica options", K(ret));
 | 
						|
  } else {
 | 
						|
  }  // no more
 | 
						|
 | 
						|
  if (OB_FAIL(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 ptr is null", K(ret));
 | 
						|
    // both new and old use tenant schema
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, new_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(new_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_database_primary_zone_gts_condition(
 | 
						|
                 *tenant_schema, *tenant_schema, *new_sys_variable, new_database_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail ot check schema primary zone gts condition", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_alter_tablegroup_replica_options(const obrpc::ObAlterTablegroupArg& arg,
 | 
						|
    share::schema::ObSchemaGetterGuard& schema_guard, const share::schema::ObTablegroupSchema& orig_tablegroup_schema,
 | 
						|
    share::schema::ObTablegroupSchema& new_tablegroup_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  const ObSysVariableSchema* new_sys_variable = NULL;
 | 
						|
  const share::schema::ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const uint64_t tenant_id = new_tablegroup_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_FAIL(new_tablegroup_schema.get_zone_list(schema_guard, 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(ObAlterTablegroupArg::PRIMARY_ZONE),
 | 
						|
                 new_tablegroup_schema,
 | 
						|
                 orig_tablegroup_schema,
 | 
						|
                 zone_list,
 | 
						|
                 schema_guard))) {
 | 
						|
    LOG_WARN("fail to check replica options", K(ret));
 | 
						|
  } else {
 | 
						|
  }  // no more
 | 
						|
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (!is_new_tablegroup_id(new_tablegroup_schema.get_tablegroup_id())) {
 | 
						|
    // by pass
 | 
						|
  } 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 ptr is null", K(ret));
 | 
						|
    // both new and old use tenant schema
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, new_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(new_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_new_tablegroup_primary_zone_gts_condition(
 | 
						|
                 *tenant_schema, *tenant_schema, *new_sys_variable, new_tablegroup_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail ot check schema primary zone gts condition", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_create_tenant_replica_options(
 | 
						|
    share::schema::ObTenantSchema& tenant_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> 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_create_table_replica_options(
 | 
						|
    share::schema::ObSimpleTableSchemaV2& table_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  const share::schema::ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const ObSysVariableSchema* new_sys_variable = NULL;
 | 
						|
  const uint64_t tenant_id = table_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_FAIL(table_schema.get_zone_list(schema_guard, zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(check_create_schema_replica_options(table_schema, zone_list, schema_guard))) {
 | 
						|
    LOG_WARN("fail to check replica options", K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_FAIL(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 ptr is null", K(ret));
 | 
						|
    // both new and old use tenant schema
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, new_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(new_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (!table_schema.has_partition()) {
 | 
						|
    // bypass
 | 
						|
  } else if (OB_FAIL(check_table_primary_zone_gts_condition(
 | 
						|
                 *tenant_schema, *tenant_schema, *new_sys_variable, table_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail ot check schema primary zone gts condition", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_create_database_replica_options(
 | 
						|
    share::schema::ObDatabaseSchema& database_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  const share::schema::ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const ObSysVariableSchema* new_sys_variable = NULL;
 | 
						|
  const uint64_t tenant_id = database_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_FAIL(database_schema.get_zone_list(schema_guard, zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(check_create_schema_replica_options(database_schema, zone_list, schema_guard))) {
 | 
						|
    LOG_WARN("fail to check replica options", K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_FAIL(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 ptr is null", K(ret));
 | 
						|
    // both new and old use tenant schema
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, new_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(new_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_database_primary_zone_gts_condition(
 | 
						|
                 *tenant_schema, *tenant_schema, *new_sys_variable, database_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail ot check schema primary zone gts condition", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_create_tablegroup_replica_options(
 | 
						|
    share::schema::ObTablegroupSchema& tablegroup_schema, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> zone_list;
 | 
						|
  const share::schema::ObTenantSchema* tenant_schema = NULL;
 | 
						|
  const ObSysVariableSchema* new_sys_variable = NULL;
 | 
						|
  const uint64_t tenant_id = tablegroup_schema.get_tenant_id();
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (OB_FAIL(tablegroup_schema.get_zone_list(schema_guard, zone_list))) {
 | 
						|
    LOG_WARN("fail to get zone list", K(ret));
 | 
						|
  } else if (OB_FAIL(check_create_schema_replica_options(tablegroup_schema, zone_list, schema_guard))) {
 | 
						|
    LOG_WARN("fail to check replica options", K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  // The newly created table group must be new tg
 | 
						|
  if (OB_FAIL(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 ptr is null", K(ret));
 | 
						|
    // both new and old use tenant schema
 | 
						|
  } else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, new_sys_variable))) {
 | 
						|
    LOG_WARN("get sys variable schema failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(new_sys_variable)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sys variable schema is null", K(ret));
 | 
						|
  } else if (OB_FAIL(check_new_tablegroup_primary_zone_gts_condition(
 | 
						|
                 *tenant_schema, *tenant_schema, *new_sys_variable, tablegroup_schema, schema_guard))) {
 | 
						|
    LOG_WARN("fail ot check schema primary zone gts condition", K(ret));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::check_schema_zone_list(common::ObArray<common::ObZone>& 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 <typename SCHEMA>
 | 
						|
int ObDDLService::check_and_set_primary_zone(
 | 
						|
    SCHEMA& schema, const common::ObIArray<common::ObZone>& zone_list, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<ObZoneRegion> 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 <typename SCHEMA>
 | 
						|
int ObDDLService::check_empty_primary_zone_locality_condition(
 | 
						|
    SCHEMA& schema, const common::ObIArray<common::ObZone>& zone_list, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchema for_alloc_schema;
 | 
						|
  ObPrimaryZone primary_zone_schema(&for_alloc_schema);
 | 
						|
  ObArray<share::ObZoneReplicaAttrSet> zone_locality;
 | 
						|
  ObArray<common::ObZone> first_primary_zone_array;
 | 
						|
  ObArray<share::schema::ObZoneRegion> 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 <typename SCHEMA>
 | 
						|
int ObDDLService::check_create_schema_replica_options(
 | 
						|
    SCHEMA& schema, common::ObArray<common::ObZone>& 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<ObZoneScore> 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 <typename SCHEMA>
 | 
						|
int ObDDLService::check_alter_schema_replica_options(const bool alter_primary_zone, SCHEMA& new_schema,
 | 
						|
    const SCHEMA& orig_schema, common::ObArray<common::ObZone>& 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<ObZoneScore> 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 <typename SCHEMA>
 | 
						|
int ObDDLService::trim_and_set_primary_zone(SCHEMA& new_schema, const SCHEMA& orig_schema,
 | 
						|
    const common::ObIArray<common::ObZone>& zone_list, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<share::schema::ObZoneScore> new_zone_score_array;
 | 
						|
  const ObIArray<ObZoneScore>& 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<ObZoneScore> 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];
 | 
						|
    common::ObArray<ObZoneRegion> zone_region_list;
 | 
						|
    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<share::schema::ObZoneScore>& 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<int32_t>(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<common::ObRegion>& region_list, const common::ObIArray<common::ObZone>& 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<share::ObZoneInfo> 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<share::schema::ObZoneRegion>& zone_region_list, common::ObRegion& region)
 | 
						|
{
 | 
						|
  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<ObZoneRegion>& zone_region_list, const common::ObIArray<common::ObZone>& 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<share::ObZoneInfo> 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 (!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 <typename SCHEMA>
 | 
						|
int ObDDLService::set_schema_replica_num_options(
 | 
						|
    SCHEMA& schema, ObLocalityDistribution& locality_dist, ObIArray<ObUnitInfo>& unit_infos)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<share::ObZoneReplicaAttrSet> 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) {
 | 
						|
        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) {
 | 
						|
        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<ObString>& pool_strs, ObIArray<ObResourcePoolName>& 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;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::init_tenant_storage_format_version(ObTenantSchema& tenant_schema)
 | 
						|
{
 | 
						|
  // storage layer no long use storage format version and work version in tenant schema
 | 
						|
  UNUSED(tenant_schema);
 | 
						|
  return OB_SUCCESS;
 | 
						|
}
 | 
						|
 | 
						|
template <typename SCHEMA>
 | 
						|
int ObDDLService::extract_first_primary_zone_array(
 | 
						|
    const SCHEMA& schema, const ObIArray<common::ObZone>& zone_list, ObIArray<common::ObZone>& first_primary_zone_array)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (schema.get_primary_zone_array().count() <= 0) {
 | 
						|
    // bypass
 | 
						|
  } else {
 | 
						|
    const ObIArray<ObZoneScore>& 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<common::ObZone>& zone_list,
 | 
						|
    const ObIArray<share::schema::ObZoneRegion>& zone_region_list,
 | 
						|
    const ObIArray<common::ObZone>& first_primary_zone_array, ObIArray<common::ObRegion>& primary_regions,
 | 
						|
    ObIArray<common::ObZone>& 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;
 | 
						|
    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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } else if (schema_guard.get_table_schema(table_schema.get_data_table_id(), data_table_schema)) {
 | 
						|
      LOG_WARN("get table schema failed", K(ret));
 | 
						|
    } 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 <typename SCHEMA>
 | 
						|
int ObDDLService::check_primary_zone_locality_condition(const SCHEMA& schema, const ObIArray<common::ObZone>& zone_list,
 | 
						|
    const ObIArray<share::schema::ObZoneRegion>& zone_region_list, share::schema::ObSchemaGetterGuard& schema_guard)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObArray<ObZone> first_primary_zone_array;
 | 
						|
  ObArray<ObRegion> primary_regions;
 | 
						|
  ObArray<ObZone> zones_in_primary_regions;
 | 
						|
  ObArray<share::ObZoneReplicaAttrSet> 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<share::schema::ObZoneRegion>& zone_region_list,
 | 
						|
    const ObIArray<common::ObZone>& first_primary_zone_array, const ObIArray<common::ObZone>& zones_in_primary_regions,
 | 
						|
    const ObIArray<common::ObRegion>& primary_regions, const ObIArray<share::ObZoneReplicaAttrSet>& 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<common::ObZone>& zones_in_primary_regions,
 | 
						|
    const ObIArray<common::ObRegion>& primary_regions, const ObIArray<share::ObZoneReplicaAttrSet>& 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<share::schema::ObZoneRegion>& zone_region_list,
 | 
						|
    const ObIArray<share::ObZoneReplicaAttrSet>& 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<common::ObZone>& 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<common::ObZone>& first_primary_zone_array,
 | 
						|
    const ObIArray<share::ObZoneReplicaAttrSet>& zone_locality)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  // first primary zone cannot be assigned in mixed locality zones
 | 
						|
  ObArray<common::ObZone> independent_zones;
 | 
						|
  ObArray<int64_t> 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<common::ObZone>& 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 (is_new_tablegroup_id(tablegroup_id)) {
 | 
						|
    ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_);
 | 
						|
    if (OB_FAIL(set_locality_with_tablegroup(schema_guard, schema))) {
 | 
						|
      LOG_WARN("fail to set locality with tablegroup", K(ret));
 | 
						|
    } else if (OB_FAIL(check_create_table_replica_options(schema, schema_guard))) {
 | 
						|
      // format and check primary_zone
 | 
						|
      LOG_WARN("fail to check create table replica options", K(ret));
 | 
						|
    } else if (OB_FAIL(helper.process_tablegroup_option_for_create_table(schema_guard, schema))) {
 | 
						|
      LOG_WARN("fail to check if table and tablegroup match", K(ret));
 | 
						|
    } else 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 {
 | 
						|
    const ObTenantSchema* tenant_info = NULL;
 | 
						|
    const uint64_t tenant_id = schema.get_tenant_id();
 | 
						|
    const bool just_check_zone_list =
 | 
						|
        (share::ObDuplicateScope::DUPLICATE_SCOPE_CLUSTER == schema.get_duplicate_scope());
 | 
						|
    if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_info))) {
 | 
						|
      LOG_WARN("fail to get tenant info", K(ret), K(tenant_id));
 | 
						|
    } else if (OB_ISNULL(tenant_info)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("get invalid tenant_info", K(ret), K(tenant_id));
 | 
						|
    } else if (OB_FAIL(
 | 
						|
                   try_set_and_check_locality_with_tenant(just_check_zone_list, schema_guard, *tenant_info, schema))) {
 | 
						|
      LOG_WARN("fail to set and check locality with tenant", K(ret));
 | 
						|
    } else if (OB_FAIL(check_create_table_replica_options(schema, schema_guard))) {
 | 
						|
      // format and check primary_zone
 | 
						|
      LOG_WARN("fail to check create table replica options", K(ret));
 | 
						|
    } else if (OB_INVALID_ID == tablegroup_id) {
 | 
						|
      // For tables that are not in the tablegroup, the following checks are not required
 | 
						|
    } else if (check_table_schema_not_in_new_tablegroup(schema_guard, schema)) {
 | 
						|
      LOG_WARN("fail to check table schema which not in new tablegroup", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::split_partition(const ObTableSchema* schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSplitPartitionArg arg;
 | 
						|
  ObSplitProgress split_status;
 | 
						|
  ObPartitionSplitExecutor executor(*rpc_proxy_);
 | 
						|
  ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
  if (OB_ISNULL(schema) || !schema->is_in_splitting()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("schema is null", K(ret), K(schema));
 | 
						|
  } else if (OB_FAIL(executor.split_table_partition(*schema, *pt_operator_, *rpc_proxy_, arg, split_status))) {
 | 
						|
    LOG_WARN("failed to construct and send split arg", K(ret), K(arg), K(split_status));
 | 
						|
  } else if ((schema->is_in_logical_split() && LOGICAL_SPLIT_FINISH == split_status) ||
 | 
						|
             (schema->is_in_physical_split() && PHYSICAL_SPLIT_FINISH == split_status) ||
 | 
						|
             (schema->is_in_logical_split() && PHYSICAL_SPLIT_FINISH == split_status)) {
 | 
						|
    // Logically splitting, logic splitting completed
 | 
						|
    // Physically splitting, physical splitting completed
 | 
						|
    // In extreme cases, the physical split is completed quickly, and the result of no intermediate logical split
 | 
						|
    // is returned.
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans", K(ret));
 | 
						|
    } else if (OB_FAIL(ddl_operator.split_partition_finish(*schema, arg, split_status, trans))) {
 | 
						|
      LOG_WARN("fail to split partition finish", K(ret));
 | 
						|
    } else {
 | 
						|
      LOG_INFO("split partition logical or physical finished success", K(ret), K(split_status), K(arg));
 | 
						|
    }
 | 
						|
    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(ret), K(tmp_ret));
 | 
						|
        ret = (OB_SUCCESS == ret ? tmp_ret : ret);
 | 
						|
      }
 | 
						|
    }
 | 
						|
    const uint64_t tenant_id = schema->get_tenant_id();
 | 
						|
    // publish schema
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
        LOG_WARN("publish schema failed", K(ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    // NOTHING TODO
 | 
						|
    LOG_INFO("table split not finish, need wait", K(ret), K(arg));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::split_tablegroup_partition(
 | 
						|
    ObSchemaGetterGuard& schema_guard, const ObTablegroupSchema* tablegroup_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObPartitionSplitExecutor executor(*rpc_proxy_);
 | 
						|
  ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
  ObSplitProgress split_status;
 | 
						|
  if (OB_ISNULL(tablegroup_schema)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(tablegroup_schema));
 | 
						|
  } else if (!tablegroup_schema->is_in_splitting()) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret), K(tablegroup_schema));
 | 
						|
  } else if (tablegroup_schema->get_binding()) {
 | 
						|
    ObSplitPartitionArg arg;
 | 
						|
    if (OB_FAIL(executor.split_binding_tablegroup_partition(
 | 
						|
            *tablegroup_schema, *pt_operator_, *rpc_proxy_, arg, split_status))) {
 | 
						|
      LOG_WARN("failed to split partition", K(ret), K(arg), K(split_status), "tablegroup_schema", *tablegroup_schema);
 | 
						|
    }
 | 
						|
  } else if (OB_FAIL(
 | 
						|
                 executor.split_tablegroup_partition(schema_guard, *tablegroup_schema, *sql_proxy_, split_status))) {
 | 
						|
    LOG_WARN("fail to split tablegroup", K(ret), "tablegroup_schema", *tablegroup_schema);
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
#ifdef ERRSIM
 | 
						|
  } else if (OB_FAIL(E(EventTable::EN_BLOCK_SPLIT_PROGRESS_RESPONSE) OB_SUCCESS)) {
 | 
						|
    if (REACH_TIME_INTERVAL(1000000)) {
 | 
						|
      LOG_WARN("ERRSIM: EN_BLOCK_SPLIT_PROGRESS_RESPONSE", K(ret));
 | 
						|
    }
 | 
						|
#endif
 | 
						|
  } else if ((tablegroup_schema->is_in_logical_split() && LOGICAL_SPLIT_FINISH == split_status) ||
 | 
						|
             (tablegroup_schema->is_in_physical_split() && PHYSICAL_SPLIT_FINISH == split_status) ||
 | 
						|
             (tablegroup_schema->is_in_logical_split() && PHYSICAL_SPLIT_FINISH == split_status)) {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("failed to start trans", K(ret));
 | 
						|
    } else if (OB_FAIL(ddl_operator.split_tablegroup_partition_finish(*tablegroup_schema, split_status, trans))) {
 | 
						|
      LOG_WARN("fail to split partition", K(ret));
 | 
						|
    }
 | 
						|
    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(ret), K(tmp_ret));
 | 
						|
        ret = (OB_SUCCESS == ret ? tmp_ret : ret);
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      LOG_INFO("split partition finished success", K(ret));
 | 
						|
      const uint64_t tenant_id = tablegroup_schema->get_tenant_id();
 | 
						|
      // publish schema
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        ret = publish_schema(tenant_id);
 | 
						|
        if (OB_FAIL(ret)) {
 | 
						|
          LOG_WARN("publish schema failed", K(ret), K(tenant_id));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else {
 | 
						|
    LOG_INFO("tablegroup split not finish, need wait", K(ret), K(tablegroup_schema));
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::finish_schema_split(int64_t& split_schema_version)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const uint64_t tenant_id = OB_SYS_TENANT_ID;
 | 
						|
  split_schema_version = OB_INVALID_VERSION;
 | 
						|
  ObArray<uint64_t> tenant_ids;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("inner stat error", K(ret));
 | 
						|
  } else if (OB_ISNULL(schema_service_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema_service is 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 tenant schema guard", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_tenant_ids(tenant_ids))) {
 | 
						|
    LOG_WARN("fail to get tenant ids", K(ret));
 | 
						|
  } else {
 | 
						|
    int64_t new_schema_version = OB_INVALID_VERSION;
 | 
						|
    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));
 | 
						|
    } 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 {
 | 
						|
      share::schema::ObDDLSqlService ddl_sql_service(*schema_service_impl);
 | 
						|
      ObDDLSQLTransaction trans(schema_service_);
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("failed to start trans", K(ret));
 | 
						|
      }
 | 
						|
      // In order to avoid liboblog tenant-level schema_version rollback, the schema_version of each tenant
 | 
						|
      // needs to be pushed up after the schema split ends.
 | 
						|
      // At present, the transaction layer also supports cross-tenant write transactions
 | 
						|
      // (cross-tenant read transactions are not supported), and transaction features can be used to
 | 
						|
      // reduce exception handling during upgrade
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < tenant_ids.count(); i++) {
 | 
						|
        if (OB_FAIL(ddl_sql_service.finish_schema_split(trans, tenant_ids.at(i), new_schema_version))) {
 | 
						|
          LOG_WARN("log schema split finish ddl operation failed",
 | 
						|
              K(ret),
 | 
						|
              "tenant_id",
 | 
						|
              tenant_ids.at(i),
 | 
						|
              K(new_schema_version));
 | 
						|
        }
 | 
						|
      }
 | 
						|
      if (OB_SUCC(ret)) {
 | 
						|
        split_schema_version = 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", K(ret), K(tmp_ret));
 | 
						|
          ret = (OB_SUCCESS == ret ? tmp_ret : ret);
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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.
 | 
						|
 *   - arg.primary_schema_versions_ : It's only used to gen new schema version when broadcast sys tenant's
 | 
						|
 * schema_version in standby cluster.
 | 
						|
 */
 | 
						|
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_SYS_TENANT_ID != tenant_id && arg.primary_schema_versions_.count() > 0) {
 | 
						|
      ret = OB_INVALID_ARGUMENT;
 | 
						|
      LOG_WARN("primary_schema_versions should be empty when tenant is non-sys", 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 tenant schema guard", KR(ret), K(tenant_id));
 | 
						|
    } else {
 | 
						|
      share::schema::ObDDLSqlService ddl_sql_service(*tmp_schema_service);
 | 
						|
      trans.set_end_tenant_id(tenant_id);
 | 
						|
      if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
        LOG_WARN("failed to start trans", KR(ret));
 | 
						|
      } else if (1 == arg.primary_schema_versions_.count()) {
 | 
						|
        // can't generate new schema_version in standby cluster, just commit
 | 
						|
      } 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::process_create_partition_failed(const int64_t tenant_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("inner stat error", K(ret));
 | 
						|
  } else if (OB_INVALID_ID == tenant_id) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("tenant id is invalid", K(ret), K(tenant_id));
 | 
						|
  } else {
 | 
						|
    // No need to refresh the schema, just generate a new schema version and write a transaction end boundary
 | 
						|
    share::schema::ObSchemaService* tmp_schema_service = schema_service_->get_schema_service();
 | 
						|
    if (OB_ISNULL(tmp_schema_service)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("schema service is null", K(ret), KP(tmp_schema_service), KP(schema_service_));
 | 
						|
    } else if (is_sync_primary_ddl()) {
 | 
						|
      // The standalone cluster synchronizes the ddl of the primary cluster, using the schema version of the primary
 | 
						|
      // cluster. If it fails, it will try again. There is no need to push the schema version.
 | 
						|
    } else {
 | 
						|
      int64_t new_schema_version = OB_INVALID_VERSION;
 | 
						|
      share::schema::ObDDLSqlService ddl_sql_service(*tmp_schema_service);
 | 
						|
      share::schema::ObSchemaOperation operation;
 | 
						|
      operation.op_type_ = share::schema::OB_DDL_END_SIGN;
 | 
						|
      operation.tenant_id_ = tenant_id;
 | 
						|
      ObString str;
 | 
						|
      // Reset the timeout period to prevent the last schema from not being pushed up due to SQL timeout
 | 
						|
      const int64_t query_timeout = GCONF.rpc_timeout;
 | 
						|
      const int64_t abs_timeout = query_timeout + ObTimeUtility::current_time();
 | 
						|
      ObTimeoutCtx timeout_ctx;
 | 
						|
      if (OB_FAIL(timeout_ctx.set_trx_timeout_us(query_timeout))) {
 | 
						|
        LOG_WARN("failed to set trans timeout", KR(ret), K(query_timeout));
 | 
						|
      } else if (OB_FAIL(timeout_ctx.set_abs_timeout(abs_timeout))) {
 | 
						|
        LOG_WARN("failed to set stmt timeout", KR(ret), K(abs_timeout));
 | 
						|
      } 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(operation, new_schema_version, str, *sql_proxy_))) {
 | 
						|
        LOG_WARN("log non ddl operation failed", KR(ret), K(operation), K(new_schema_version));
 | 
						|
      } else if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
        LOG_WARN("publish schema failed", KR(ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::try_compensate_readonly_all_server(share::schema::ObSchemaGetterGuard& schema_guard,
 | 
						|
    const share::schema::ObPartitionSchema& schema, share::schema::ZoneLocalityIArray& zone_locality)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  UNUSED(schema_guard);
 | 
						|
  bool compensate_readonly_all_server =
 | 
						|
      (ObDuplicateScope::DUPLICATE_SCOPE_CLUSTER == schema.get_duplicate_scope() && schema.get_locality_str().empty());
 | 
						|
  if (compensate_readonly_all_server) {
 | 
						|
    common::ObArray<share::ReplicaAttr> readonly_set;
 | 
						|
    if (OB_FAIL(readonly_set.push_back(ReplicaAttr(ObLocalityDistribution::ALL_SERVER_CNT, 100 /*percent*/)))) {
 | 
						|
      LOG_WARN("fail to push back", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < zone_locality.count(); ++i) {
 | 
						|
        share::ObZoneReplicaAttrSet& locality_set = zone_locality.at(i);
 | 
						|
        if (OB_FAIL(locality_set.replica_attr_set_.set_readonly_replica_attr_array(readonly_set))) {
 | 
						|
          LOG_WARN("fail to set readonly replica attr array", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// After 2.0, locality no longer supports the form of @region, so only @zone is considered here.
 | 
						|
// The assumption here is that paxos members have no @region form
 | 
						|
template <typename Schema>
 | 
						|
int ObDDLService::calc_schema_replica_num(ObSchemaGetterGuard& schema_guard, const Schema& schema,
 | 
						|
    int64_t& paxos_replica_num, int64_t& non_paxos_replica_num, const ObCreateTableMode create_mode)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<share::ObZoneReplicaNumSet> zone_locality;
 | 
						|
  paxos_replica_num = 0;
 | 
						|
  non_paxos_replica_num = 0;
 | 
						|
  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(try_compensate_readonly_all_server(schema_guard, schema, zone_locality))) {
 | 
						|
    LOG_WARN("fail to try compensate readonly all server", K(ret));
 | 
						|
  } else {
 | 
						|
    FOREACH_CNT_X(zone_loc, zone_locality, OB_SUCCESS == ret)
 | 
						|
    {
 | 
						|
      if (OB_ISNULL(zone_loc)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("get invalid locality set", K(ret), KP(zone_loc));
 | 
						|
      } else {
 | 
						|
        paxos_replica_num += zone_loc->get_paxos_replica_num();
 | 
						|
        int64_t zone_paxos_replica_num = zone_loc->get_paxos_replica_num();
 | 
						|
        int64_t readonly_replica_num = zone_loc->get_readonly_replica_num();
 | 
						|
        if (ObLocalityDistribution::ALL_SERVER_CNT == readonly_replica_num) {
 | 
						|
          const uint64_t tenant_id = schema.get_tenant_id();
 | 
						|
          ObArray<ObUnitInfo> unit_infos;
 | 
						|
          if (OB_FAIL(unit_mgr_->get_zone_alive_unit_infos_by_tenant(tenant_id, zone_loc->zone_, unit_infos))) {
 | 
						|
            LOG_WARN("fail to get zone alive unit infos by tenant", K(ret), K(tenant_id), "zone", zone_loc->zone_);
 | 
						|
          } else if (unit_infos.count() < zone_paxos_replica_num) {
 | 
						|
            if (OB_CREATE_TABLE_MODE_STRICT == create_mode) {
 | 
						|
              ret = OB_OP_NOT_ALLOW;
 | 
						|
              LOG_WARN("pool unit num is not enough for locality",
 | 
						|
                  K(ret),
 | 
						|
                  K(zone_loc->zone_),
 | 
						|
                  K(unit_infos.count()),
 | 
						|
                  K(zone_paxos_replica_num));
 | 
						|
            }
 | 
						|
          } else {
 | 
						|
            non_paxos_replica_num += (unit_infos.count() - zone_paxos_replica_num);
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          non_paxos_replica_num += readonly_replica_num;
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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;
 | 
						|
    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(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } 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_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::sync_readonly_sys_param(
 | 
						|
    share::schema::ObSysVariableSchema &new_sys_variable,
 | 
						|
    share::schema::ObTenantSchema &new_tenant_schema,
 | 
						|
    const share::schema::ObTenantSchema &ori_tenant_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  bool read_only_value =  0;
 | 
						|
  const ObSysVarSchema *sysvar_schema = nullptr;
 | 
						|
  if (OB_FAIL(new_sys_variable.get_sysvar_schema(SYS_VAR_READ_ONLY, sysvar_schema))) {
 | 
						|
    if (OB_ERR_SYS_VARIABLE_UNKNOWN == ret) {
 | 
						|
      // Currently, sysvar_schemas in new_sys_variable only saves the sysvar_schema corresponding to the set system
 | 
						|
variable
 | 
						|
      // So if the set object is not READ_ONLY, get_sysvar_schema returns OB_ERR_SYS_VARIABLE_UNKNOWN
 | 
						|
      // So here reset to OB_SUCCESS
 | 
						|
      ret = OB_SUCCESS;
 | 
						|
      // Before the schema is split, read_only is temporarily synchronized. If the system variable is not set,
 | 
						|
      // the tenant option shall prevail
 | 
						|
      ObSysParam sys_param;
 | 
						|
      ObSysVarSchema sysvar_schema;
 | 
						|
      if (new_tenant_schema.is_read_only() != ori_tenant_schema.is_read_only()) {
 | 
						|
        ObString read_only_value = new_tenant_schema.is_read_only() ? "1" : "0";
 | 
						|
        int64_t store_idx = OB_INVALID_ID;
 | 
						|
        if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(SYS_VAR_READ_ONLY, store_idx))) {
 | 
						|
          LOG_WARN("failed to calc sys var store idx", K(ret));
 | 
						|
        } 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(store_idx), K(ret));
 | 
						|
        } else if (OB_FAIL(sys_param.init(new_sys_variable.get_tenant_id(),
 | 
						|
                                          "",
 | 
						|
                                          ObSysVariables::get_name(store_idx),
 | 
						|
                                          ObSysVariables::get_type(store_idx),
 | 
						|
                                          read_only_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 sys variable", K(ret));
 | 
						|
        } else if (OB_FAIL(ObSchemaUtils::convert_sys_param_to_sysvar_schema(sys_param, sysvar_schema))) {
 | 
						|
          LOG_WARN("convert to sysvar schema failed", K(ret));
 | 
						|
        } else if (OB_FAIL(new_sys_variable.add_sysvar_schema(sysvar_schema))) {
 | 
						|
          LOG_WARN("add system variable failed", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    } else {
 | 
						|
      LOG_WARN("get read_only sysvar schema failed", K(ret));
 | 
						|
    }
 | 
						|
  } else if (nullptr == sysvar_schema) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("read_only sysvar schema is null", K(ret));
 | 
						|
  } else {
 | 
						|
    read_only_value = static_cast<bool> ((sysvar_schema->get_value())[0] - '0');
 | 
						|
    new_tenant_schema.set_read_only(read_only_value);
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
*/
 | 
						|
 | 
						|
/**
 | 
						|
 * @note: All OB-specific non-mysql system variables begin with ob_ to distinguish them from
 | 
						|
 *  the namespace of mysql system variables. On the contrary, all system variables that do not start with ob_ must be
 | 
						|
 *  fully compatible with mysql semantics.
 | 
						|
 */
 | 
						|
// The value of certain system variables of the system tenant
 | 
						|
int ObDDLService::update_sys_tenant_sys_var(
 | 
						|
    const ObSysVariableSchema& sys_variable, ObSysParam* sys_params, int64_t params_capacity)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(sys_params) || OB_UNLIKELY(params_capacity < ObSysVarFactory::ALL_SYS_VARS_COUNT)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid arguments", K(sys_params), K(params_capacity), K(ret));
 | 
						|
  } else if (OB_SYS_TENANT_ID == sys_variable.get_tenant_id()) {
 | 
						|
    ObString lower_case_table_name(OB_SV_LOWER_CASE_TABLE_NAMES);
 | 
						|
    ObString tcp_invited_nodes(OB_SV_TCP_INVITED_NODES);
 | 
						|
    ObString timestamp_service(OB_SV_TIMESTAMP_SERVICE);
 | 
						|
    int64_t lower_case_table_idx = OB_INVALID_INDEX;
 | 
						|
    int64_t tcp_invited_nodes_idx = OB_INVALID_INDEX;
 | 
						|
    int64_t timestamp_service_idx = OB_INVALID_INDEX;
 | 
						|
    if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx_by_name(lower_case_table_name, lower_case_table_idx))) {
 | 
						|
      LOG_WARN("failed to calc_sys_var_store_idx_by_name", K(sys_variable), K(lower_case_table_name), K(ret));
 | 
						|
    } else if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx_by_name(tcp_invited_nodes, tcp_invited_nodes_idx))) {
 | 
						|
      LOG_WARN("failed to calc_sys_var_store_idx_by_name", K(sys_variable), K(tcp_invited_nodes), K(ret));
 | 
						|
    } else if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx_by_name(timestamp_service, timestamp_service_idx))) {
 | 
						|
      LOG_WARN("failed to calc_sys_var_store_idx_by_name", K(sys_variable), K(timestamp_service), K(ret));
 | 
						|
    } else if (OB_UNLIKELY(lower_case_table_idx < 0) ||
 | 
						|
               OB_UNLIKELY(lower_case_table_idx >= ObSysVarFactory::ALL_SYS_VARS_COUNT) ||
 | 
						|
               OB_UNLIKELY(timestamp_service_idx < 0) ||
 | 
						|
               OB_UNLIKELY(timestamp_service_idx >= ObSysVarFactory::ALL_SYS_VARS_COUNT) ||
 | 
						|
               OB_UNLIKELY(tcp_invited_nodes_idx < 0) ||
 | 
						|
               OB_UNLIKELY(tcp_invited_nodes_idx >= ObSysVarFactory::ALL_SYS_VARS_COUNT)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("got store_idx is invalid",
 | 
						|
          K(lower_case_table_idx),
 | 
						|
          K(tcp_invited_nodes_idx),
 | 
						|
          K(timestamp_service_idx),
 | 
						|
          K(ret));
 | 
						|
    } else {
 | 
						|
      ObString lts_int_value("0");
 | 
						|
      if (OB_UNLIKELY(
 | 
						|
              0 >
 | 
						|
              snprintf(
 | 
						|
                  sys_params[tcp_invited_nodes_idx].value_, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s", OB_SYS_HOST_NAME))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("failed to update value of ob_tcp_invited_nodes", K(sys_variable), K(ret));
 | 
						|
      } else if (OB_UNLIKELY(0 > snprintf(sys_params[lower_case_table_idx].value_,
 | 
						|
                                     OB_MAX_SYS_PARAM_VALUE_LENGTH,
 | 
						|
                                     "%d",
 | 
						|
                                     sys_variable.get_name_case_mode()))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("failed to update value of lower_case_table_names", K(sys_variable), K(ret));
 | 
						|
      } else if (OB_UNLIKELY(0 > snprintf(sys_params[timestamp_service_idx].value_,
 | 
						|
                                     OB_MAX_SYS_PARAM_VALUE_LENGTH,
 | 
						|
                                     "%s",
 | 
						|
                                     lts_int_value.ptr()))) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("failed to update value of ob_timestamp_service", K(sys_variable), K(ret));
 | 
						|
      } else {
 | 
						|
        // do nothing
 | 
						|
      }
 | 
						|
    }
 | 
						|
  } else { /*do nothing*/
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::init_system_variables(const ObCreateTenantArg& arg, ObTenantSchema& tenant_schema,
 | 
						|
    const ObSysVariableSchema& sys_variable_schema, ObSysParam* sys_params, int64_t params_capacity)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  int64_t var_amount = ObSysVariables::get_amount();
 | 
						|
  if (OB_ISNULL(sys_params) || OB_UNLIKELY(var_amount > params_capacity)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid arguments", K(sys_params), K(params_capacity), K(var_amount), K(ret));
 | 
						|
  } else {
 | 
						|
    // just init default values
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < var_amount; ++i) {
 | 
						|
      ret = sys_params[i].init(sys_variable_schema.get_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));
 | 
						|
      if (OB_FAIL(ret)) {
 | 
						|
        LOG_WARN("fail to init param, ", K(sys_variable_schema), K(i), K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    int64_t set_sys_var_count = arg.sys_var_list_.count();
 | 
						|
    ObSysVarIdValue tmp_sys_var;  // for init
 | 
						|
    bool modify_read_only = false;
 | 
						|
    bool is_oracle_tenant = false;
 | 
						|
    bool use_default_parallel_max_servers = true;
 | 
						|
    bool use_default_parallel_servers_target = true;
 | 
						|
 | 
						|
    for (int64_t j = 0; OB_SUCC(ret) && j < set_sys_var_count; ++j) {
 | 
						|
      int64_t store_idx = OB_INVALID_INDEX;
 | 
						|
      ObSysVarIdValue& sys_var = tmp_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 if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(sys_var.sys_id_, store_idx))) {
 | 
						|
        LOG_WARN("failed to calc sys var store idx", K(sys_var), K(ret));
 | 
						|
      } 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(store_idx), K(ret));
 | 
						|
      } else {
 | 
						|
        if (SYS_VAR_READ_ONLY == sys_var.sys_id_) {
 | 
						|
          modify_read_only = true;
 | 
						|
        } else if (SYS_VAR_PARALLEL_MAX_SERVERS == sys_var.sys_id_) {
 | 
						|
          use_default_parallel_max_servers = false;
 | 
						|
        } else if (SYS_VAR_PARALLEL_SERVERS_TARGET == sys_var.sys_id_) {
 | 
						|
          use_default_parallel_servers_target = false;
 | 
						|
        }
 | 
						|
        // set tenant schema compatibility mode
 | 
						|
        if (0 == ObSysVariables::get_name(store_idx).compare("ob_compatibility_mode") &&
 | 
						|
            0 == sys_var.value_.compare("1")) {
 | 
						|
          is_oracle_tenant = true;
 | 
						|
          tenant_schema.set_compatibility_mode(ObCompatibilityMode::ORACLE_MODE);
 | 
						|
          LOG_DEBUG("is oracle tenant", K(ret), K(sys_var), K(tenant_schema));
 | 
						|
        }
 | 
						|
        ObString& new_value = sys_var.value_;
 | 
						|
        if (OB_FAIL(sys_params[store_idx].init(sys_variable_schema.get_tenant_id(),
 | 
						|
                "",
 | 
						|
                ObSysVariables::get_name(store_idx),
 | 
						|
                ObSysVariables::get_type(store_idx),
 | 
						|
                new_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 sys variable", K(arg), K(j), K(sys_var), K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }  // end of for
 | 
						|
 | 
						|
    // The mysql tenant is created, and the server-level charset and collation need to be set in mysql mode
 | 
						|
    if (OB_SUCC(ret) && !is_oracle_tenant) {
 | 
						|
      // If it is a tenant in mysql mode, you need to consider setting the charset and collation
 | 
						|
      // corresponding to the tenant to sys var
 | 
						|
      char val_buf[OB_MAX_SYS_PARAM_VALUE_LENGTH];
 | 
						|
      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);
 | 
						|
    }
 | 
						|
    // 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
 | 
						|
    // SET_TENANT_VARIABLE(ID,VAL),value must be string
 | 
						|
    // If you don't want to write hard code, use VAR_INT_TO_STRING to write int to buf
 | 
						|
    if (OB_SUCC(ret) && is_oracle_tenant) {
 | 
						|
      char val_buf[OB_MAX_SYS_PARAM_VALUE_LENGTH];
 | 
						|
      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()) {
 | 
						|
        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"));
 | 
						|
        }
 | 
						|
        if (OB_SUCC(ret)) {
 | 
						|
          int64_t store_idx = OB_INVALID_ID;
 | 
						|
          if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(SYS_VAR_NLS_CHARACTERSET, store_idx))) {
 | 
						|
            LOG_WARN("failed to calc sys var store idx", K(ret));
 | 
						|
          } 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(store_idx), K(ret));
 | 
						|
          } 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),
 | 
						|
                         val_buf,
 | 
						|
                         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 sys variable", K(ret));
 | 
						|
          }
 | 
						|
        }
 | 
						|
      } 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));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // When creating a tenant, read_only is subject to system variables, if sys variables is not set,
 | 
						|
    // subject to tenant option
 | 
						|
    if (OB_SUCC(ret) && !modify_read_only) {
 | 
						|
      ObString read_only_value = tenant_schema.is_read_only() ? "1" : "0";
 | 
						|
      int64_t store_idx = OB_INVALID_ID;
 | 
						|
      if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(SYS_VAR_READ_ONLY, store_idx))) {
 | 
						|
        LOG_WARN("failed to calc sys var store idx", K(ret));
 | 
						|
      } 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(store_idx), K(ret));
 | 
						|
      } 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),
 | 
						|
                     read_only_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 sys variable", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
 | 
						|
    // If the user does not specify parallel_servers_target, parallel_max_servers 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 || use_default_parallel_max_servers)) {
 | 
						|
      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_MAX_SYS_CPU
 | 
						|
        default_px_thread_count = ObTenantCpuShare::calc_px_pool_share(
 | 
						|
            sys_variable_schema.get_tenant_id(), static_cast<int64_t>(DEFAULT_MAX_SYS_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<int64_t>(unit_config.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<int64_t>(default_px_thread_count * 0.8));
 | 
						|
      char buf[32];
 | 
						|
      snprintf(buf, 32, "%ld", default_px_servers_target);
 | 
						|
      ObString value(buf);
 | 
						|
      int64_t store_idx = OB_INVALID_ID;
 | 
						|
      if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(SYS_VAR_PARALLEL_SERVERS_TARGET, store_idx))) {
 | 
						|
        LOG_WARN("failed to calc sys var store idx", K(ret));
 | 
						|
      } 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(store_idx), K(ret));
 | 
						|
      } 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 sys variable", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret) && use_default_parallel_max_servers && default_px_thread_count > 0) {
 | 
						|
      char buf[32];
 | 
						|
      snprintf(buf, 32, "%ld", default_px_thread_count);
 | 
						|
      ObString value(buf);
 | 
						|
      int64_t store_idx = OB_INVALID_ID;
 | 
						|
      if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(SYS_VAR_PARALLEL_MAX_SERVERS, store_idx))) {
 | 
						|
        LOG_WARN("failed to calc sys var store idx", K(ret));
 | 
						|
      } 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(store_idx), K(ret));
 | 
						|
      } 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 sys variable", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      char buf[32];
 | 
						|
      snprintf(buf, 32, "%ld", 0L /* primary zone entity count zero*/);
 | 
						|
      ObString value(buf);
 | 
						|
      int64_t store_idx = OB_INVALID_ID;
 | 
						|
      if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(SYS_VAR__PRIMARY_ZONE_ENTITY_COUNT, store_idx))) {
 | 
						|
        LOG_WARN("failed to calc sys var store idx", K(ret));
 | 
						|
      } 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 sys variable", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // set special values for sys tenant
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(update_sys_tenant_sys_var(sys_variable_schema, sys_params, params_capacity))) {
 | 
						|
        LOG_WARN("failed to update_sys_tenant_sys_var", K(sys_variable_schema), K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLSQLTransaction::start(ObISQLClient* proxy, bool with_snapshot /*= false*/)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("schema service is null", K(ret));
 | 
						|
  } else {
 | 
						|
    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, with_snapshot))) {
 | 
						|
      LOG_WARN("fail to start trans", K(ret));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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) {
 | 
						|
    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() &&
 | 
						|
               !schema_service_impl->is_sync_primary_ddl()) {
 | 
						|
      // If it is a ddl synchronized from the primary cluster, even if there is only one transaction boundary,
 | 
						|
      // it is necessary to write down the transaction boundary
 | 
						|
      LOG_INFO("ddl operation is same, just skip",
 | 
						|
          K(ret),
 | 
						|
          K_(start_operation_schema_version),
 | 
						|
          K_(start_operation_tenant_id));
 | 
						|
    } else {
 | 
						|
      // If you need to commit this transaction, first write a ddl end tag
 | 
						|
      // push schema version
 | 
						|
      // If there is no defined tenant_id, the previous tenant_id will be used directly by default
 | 
						|
      // Special handling of tenant id is needed when creating tenant transaction one
 | 
						|
      const uint64_t tenant_id =
 | 
						|
          OB_INVALID_ID != tenant_id_ ? tenant_id_ : schema_service_impl->get_last_operation_tenant_id();
 | 
						|
      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", K(ret), K(tenant_id));
 | 
						|
      } 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));
 | 
						|
      } else if (OB_FAIL(try_lock_all_ddl_operation(*this, tenant_id))) {
 | 
						|
        LOG_WARN("fail to lock all ddl operation", K(ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
}
 | 
						|
 | 
						|
// 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.
 | 
						|
//
 | 
						|
// In order to avoid the re-release of liboblog (the schema_version reversal of __all_ddl_operation clog caused by
 | 
						|
// 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::try_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())) {
 | 
						|
        if (OB_ITER_END == ret) {
 | 
						|
          ret = OB_SUCCESS;
 | 
						|
          LOG_INFO("maybe bootstrap or create tenant, just ignore", KR(ret), K(tenant_id));
 | 
						|
        } else {
 | 
						|
          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::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();
 | 
						|
  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 {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    }
 | 
						|
 | 
						|
    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<uint64_t, 64> user_ids_with_profile;
 | 
						|
      if (OB_FAIL(get_all_users_in_tenant_with_profile(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::statistic_primary_zone_entity_count()
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  const bool enable_ddl = GCONF.enable_ddl;
 | 
						|
  const bool enable_sys_table_ddl = GCONF.enable_sys_table_ddl;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  ObArray<uint64_t> tenant_ids;
 | 
						|
  int64_t job_id = OB_INVALID_ID;
 | 
						|
  ObRsJobType job_type = ObRsJobType::JOB_TYPE_STATISTIC_PRIMARY_ZONE_ENTITY_COUNT;
 | 
						|
  SpinWLockGuard guard(pz_entity_cnt_lock_);  // lock
 | 
						|
  ObRsJobInfo job_info;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", K(ret));
 | 
						|
  } else if (enable_ddl || enable_sys_table_ddl) {
 | 
						|
    ret = OB_OP_NOT_ALLOW;
 | 
						|
    LOG_WARN("cannot statistic primary zone entity when ddl enable", K(ret));
 | 
						|
  } else if (nullptr == sql_proxy_) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("sql_proxy_ is null", K(ret));
 | 
						|
  } else if (OB_FAIL(RS_JOB_FIND(job_info,
 | 
						|
                 *sql_proxy_,
 | 
						|
                 "job_type",
 | 
						|
                 "STATISTIC_PRIMARY_ZONE_ENTITY_COUNT",
 | 
						|
                 "job_status",
 | 
						|
                 "INPROGRESS"))) {
 | 
						|
    if (OB_ENTRY_NOT_EXIST == ret) {
 | 
						|
      ret = OB_SUCCESS;
 | 
						|
    } else {
 | 
						|
      LOG_WARN("fail to find job", K(ret));
 | 
						|
    }
 | 
						|
  } else if (job_info.job_id_ <= 0) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("find a job id unexpected", K(ret));
 | 
						|
  } else if (OB_FAIL(RS_JOB_COMPLETE(job_info.job_id_, -1, *sql_proxy_))) {
 | 
						|
    LOG_WARN("fail to complete job", K(ret));
 | 
						|
  }
 | 
						|
 | 
						|
  if (OB_SUCC(ret)) {
 | 
						|
    if (OB_FAIL(RS_JOB_CREATE_WITH_RET(job_id, job_type, *sql_proxy_, "tenant_id", 0))) {
 | 
						|
      LOG_WARN("fail to create rs job", 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 tenant schema guard with version in inner table", K(ret));
 | 
						|
    } else if (OB_FAIL(schema_guard.get_tenant_ids(tenant_ids))) {
 | 
						|
      LOG_WARN("fail to get tenant ids", K(ret));
 | 
						|
    } else {
 | 
						|
      for (int64_t i = 0; OB_SUCC(ret) && i < tenant_ids.count(); ++i) {
 | 
						|
        const uint64_t tenant_id = tenant_ids.at(i);
 | 
						|
        if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("err unexpected", K(ret), K(tenant_id));
 | 
						|
        } else if (OB_FAIL(statistic_tenant_primary_zone_entity_count(tenant_id))) {
 | 
						|
          LOG_WARN("fail to statistic tenant primary zone entity count", K(ret));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    }
 | 
						|
    int tmp_ret = OB_SUCCESS;
 | 
						|
    if (job_id > 0) {
 | 
						|
      if (OB_SUCCESS != (tmp_ret = RS_JOB_COMPLETE(job_id, ret, *sql_proxy_))) {
 | 
						|
        LOG_ERROR("fail to complete job", K(tmp_ret), K(ret), K(job_id));
 | 
						|
        ret = (OB_FAIL(ret)) ? ret : tmp_ret;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::upgrade_cluster_create_tenant_ha_gts_util(const uint64_t tenant_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::schema::ObSchemaGetterGuard schema_guard;
 | 
						|
  const share::schema::ObTenantSchema* tenant_schema = nullptr;
 | 
						|
  common::ObRegion primary_region;
 | 
						|
  common::ObZone first_primary_zone;
 | 
						|
  ObRootService* root_service = GCTX.root_service_;
 | 
						|
  bool ha_gts_exist = false;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("check inner stat failed", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == zone_mgr_ || nullptr == root_service)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("zone mgr ptr is null", K(ret));
 | 
						|
  } else if (OB_FAIL(root_service->get_rs_gts_manager().check_tenant_ha_gts_exist(tenant_id, ha_gts_exist))) {
 | 
						|
    LOG_WARN("fail to check tenant ha gts exist", K(ret), K(tenant_id));
 | 
						|
  } else if (ha_gts_exist) {
 | 
						|
    // ha ts exist, bypass
 | 
						|
  } else if (OB_UNLIKELY(nullptr == schema_service_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema service ptr is null", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(OB_SYS_TENANT_ID, schema_guard))) {
 | 
						|
    LOG_WARN("fail to get tenant schema guard", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
 | 
						|
    LOG_WARN("fail to get tenant info", K(ret), K(tenant_id));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == tenant_schema)) {
 | 
						|
    ret = OB_TENANT_NOT_EXIST;
 | 
						|
    LOG_WARN("tenant schema not exist", K(ret));
 | 
						|
  } else {
 | 
						|
    if (tenant_schema->get_primary_zone_array().count() <= 0) {
 | 
						|
      primary_region.reset();  // there is not primary region
 | 
						|
    } else if (FALSE_IT(first_primary_zone = tenant_schema->get_primary_zone_array().at(0).zone_)) {
 | 
						|
      // share never be here
 | 
						|
    } else if (OB_FAIL(zone_mgr_->get_region(first_primary_zone, primary_region))) {
 | 
						|
      LOG_WARN("fail to get region", K(first_primary_zone));
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(root_service->get_rs_gts_manager().alloc_gts_instance_for_tenant(tenant_id, primary_region))) {
 | 
						|
        LOG_WARN("fail to alloc gts instance for tenant", K(ret), K(tenant_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::upgrade_cluster_create_ha_gts_util()
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::schema::ObSchemaGetterGuard sys_schema_guard;
 | 
						|
  common::ObArray<uint64_t> tenant_ids;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("check inner stat failed", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == schema_service_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema service ptr is null", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(OB_SYS_TENANT_ID, sys_schema_guard))) {
 | 
						|
    LOG_WARN("fail to get tenant schema guard", K(ret));
 | 
						|
  } else if (OB_FAIL(sys_schema_guard.get_tenant_ids(tenant_ids))) {
 | 
						|
    LOG_WARN("fail to get tenant ids", K(ret));
 | 
						|
  } else {
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < tenant_ids.count(); ++i) {
 | 
						|
      const uint64_t tenant_id = tenant_ids.at(i);
 | 
						|
      if (OB_SYS_TENANT_ID == tenant_id || OB_GTS_TENANT_ID == tenant_id) {
 | 
						|
        // bypass
 | 
						|
      } else if (OB_FAIL(upgrade_cluster_create_tenant_ha_gts_util(tenant_id))) {
 | 
						|
        LOG_WARN("fail to update cluster create tenant ha gts util", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::statistic_tenant_primary_zone_entity_count(const uint64_t tenant_id)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  share::schema::ObSchemaGetterGuard schema_guard;
 | 
						|
  int64_t pz_entity_count = 0;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("variable is not init", KR(ret));
 | 
						|
  } else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", KR(ret), K(tenant_id));
 | 
						|
  } else if (OB_UNLIKELY(nullptr == schema_service_ || nullptr == sql_proxy_)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("schema_service or sql_proxy is null", KR(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 with version in inner table", KR(ret), K(tenant_id));
 | 
						|
  } else if (OB_FAIL(get_tenant_primary_zone_entity_count(tenant_id, schema_guard, pz_entity_count))) {
 | 
						|
    LOG_WARN("fail to get tenant primary zone entity", KR(ret), K(tenant_id));
 | 
						|
  } else {
 | 
						|
    const common::ObString pz_var_name(share::OB_SV__PRIMARY_ZONE_ENTITY_COUNT);
 | 
						|
    const int64_t MAX_BUF_SIZE = 64;
 | 
						|
    char val_buf[MAX_BUF_SIZE] = "";
 | 
						|
    obrpc::ObModifySysVarArg arg;
 | 
						|
    arg.tenant_id_ = tenant_id;
 | 
						|
    arg.exec_tenant_id_ = tenant_id;
 | 
						|
    ObSysVarSchema sysvar_schema;
 | 
						|
    const ObSysVarSchema* orig_sys_var = NULL;
 | 
						|
    if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id, pz_var_name, orig_sys_var))) {
 | 
						|
      LOG_WARN("fail to get sys var schema", KR(ret), K(tenant_id), K(pz_var_name));
 | 
						|
    } else if (OB_ISNULL(orig_sys_var)) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("sys var schema is null", KR(ret), K(tenant_id), K(pz_var_name));
 | 
						|
    } else if (OB_FAIL(sysvar_schema.assign(*orig_sys_var))) {
 | 
						|
      LOG_WARN("fail to assign sys var schema", KR(ret), K(tenant_id), KPC(orig_sys_var));
 | 
						|
    } else if (0 > (snprintf(val_buf, MAX_BUF_SIZE, "%ld", pz_entity_count))) {
 | 
						|
      ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("fail to print data to buf", KR(ret), K(tenant_id), K(pz_entity_count));
 | 
						|
    } else if (OB_FAIL(sysvar_schema.set_value(ObString(val_buf)))) {
 | 
						|
      LOG_WARN("set sysvar schema value failed", KR(ret), K(tenant_id), K(pz_entity_count));
 | 
						|
    } else {
 | 
						|
      sysvar_schema.set_tenant_id(arg.tenant_id_);
 | 
						|
      if (OB_FAIL(arg.sys_var_list_.push_back(sysvar_schema))) {
 | 
						|
        LOG_WARN("store sys var to array failed", KR(ret), K(tenant_id), K(pz_entity_count));
 | 
						|
      } else if (OB_FAIL(common_rpc_->modify_system_variable(arg))) {
 | 
						|
        LOG_WARN("alter system variable failed", KR(ret), K(tenant_id), K(pz_entity_count));
 | 
						|
      } else {
 | 
						|
        LOG_INFO("[UPGRADE] alter _primary_zone_entity_count success", KR(ret), K(tenant_id), K(pz_entity_count));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::get_tenant_primary_zone_entity_count(
 | 
						|
    const uint64_t tenant_id, share::schema::ObSchemaGetterGuard& schema_guard, int64_t& pz_entity_count)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<const ObDatabaseSchema*> database_schemas;
 | 
						|
  common::ObArray<const ObSimpleTableSchemaV2*> table_schemas;
 | 
						|
  common::ObArray<const ObTablegroupSchema*> tablegroup_schemas;
 | 
						|
  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(schema_guard.get_table_schemas_in_tenant(tenant_id, table_schemas))) {
 | 
						|
    LOG_WARN("get table schemas in tenant failed", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_database_schemas_in_tenant(tenant_id, database_schemas))) {
 | 
						|
    LOG_WARN("get database schemas in tenant failed", K(ret));
 | 
						|
  } else if (OB_FAIL(schema_guard.get_tablegroup_schemas_in_tenant(tenant_id, tablegroup_schemas))) {
 | 
						|
    LOG_WARN("get tablegroup schemas in tenant failed", K(ret));
 | 
						|
  } else {
 | 
						|
    pz_entity_count = 0;
 | 
						|
    // table schemas
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); ++i) {
 | 
						|
      const ObSimpleTableSchemaV2* table_schema = table_schemas.at(i);
 | 
						|
      if (NULL == table_schema) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("table schema is null", K(ret));
 | 
						|
      } else if (table_schema->get_primary_zone().empty()) {
 | 
						|
        // go on next
 | 
						|
      } else {
 | 
						|
        ++pz_entity_count;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    // databases
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < database_schemas.count(); ++i) {
 | 
						|
      const ObDatabaseSchema* database_schema = database_schemas.at(i);
 | 
						|
      if (NULL == database_schema) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("database schema is null", K(ret));
 | 
						|
      } else if (database_schema->get_primary_zone().empty()) {
 | 
						|
        // go on next
 | 
						|
      } else {
 | 
						|
        ++pz_entity_count;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    for (int64_t i = 0; OB_SUCC(ret) && i < tablegroup_schemas.count(); ++i) {
 | 
						|
      const ObTablegroupSchema* tablegroup_schema = tablegroup_schemas.at(i);
 | 
						|
      if (NULL == tablegroup_schema) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("database schema is null", K(ret));
 | 
						|
      } else if (tablegroup_schema->get_primary_zone().empty()) {
 | 
						|
        // go on next
 | 
						|
      } else {
 | 
						|
        ++pz_entity_count;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
// Because the system table uses the schema of the system tenant, the system table cannot be cleaned up
 | 
						|
// with the user table using the schema version
 | 
						|
// When the system table is cleaned up, the scope of the schema_version passed in should be
 | 
						|
// the schema version of the system tenant
 | 
						|
int ObDDLService::clear_partition_member_list(
 | 
						|
    const int64_t max_schema_version, const int64_t tenant_id, const bool is_inner_table)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("inner stat error", K(ret));
 | 
						|
  } else if (OB_UNLIKELY(OB_INVALID_VERSION == max_schema_version) || OB_INVALID_TENANT_ID == tenant_id) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("schema version or tenant_id is invalid", K(ret), K(max_schema_version), K(tenant_id));
 | 
						|
  } else {
 | 
						|
    ObSqlString sql;
 | 
						|
    int64_t affected_row = 0;
 | 
						|
    if (OB_FAIL(sql.assign_fmt("delete from %s where tenant_id = %ld and schema_version <= %ld ",
 | 
						|
            OB_ALL_PARTITION_MEMBER_LIST_TNAME,
 | 
						|
            tenant_id,
 | 
						|
            max_schema_version))) {
 | 
						|
      LOG_WARN("failed to assign sql", KR(ret), K(sql), K(tenant_id), K(max_schema_version));
 | 
						|
    } else if (is_inner_table) {
 | 
						|
      if (OB_FAIL(sql.append_fmt("and table_id & 0xffffffffff < %lu", OB_MIN_USER_TABLE_ID))) {
 | 
						|
        LOG_WARN("failed to append sql", KR(ret), K(sql), K(tenant_id), K(max_schema_version), K(is_inner_table));
 | 
						|
      }
 | 
						|
    } else if (OB_FAIL(sql.append_fmt("and table_id & 0xffffffffff > %lu", OB_MIN_USER_TABLE_ID))) {
 | 
						|
      LOG_WARN("failed to append sql", KR(ret), K(sql), K(tenant_id), K(max_schema_version), K(is_inner_table));
 | 
						|
    }
 | 
						|
    if (OB_FAIL(ret)) {
 | 
						|
    } else if (OB_FAIL(sql_proxy_->write(sql.ptr(), affected_row))) {
 | 
						|
      LOG_WARN("failed to write", K(ret), K(sql), K(affected_row));
 | 
						|
    } else if (0 == affected_row) {
 | 
						|
      // no error
 | 
						|
      LOG_WARN("affected row is zero",
 | 
						|
          K(ret),
 | 
						|
          K(sql),
 | 
						|
          K(affected_row),
 | 
						|
          K(is_inner_table),
 | 
						|
          K(tenant_id),
 | 
						|
          K(max_schema_version));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::force_set_locality(ObSchemaGetterGuard& schema_guard, ObTenantSchema& new_tenant)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  common::ObArray<common::ObZone> zones_in_pool;
 | 
						|
  common::ObArray<share::schema::ObZoneRegion> zone_region_list;
 | 
						|
  common::ObArray<share::ObResourcePoolName> resource_pool_names;
 | 
						|
 | 
						|
  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", K(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", K(ret), K(new_tenant));
 | 
						|
  } else {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    trans.set_end_tenant_id(OB_SYS_TENANT_ID);
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed, ", K(ret));
 | 
						|
    } else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant, trans))) {
 | 
						|
      LOG_WARN("failed to alter tenant", 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));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::update_table_schema_version(const ObTableSchema* table_schema)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObTableSchema* new_schema = NULL;
 | 
						|
  ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
 | 
						|
  int64_t new_schema_version = 0;
 | 
						|
  int64_t tenant_id = OB_INVALID_TENANT_ID;
 | 
						|
  int64_t table_id = OB_INVALID_ID;
 | 
						|
  if (OB_FAIL(check_inner_stat())) {
 | 
						|
    LOG_WARN("check inner stat failed", K(ret));
 | 
						|
  } else if (OB_ISNULL(table_schema)) {
 | 
						|
    ret = OB_INVALID_ARGUMENT;
 | 
						|
    LOG_WARN("invalid argument", KR(ret));
 | 
						|
  } else {
 | 
						|
    tenant_id = table_schema->get_tenant_id();
 | 
						|
    table_id = table_schema->get_table_id();
 | 
						|
    if (OB_SYS_TENANT_ID != tenant_id || !is_inner_table(table_id)) {
 | 
						|
      ret = OB_INVALID_ARGUMENT;
 | 
						|
      LOG_WARN("invalid argument, only sys tenant inner table can update schema version",
 | 
						|
          KR(ret),
 | 
						|
          K(tenant_id),
 | 
						|
          K(table_id));
 | 
						|
    }
 | 
						|
  }
 | 
						|
  if (OB_FAIL(ret)) {
 | 
						|
  } else if (OB_FAIL(ObSchemaUtils::alloc_schema(allocator, *table_schema, new_schema))) {
 | 
						|
    LOG_WARN("fail to alloc schema", KR(ret));
 | 
						|
  } else if (OB_ISNULL(new_schema)) {
 | 
						|
    ret = OB_ERR_UNEXPECTED;
 | 
						|
    LOG_WARN("get invalid schema", KR(ret), KP(new_schema));
 | 
						|
  } 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 {
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    new_schema->set_schema_version(new_schema_version);
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } else if (OB_FAIL(ddl_operator.update_table_schema_version(trans, *new_schema))) {
 | 
						|
      LOG_WARN("fail to update table schema version", KR(ret), K(tenant_id), K(table_id));
 | 
						|
    } else {
 | 
						|
      LOG_INFO("update table schema version success", KR(ret), K(tenant_id), K(table_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;
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
int ObDDLService::modify_schema_in_restore(const obrpc::ObRestoreModifySchemaArg& arg)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  ObSchemaGetterGuard schema_guard;
 | 
						|
  const uint64_t tenant_id = arg.exec_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(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 {
 | 
						|
    ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
 | 
						|
    ObDDLSQLTransaction trans(schema_service_);
 | 
						|
    ObString empty_str("");
 | 
						|
    if (OB_FAIL(trans.start(sql_proxy_))) {
 | 
						|
      LOG_WARN("start transaction failed", K(ret));
 | 
						|
    } else if (ObRestoreModifySchemaArg::RESET_DATABASE_PRIMARY_ZONE == arg.type_) {
 | 
						|
      // database
 | 
						|
      const uint64_t database_id = arg.schema_id_;
 | 
						|
      const ObDatabaseSchema* database = NULL;
 | 
						|
      ObDatabaseSchema new_database;
 | 
						|
      if (OB_FAIL(schema_guard.get_database_schema(database_id, database))) {
 | 
						|
        LOG_WARN("fail to get database", K(ret), K(database_id));
 | 
						|
      } else if (OB_ISNULL(database)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("database not exist", K(ret), K(database_id));
 | 
						|
      } else if (OB_FAIL(new_database.assign(*database))) {
 | 
						|
        LOG_WARN("fail to assign new database", K(ret), K(database_id));
 | 
						|
      } else if (OB_FAIL(new_database.set_primary_zone(empty_str))) {
 | 
						|
        LOG_WARN("fail to reset primary_zone", K(ret), K(database_id));
 | 
						|
      } else if (OB_FAIL(ddl_operator.alter_database(new_database, trans, OB_DDL_ALTER_DATABASE))) {
 | 
						|
        LOG_WARN("fail to alter database", K(ret), K(database_id));
 | 
						|
      }
 | 
						|
    } else if (ObRestoreModifySchemaArg::RESET_TABLEGROUP_PRIMARY_ZONE <= arg.type_ &&
 | 
						|
               ObRestoreModifySchemaArg::RESET_TABLEGROUP_PREVIOUS_LOCALITY >= arg.type_) {
 | 
						|
      // tablegroup
 | 
						|
      const uint64_t tablegroup_id = arg.schema_id_;
 | 
						|
      const ObTablegroupSchema* tablegroup = NULL;
 | 
						|
      ObTablegroupSchema new_tablegroup;
 | 
						|
      if (OB_FAIL(schema_guard.get_tablegroup_schema(tablegroup_id, tablegroup))) {
 | 
						|
        LOG_WARN("fail to get tablegroup", K(ret), K(tablegroup_id));
 | 
						|
      } else if (OB_ISNULL(tablegroup)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("tablegroup not exist", K(ret), K(tablegroup_id));
 | 
						|
      } else if (OB_FAIL(new_tablegroup.assign(*tablegroup))) {
 | 
						|
        LOG_WARN("fail to assign new tablegroup", K(ret), K(tablegroup_id));
 | 
						|
      } else {
 | 
						|
        if (ObRestoreModifySchemaArg::RESET_TABLEGROUP_PRIMARY_ZONE == arg.type_) {
 | 
						|
          if (OB_FAIL(new_tablegroup.set_primary_zone(empty_str))) {
 | 
						|
            LOG_WARN("fail to reset primary_zone", K(ret), K(tablegroup_id));
 | 
						|
          }
 | 
						|
        } else if (ObRestoreModifySchemaArg::RESET_TABLEGROUP_LOCALITY == arg.type_) {
 | 
						|
          if (OB_FAIL(new_tablegroup.set_locality(empty_str))) {
 | 
						|
            LOG_WARN("fail to reset locality", K(ret), K(tablegroup_id));
 | 
						|
          }
 | 
						|
        } else if (ObRestoreModifySchemaArg::RESET_TABLEGROUP_PREVIOUS_LOCALITY == arg.type_) {
 | 
						|
          if (OB_FAIL(new_tablegroup.set_previous_locality(empty_str))) {
 | 
						|
            LOG_WARN("fail to reset previous_locality", K(ret), K(tablegroup_id));
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("invalid type", K(ret), K(arg));
 | 
						|
        }
 | 
						|
        if (OB_FAIL(ret)) {
 | 
						|
        } else if (OB_FAIL(ddl_operator.alter_tablegroup(new_tablegroup, trans))) {
 | 
						|
          LOG_WARN("fail to alter tablegroup", K(ret), K(tablegroup_id));
 | 
						|
        }
 | 
						|
      }
 | 
						|
    } else if (ObRestoreModifySchemaArg::RESET_TABLE_PRIMARY_ZONE <= arg.type_ &&
 | 
						|
               ObRestoreModifySchemaArg::RESET_TABLE_PREVIOUS_LOCALITY >= arg.type_) {
 | 
						|
      // table
 | 
						|
      const uint64_t table_id = arg.schema_id_;
 | 
						|
      const ObTableSchema* table = NULL;
 | 
						|
      ObTableSchema new_table;
 | 
						|
      if (OB_FAIL(schema_guard.get_table_schema(table_id, table))) {
 | 
						|
        LOG_WARN("fail to get table", K(ret), K(table_id));
 | 
						|
      } else if (OB_ISNULL(table)) {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("table not exist", K(ret), K(table_id));
 | 
						|
      } else if (OB_FAIL(new_table.assign(*table))) {
 | 
						|
        LOG_WARN("fail to assign new table", K(ret), K(table_id));
 | 
						|
      } else {
 | 
						|
        if (ObRestoreModifySchemaArg::RESET_TABLE_PRIMARY_ZONE == arg.type_) {
 | 
						|
          if (OB_FAIL(new_table.set_primary_zone(empty_str))) {
 | 
						|
            LOG_WARN("fail to reset primary_zone", K(ret), K(table_id));
 | 
						|
          }
 | 
						|
        } else if (ObRestoreModifySchemaArg::RESET_TABLE_LOCALITY == arg.type_) {
 | 
						|
          if (OB_FAIL(new_table.set_locality(empty_str))) {
 | 
						|
            LOG_WARN("fail to reset locality", K(ret), K(table_id));
 | 
						|
          }
 | 
						|
        } else if (ObRestoreModifySchemaArg::RESET_TABLE_PREVIOUS_LOCALITY == arg.type_) {
 | 
						|
          if (OB_FAIL(new_table.set_previous_locality(empty_str))) {
 | 
						|
            LOG_WARN("fail to reset previous_locality", K(ret), K(table_id));
 | 
						|
          }
 | 
						|
        } else {
 | 
						|
          ret = OB_ERR_UNEXPECTED;
 | 
						|
          LOG_WARN("invalid type", K(ret), K(arg));
 | 
						|
        }
 | 
						|
        if (OB_FAIL(ret)) {
 | 
						|
        } else if (OB_FAIL(ddl_operator.alter_table_options(
 | 
						|
                       schema_guard, new_table, *table, false /*update_index_table*/, trans))) {
 | 
						|
          LOG_WARN("fail to alter table", K(ret), K(table_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;
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      if (OB_FAIL(publish_schema(tenant_id))) {
 | 
						|
        LOG_WARN("publish schema failed", K(ret));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return ret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObDDLService::is_sync_primary_ddl()
 | 
						|
{
 | 
						|
  bool bret = false;
 | 
						|
  int tmp_ret = OB_SUCCESS;
 | 
						|
  bool is_standby = false;
 | 
						|
  if (OB_SUCCESS != (tmp_ret = check_inner_stat())) {
 | 
						|
    LOG_WARN("inner stat error", K(tmp_ret));
 | 
						|
  } else {
 | 
						|
    share::schema::ObSchemaService* schema_service = schema_service_->get_schema_service();
 | 
						|
    if (OB_ISNULL(schema_service)) {
 | 
						|
      tmp_ret = OB_ERR_UNEXPECTED;
 | 
						|
      LOG_WARN("schema service is null", K(tmp_ret), KP(schema_service), KP(schema_service_));
 | 
						|
    } else if (OB_SUCCESS != (tmp_ret = get_is_standby_cluster(is_standby))) {
 | 
						|
      LOG_WARN("failed to get is stanbdy cluster", K(tmp_ret));
 | 
						|
    } else {
 | 
						|
      bret = schema_service->is_sync_primary_ddl() && is_standby;
 | 
						|
    }
 | 
						|
  }
 | 
						|
  return bret;
 | 
						|
}
 | 
						|
 | 
						|
bool ObDDLService::add_sys_table_index(const uint64_t table_id, common::ObIArray<share::schema::ObTableSchema>& schemas)
 | 
						|
{
 | 
						|
  int ret = OB_SUCCESS;
 | 
						|
  uint64_t pure_id = extract_pure_id(table_id);
 | 
						|
  uint64_t tenant_id = extract_tenant_id(table_id);
 | 
						|
  if (OB_ALL_TABLE_V2_HISTORY_TID == pure_id || OB_ALL_TABLE_HISTORY_TID == pure_id ||
 | 
						|
      OB_ALL_BACKUP_PIECE_FILES_TID == pure_id) {
 | 
						|
    ObTableSchema index_schema;
 | 
						|
    switch (pure_id) {
 | 
						|
      case OB_ALL_TABLE_HISTORY_TID: {
 | 
						|
        if (OB_FAIL(ObInnerTableSchema::all_table_history_idx_data_table_id_schema(index_schema))) {
 | 
						|
          LOG_WARN("fail to create index schema", K(ret), K(index_schema));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case OB_ALL_TABLE_V2_HISTORY_TID: {
 | 
						|
        if (OB_FAIL(ObInnerTableSchema::all_table_v2_history_idx_data_table_id_schema(index_schema))) {
 | 
						|
          LOG_WARN("fail to create index schema", K(ret), K(index_schema));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      case OB_ALL_BACKUP_PIECE_FILES_TID: {
 | 
						|
        if (OB_FAIL(ObInnerTableSchema::all_backup_piece_files_idx_data_table_id_schema(index_schema))) {
 | 
						|
          LOG_WARN("fail to create index schema", K(ret), K(index_schema));
 | 
						|
        }
 | 
						|
        break;
 | 
						|
      }
 | 
						|
      default: {
 | 
						|
        ret = OB_ERR_UNEXPECTED;
 | 
						|
        LOG_WARN("invalid table_id", K(ret), K(table_id));
 | 
						|
      }
 | 
						|
    }
 | 
						|
    if (OB_SUCC(ret)) {
 | 
						|
      index_schema.set_tenant_id(tenant_id);
 | 
						|
      index_schema.set_table_id(combine_id(tenant_id, index_schema.get_table_id()));
 | 
						|
      index_schema.set_database_id(combine_id(tenant_id, index_schema.get_database_id()));
 | 
						|
      if (index_schema.get_tablegroup_id() != OB_INVALID_ID) {
 | 
						|
        index_schema.set_tablegroup_id(combine_id(tenant_id, index_schema.get_tablegroup_id()));
 | 
						|
      }
 | 
						|
      if (index_schema.get_data_table_id() > 0) {
 | 
						|
        index_schema.set_data_table_id(combine_id(tenant_id, index_schema.get_data_table_id()));
 | 
						|
      }
 | 
						|
      if (OB_FAIL(schemas.push_back(index_schema))) {
 | 
						|
        LOG_WARN("fail to push back", K(ret), K(index_schema));
 | 
						|
      }
 | 
						|
    }
 | 
						|
  }
 | 
						|
  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;
 | 
						|
}
 | 
						|
 | 
						|
}  // end namespace rootserver
 | 
						|
}  // end namespace oceanbase
 |